From d50d5bd56b20c3065ce6f2a004a51c5dd7a6b3e5 Mon Sep 17 00:00:00 2001 From: AydarZaynutdinov Date: Tue, 28 Dec 2021 17:25:47 +0300 Subject: [PATCH 1/5] [BEAM-13398][Playground] Add LifeCycleDto to separate DTO and business logic. --- .../code_processing/code_processing.go | 18 +- .../code_processing/code_processing_test.go | 10 +- playground/backend/internal/fs_tool/fs.go | 109 ++-- .../backend/internal/fs_tool/fs_test.go | 489 ++++++++++++------ .../backend/internal/fs_tool/go_fs_test.go | 32 +- .../backend/internal/fs_tool/java_fs.go | 2 +- .../backend/internal/fs_tool/java_fs_test.go | 34 +- .../internal/fs_tool/lc_constructor.go | 40 +- .../internal/fs_tool/python_fs_test.go | 32 +- .../preparators/java_preparators_test.go | 12 +- .../setup_tools/builder/setup_builder.go | 22 +- .../setup_tools/builder/setup_builder_test.go | 16 +- .../life_cycle/life_cycle_setuper.go | 18 +- .../life_cycle/life_cycle_setuper_test.go | 8 +- 14 files changed, 518 insertions(+), 324 deletions(-) diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 491a87708faf..d0f4cc75c90e 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -69,7 +69,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl go cancelCheck(pipelineLifeCycleCtx, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) + executorBuilder, err := builder.SetupExecutorBuilder(lc.Dto, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) if err != nil { _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) return @@ -122,7 +122,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl validateIsUnitTest, _ := validationResults.Load(validators.UnitTestValidatorName) isUnitTest := validateIsUnitTest.(bool) - // This condition is used for cases when the playground doesn't compile source files. For the Python code and the Go Unit Tests + // This condition is used for cases when the playground doesn't compile source files. For the Python code and the Go Unit Tests if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_PYTHON || (sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && isUnitTest) { if err := processCompileSuccess(pipelineLifeCycleCtx, []byte(""), pipelineId, cacheService); err != nil { return @@ -131,7 +131,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Compile if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { executor = executorBuilder.WithCompiler(). - WithFileName(builder.GetFileNameFromFolder(lc.GetAbsoluteSourceFolderPath())).Build() // Need changed name for unit tests + WithFileName(builder.GetFileNameFromFolder(lc.Dto.GetAbsoluteSourceFileFolderPath())).Build() // Need changed name for unit tests } logger.Infof("%s: Compile() ...\n", pipelineId) compileCmd := executor.Compile(pipelineLifeCycleCtx) @@ -155,7 +155,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Run if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { - executor, err = setJavaExecutableFile(lc, pipelineId, cacheService, pipelineLifeCycleCtx, executorBuilder, appEnv.WorkingDir()) + executor, err = setJavaExecutableFile(lc.Dto, pipelineId, cacheService, pipelineLifeCycleCtx, executorBuilder, appEnv.WorkingDir()) if err != nil { return } @@ -164,11 +164,11 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl runCmd := getExecuteCmd(&validationResults, &executor, pipelineLifeCycleCtx) var runError bytes.Buffer runOutput := streaming.RunOutputWriter{Ctx: pipelineLifeCycleCtx, CacheService: cacheService, PipelineId: pipelineId} - go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, lc.GetAbsoluteLogFilePath(), pipelineId, stopReadLogsChannel, finishReadLogsChannel) + go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, lc.Dto.GetAbsoluteLogFilePath(), pipelineId, stopReadLogsChannel, finishReadLogsChannel) if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO { // For go SDK all logs are placed to stdErr. - file, err := os.Create(lc.GetAbsoluteLogFilePath()) + file, err := os.Create(lc.Dto.GetAbsoluteLogFilePath()) if err != nil { // If some error with creating a log file do the same as with other SDK. logger.Errorf("%s: error during create log file (go sdk): %s", pipelineId, err.Error()) @@ -194,7 +194,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Run step is finished, but code contains some error (divide by 0 for example) if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO { // For Go SDK stdErr was redirected to the log file. - errData, err := os.ReadFile(lc.GetAbsoluteLogFilePath()) + errData, err := os.ReadFile(lc.Dto.GetAbsoluteLogFilePath()) if err != nil { logger.Errorf("%s: error during read errors from log file (go sdk): %s", pipelineId, err.Error()) } @@ -219,8 +219,8 @@ func getExecuteCmd(valRes *sync.Map, executor *executors.Executor, ctxWithTimeou } // setJavaExecutableFile sets executable file name to runner (JAVA class name is known after compilation step) -func setJavaExecutableFile(lc *fs_tool.LifeCycle, id uuid.UUID, service cache.Cache, ctx context.Context, executorBuilder *executors.ExecutorBuilder, dir string) (executors.Executor, error) { - className, err := lc.ExecutableName(id, dir) +func setJavaExecutableFile(lcDto fs_tool.LifeCycleDTO, id uuid.UUID, service cache.Cache, ctx context.Context, executorBuilder *executors.ExecutorBuilder, dir string) (executors.Executor, error) { + className, err := lcDto.ExecutableName(id, dir) if err != nil { if err = processSetupError(err, id, service, ctx); err != nil { return executorBuilder.Build(), err diff --git a/playground/backend/internal/code_processing/code_processing_test.go b/playground/backend/internal/code_processing/code_processing_test.go index 7c7184ddb07a..0c6df550dfef 100644 --- a/playground/backend/internal/code_processing/code_processing_test.go +++ b/playground/backend/internal/code_processing/code_processing_test.go @@ -248,7 +248,7 @@ func Test_Process(t *testing.T) { t.Fatalf("error during prepare folders: %s", err.Error()) } if tt.createExecFile { - _, _ = lc.CreateSourceCodeFile(tt.code) + _ = lc.CreateSourceCodeFile(tt.code) } if err = utils.SetToCache(tt.args.ctx, cacheService, tt.args.pipelineId, cache.Canceled, false); err != nil { t.Fatal("error during set cancel flag to cache") @@ -269,7 +269,7 @@ func Test_Process(t *testing.T) { compileOutput, _ := cacheService.GetValue(tt.args.ctx, tt.args.pipelineId, cache.CompileOutput) if tt.expectedCompileOutput != nil && strings.Contains(tt.expectedCompileOutput.(string), "%s") { - tt.expectedCompileOutput = fmt.Sprintf(tt.expectedCompileOutput.(string), lc.GetAbsoluteSourceFilePath()) + tt.expectedCompileOutput = fmt.Sprintf(tt.expectedCompileOutput.(string), lc.Dto.GetAbsoluteSourceFilePath()) } if !reflect.DeepEqual(compileOutput, tt.expectedCompileOutput) { t.Errorf("processCode() set compileOutput: %s, but expectes: %s", compileOutput, tt.expectedCompileOutput) @@ -535,7 +535,7 @@ func TestGetLastIndex(t *testing.T) { func Test_setJavaExecutableFile(t *testing.T) { pipelineId := uuid.New() lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, pipelineId, os.Getenv("APP_WORK_DIR")) - lc.ExecutableName = fakeExecutableName + lc.Dto.ExecutableName = fakeExecutableName executorBuilder := executors.NewExecutorBuilder().WithRunner().WithCommand("fake cmd").ExecutorBuilder type args struct { lc *fs_tool.LifeCycle @@ -572,7 +572,7 @@ func Test_setJavaExecutableFile(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := setJavaExecutableFile(tt.args.lc, tt.args.id, tt.args.service, tt.args.ctx, tt.args.executorBuilder, tt.args.dir) + got, err := setJavaExecutableFile(tt.args.lc.Dto, tt.args.id, tt.args.service, tt.args.ctx, tt.args.executorBuilder, tt.args.dir) if (err != nil) != tt.wantErr { t.Errorf("setJavaExecutableFile() error = %v, wantErr %v", err, tt.wantErr) } @@ -692,7 +692,7 @@ func prepareFiles(b *testing.B, pipelineId uuid.UUID, code string, sdk pb.Sdk) * if err != nil { b.Fatalf("error during prepare folders: %s", err.Error()) } - _, err = lc.CreateSourceCodeFile(code) + err = lc.CreateSourceCodeFile(code) if err != nil { b.Fatalf("error during prepare source code file: %s", err.Error()) } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index c1cb75a97598..3d0c11a50bd2 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -45,14 +45,65 @@ type Extension struct { ExecutableFileExtension string } -// LifeCycle is used for preparing folders and files to process code for one request. -// For each SDK folders (Folder) and extensions (Extension) should be set correctly. -type LifeCycle struct { - folderGlobs []string //folders that should be created to process code +// LifeCycleDTO contains methods to receive files/folders paths +type LifeCycleDTO struct { + PipelineId uuid.UUID Folder Folder Extension Extension ExecutableName func(uuid.UUID, string) (string, error) - pipelineId uuid.UUID +} + +// GetSourceFileName returns name of the source file like {pipelineId}.{sourceFileExtension}. +func (l *LifeCycleDTO) GetSourceFileName() string { + return l.PipelineId.String() + l.Extension.SourceFileExtension +} + +// GetAbsoluteSourceFileFolderPath returns absolute filepath to source folder (/path/to/workingDir/executable_files/{pipelineId}/src). +func (l *LifeCycleDTO) GetAbsoluteSourceFileFolderPath() string { + absolutePath, _ := filepath.Abs(l.Folder.SourceFileFolder) + return absolutePath +} + +// GetAbsoluteSourceFilePath returns absolute filepath to source file (/path/to/workingDir/executable_files/{pipelineId}/src/{pipelineId}.{sourceFileExtension}). +func (l *LifeCycleDTO) GetAbsoluteSourceFilePath() string { + absolutePath, _ := filepath.Abs(filepath.Join(l.GetAbsoluteSourceFileFolderPath(), l.GetSourceFileName())) + return absolutePath +} + +// GetExecutableFileName returns name of the executable file like {pipelineId}.{executableFileExtension}. +func (l *LifeCycleDTO) GetExecutableFileName() string { + return l.PipelineId.String() + l.Extension.ExecutableFileExtension +} + +// GetAbsoluteExecutableFileFolderPath returns absolute filepath to executable folder (/path/to/workingDir/executable_files/{pipelineId}/bin). +func (l *LifeCycleDTO) GetAbsoluteExecutableFileFolderPath() string { + absolutePath, _ := filepath.Abs(l.Folder.ExecutableFileFolder) + return absolutePath +} + +// GetAbsoluteExecutableFilePath returns absolute filepath to executable file (/path/to/workingDir/executable_files/{pipelineId}/bin/{pipelineId}.{executableFileExtension}). +func (l *LifeCycleDTO) GetAbsoluteExecutableFilePath() string { + absolutePath, _ := filepath.Abs(filepath.Join(l.GetAbsoluteExecutableFileFolderPath(), l.GetExecutableFileName())) + return absolutePath +} + +// GetAbsoluteBaseFolderPath returns absolute path to executable folder (/path/to/workingDir/executable_files/{pipelineId}). +func (l *LifeCycleDTO) GetAbsoluteBaseFolderPath() string { + absolutePath, _ := filepath.Abs(l.Folder.BaseFolder) + return absolutePath +} + +// GetAbsoluteLogFilePath returns absolute path to the logs file (/path/to/workingDir/executable_files/{pipelineId}/logs.log) +func (l *LifeCycleDTO) GetAbsoluteLogFilePath() string { + filePath := filepath.Join(l.GetAbsoluteBaseFolderPath(), logFileName) + absoluteFilePath, _ := filepath.Abs(filePath) + return absoluteFilePath +} + +// LifeCycle is used for preparing folders and files to process code for one code processing request. +type LifeCycle struct { + folderGlobs []string //folders that should be created to process code + Dto LifeCycleDTO } // NewLifeCycle returns a corresponding LifeCycle depending on the given SDK. @@ -93,26 +144,17 @@ func (l *LifeCycle) DeleteFolders() error { } // CreateSourceCodeFile creates an executable file (i.e. file.{sourceFileExtension}). -func (l *LifeCycle) CreateSourceCodeFile(code string) (string, error) { - if _, err := os.Stat(l.Folder.SourceFileFolder); os.IsNotExist(err) { - return "", err +func (l *LifeCycle) CreateSourceCodeFile(code string) error { + if _, err := os.Stat(l.Dto.GetAbsoluteSourceFileFolderPath()); os.IsNotExist(err) { + return err } - fileName := l.pipelineId.String() + l.Extension.SourceFileExtension - filePath := filepath.Join(l.Folder.SourceFileFolder, fileName) + filePath := l.Dto.GetAbsoluteSourceFilePath() err := os.WriteFile(filePath, []byte(code), fileMode) if err != nil { - return "", err + return err } - return fileName, nil -} - -// GetAbsoluteSourceFilePath returns absolute filepath to executable file (/path/to/workingDir/executable_files/{pipelineId}/src/{pipelineId}.{sourceFileExtension}). -func (l *LifeCycle) GetAbsoluteSourceFilePath() string { - fileName := l.pipelineId.String() + l.Extension.SourceFileExtension - filePath := filepath.Join(l.Folder.SourceFileFolder, fileName) - absoluteFilePath, _ := filepath.Abs(filePath) - return absoluteFilePath + return nil } // CopyFile copies a file with fileName from sourceDir to destinationDir. @@ -145,30 +187,3 @@ func (l *LifeCycle) CopyFile(fileName, sourceDir, destinationDir string) error { } return nil } - -// GetAbsoluteExecutableFilePath returns absolute filepath to compiled file (/path/to/workingDir/executable_files/{pipelineId}/bin/{pipelineId}.{executableExtension}). -func (l *LifeCycle) GetAbsoluteExecutableFilePath() string { - fileName := l.pipelineId.String() + l.Extension.ExecutableFileExtension - filePath := filepath.Join(l.Folder.ExecutableFileFolder, fileName) - absoluteFilePath, _ := filepath.Abs(filePath) - return absoluteFilePath -} - -// GetAbsoluteBaseFolderPath returns absolute path to executable folder (/path/to/workingDir/executable_files/{pipelineId}). -func (l *LifeCycle) GetAbsoluteBaseFolderPath() string { - absoluteFilePath, _ := filepath.Abs(l.Folder.BaseFolder) - return absoluteFilePath -} - -// GetAbsoluteLogFilePath returns absolute path to the logs file (/path/to/workingDir/executable_files/{pipelineId}/logs.log) -func (l *LifeCycle) GetAbsoluteLogFilePath() string { - filePath := filepath.Join(l.Folder.BaseFolder, logFileName) - absoluteFilePath, _ := filepath.Abs(filePath) - return absoluteFilePath -} - -// GetAbsoluteSourceFolderPath returns absolute path to executable folder (/path/to/workingDir/executable_files/{pipelineId}/src). -func (l *LifeCycle) GetAbsoluteSourceFolderPath() string { - absoluteFilePath, _ := filepath.Abs(l.Folder.SourceFileFolder) - return absoluteFilePath -} diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index 4a20c00a6151..d545e79437e8 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -69,7 +69,7 @@ func teardown() { } } -func TestLifeCycle_CreateExecutableFile(t *testing.T) { +func TestLifeCycle_CreateSourceCodeFile(t *testing.T) { pipelineId := uuid.New() baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) srcFileFolder := baseFileFolder + "/src" @@ -114,7 +114,6 @@ func TestLifeCycle_CreateExecutableFile(t *testing.T) { pipelineId: pipelineId, }, args: args{code: "TEST_CODE"}, - want: pipelineId.String() + JavaSourceFileExtension, wantErr: false, }, } @@ -125,18 +124,17 @@ func TestLifeCycle_CreateExecutableFile(t *testing.T) { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Folder: tt.fields.folder, - Extension: tt.fields.extension, - pipelineId: tt.fields.pipelineId, + Dto: LifeCycleDTO{ + Folder: tt.fields.folder, + Extension: tt.fields.extension, + PipelineId: tt.fields.pipelineId, + }, } - got, err := l.CreateSourceCodeFile(tt.args.code) + err := l.CreateSourceCodeFile(tt.args.code) if (err != nil) != tt.wantErr { t.Errorf("CreateSourceCodeFile() error = %v, wantErr %v", err, tt.wantErr) return } - if got != tt.want { - t.Errorf("CreateSourceCodeFile() got = %v, want %v", got, tt.want) - } }) os.RemoveAll(baseFileFolder) } @@ -167,9 +165,11 @@ func TestLifeCycle_CreateFolders(t *testing.T) { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Folder: tt.fields.folder, - Extension: tt.fields.extension, - pipelineId: tt.fields.pipelineId, + Dto: LifeCycleDTO{ + Folder: tt.fields.folder, + Extension: tt.fields.extension, + PipelineId: tt.fields.pipelineId, + }, } if err := l.CreateFolders(); (err != nil) != tt.wantErr { t.Errorf("CreateFolders() error = %v, wantErr %v", err, tt.wantErr) @@ -212,9 +212,11 @@ func TestLifeCycle_DeleteFolders(t *testing.T) { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Folder: tt.fields.folder, - Extension: tt.fields.extension, - pipelineId: tt.fields.pipelineId, + Dto: LifeCycleDTO{ + Folder: tt.fields.folder, + Extension: tt.fields.extension, + PipelineId: tt.fields.pipelineId, + }, } if err := l.DeleteFolders(); (err != nil) != tt.wantErr { t.Errorf("DeleteFolders() error = %v, wantErr %v", err, tt.wantErr) @@ -250,17 +252,19 @@ func TestNewLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - Extension: Extension{ - SourceFileExtension: JavaSourceFileExtension, - ExecutableFileExtension: javaCompiledFileExtension, + Dto: LifeCycleDTO{ + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: srcFileFolder, + ExecutableFileFolder: binFileFolder, + }, + Extension: Extension{ + SourceFileExtension: JavaSourceFileExtension, + ExecutableFileExtension: javaCompiledFileExtension, + }, + ExecutableName: executableName, + PipelineId: pipelineId, }, - ExecutableName: executableName, - pipelineId: pipelineId, }, wantErr: false, }, @@ -285,237 +289,402 @@ func TestNewLifeCycle(t *testing.T) { if !tt.wantErr && !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("NewLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !tt.wantErr && !reflect.DeepEqual(got.Folder, tt.want.Folder) { - t.Errorf("NewLifeCycle() Folder = %v, want %v", got.Folder, tt.want.Folder) + if !tt.wantErr && !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { + t.Errorf("NewLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) } - if !tt.wantErr && !reflect.DeepEqual(got.Extension, tt.want.Extension) { - t.Errorf("NewLifeCycle() Extension = %v, want %v", got.Extension, tt.want.Extension) + if !tt.wantErr && !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { + t.Errorf("NewLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) } - if !tt.wantErr && !reflect.DeepEqual(got.pipelineId, tt.want.pipelineId) { - t.Errorf("NewLifeCycle() pipelineId = %v, want %v", got.pipelineId, tt.want.pipelineId) + if !tt.wantErr && !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { + t.Errorf("NewLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) } }) } } -func TestLifeCycle_GetAbsoluteExecutableFilePath(t *testing.T) { - pipelineId := uuid.New() - baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) - srcFileFolder := baseFileFolder + "/src" - - filePath := filepath.Join(srcFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), JavaSourceFileExtension)) - absolutePath, _ := filepath.Abs(filePath) +func TestLifeCycle_CopyFile(t *testing.T) { type fields struct { - folderGlobs []string - Folder Folder - Extension Extension - pipelineId uuid.UUID + folderGlobs []string + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + pipelineId uuid.UUID + } + type args struct { + fileName string + sourceDir string + destinationDir string } tests := []struct { name string fields fields - want string + args args wantErr bool }{ { - name: "GetAbsoluteSourceFilePath", + name: "file doesn't exist", fields: fields{ - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - }, - Extension: Extension{SourceFileExtension: JavaSourceFileExtension}, - pipelineId: pipelineId, + folderGlobs: nil, + Folder: Folder{}, + Extension: Extension{}, + ExecutableName: nil, + pipelineId: uuid.UUID{}, }, - want: absolutePath, + args: args{ + fileName: "file1.txt", + sourceDir: sourceDir, + destinationDir: destinationDir, + }, + wantErr: true, + }, + { + name: "file exists", + fields: fields{ + folderGlobs: nil, + Folder: Folder{}, + Extension: Extension{}, + ExecutableName: nil, + pipelineId: uuid.UUID{}, + }, + args: args{ + fileName: "file.txt", + sourceDir: sourceDir, + destinationDir: destinationDir, + }, + wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - pipelineId: tt.fields.pipelineId, + Dto: LifeCycleDTO{ + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + PipelineId: tt.fields.pipelineId, + }, } - got := l.GetAbsoluteSourceFilePath() - if got != tt.want { - t.Errorf("GetAbsoluteSourceFilePath() got = %v, want %v", got, tt.want) + err := l.CopyFile(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir) + if (err != nil) != tt.wantErr { + t.Errorf("CopyFile() error = %v, wantErr %v", err, tt.wantErr) + } + if err == nil && !tt.wantErr { + newFilePath := filepath.Join(destinationDir, tt.args.fileName) + _, err = os.Stat(newFilePath) + if os.IsNotExist(err) { + t.Errorf("CopyFile() should create a new file: %s", newFilePath) + } + } + }) + } +} + +func TestLifeCycleDTO_GetAbsoluteBaseFolderPath(t *testing.T) { + baseFolder := "baseFolder" + absoluteBaseFileFolder, _ := filepath.Abs(baseFolder) + + type fields struct { + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "GetAbsoluteBaseFolderPath", + fields: fields{Folder: Folder{BaseFolder: baseFolder}}, + want: absoluteBaseFileFolder, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + } + if got := l.GetAbsoluteBaseFolderPath(); got != tt.want { + t.Errorf("GetAbsoluteBaseFolderPath() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestLifeCycleDTO_GetAbsoluteExecutableFileFolderPath(t *testing.T) { + executableFileFolder := "executableFileFolder" + absoluteExecutableFileFolder, _ := filepath.Abs(executableFileFolder) + + type fields struct { + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "GetAbsoluteExecutableFileFolderPath", + fields: fields{Folder: Folder{ExecutableFileFolder: executableFileFolder}}, + want: absoluteExecutableFileFolder, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + } + if got := l.GetAbsoluteExecutableFileFolderPath(); got != tt.want { + t.Errorf("GetAbsoluteExecutableFileFolderPath() = %v, want %v", got, tt.want) } }) } } -func TestLifeCycle_GetAbsoluteExecutableFilesFolderPath(t *testing.T) { +func TestLifeCycleDTO_GetAbsoluteExecutableFilePath(t *testing.T) { pipelineId := uuid.New() - baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) + executableFileFolder := "executableFileFolder" + executableFileExtension := ".executableFileExtension" + absoluteExecutableFilePath, _ := filepath.Abs(filepath.Join(executableFileFolder, pipelineId.String()+executableFileExtension)) - absolutePath, _ := filepath.Abs(baseFileFolder) type fields struct { - folderGlobs []string - Folder Folder - Extension Extension - pipelineId uuid.UUID + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) } tests := []struct { - name string - fields fields - want string - wantErr bool + name string + fields fields + want string }{ { - name: "GetAbsoluteExecutableFolderPath", + name: "GetAbsoluteExecutableFilePath", fields: fields{ - Folder: Folder{BaseFolder: baseFileFolder}, - Extension: Extension{SourceFileExtension: JavaSourceFileExtension}, - pipelineId: pipelineId, + PipelineId: pipelineId, + Folder: Folder{ExecutableFileFolder: executableFileFolder}, + Extension: Extension{ExecutableFileExtension: executableFileExtension}, }, - want: absolutePath, + want: absoluteExecutableFilePath, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - l := &LifeCycle{ - folderGlobs: tt.fields.folderGlobs, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - pipelineId: tt.fields.pipelineId, + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, } - got := l.GetAbsoluteBaseFolderPath() - if got != tt.want { - t.Errorf("GetAbsoluteBaseFolderPath() got = %v, want %v", got, tt.want) + if got := l.GetAbsoluteExecutableFilePath(); got != tt.want { + t.Errorf("GetAbsoluteExecutableFilePath() = %v, want %v", got, tt.want) } }) } } -func TestLifeCycle_ExecutableName(t *testing.T) { - pipelineId := uuid.New() - workingDir := "workingDir" - baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) - binFileFolder := baseFileFolder + "/bin" +func TestLifeCycleDTO_GetAbsoluteLogFilePath(t *testing.T) { + baseFileFolder := "baseFileFolder" + absoluteLogFilePath, _ := filepath.Abs(filepath.Join(baseFileFolder, logFileName)) type fields struct { - folderGlobs []string + PipelineId uuid.UUID Folder Folder Extension Extension ExecutableName func(uuid.UUID, string) (string, error) - pipelineId uuid.UUID } tests := []struct { - name string - fields fields - want string - wantErr bool + name string + fields fields + want string }{ { - name: "ExecutableName", + name: "GetAbsoluteLogFilePath", fields: fields{ - Folder: Folder{ - BaseFolder: baseFileFolder, - ExecutableFileFolder: binFileFolder, - }, - ExecutableName: func(u uuid.UUID, s string) (string, error) { - return "MOCK_EXECUTABLE_NAME", nil - }, - pipelineId: pipelineId, - folderGlobs: []string{baseFileFolder, binFileFolder}, + Folder: Folder{BaseFolder: baseFileFolder}, }, - want: "MOCK_EXECUTABLE_NAME", - wantErr: false, + want: absoluteLogFilePath, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - l := &LifeCycle{ - folderGlobs: tt.fields.folderGlobs, + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, Folder: tt.fields.Folder, Extension: tt.fields.Extension, ExecutableName: tt.fields.ExecutableName, - pipelineId: tt.fields.pipelineId, - } - got, err := l.ExecutableName(pipelineId, workingDir) - if got != tt.want { - t.Errorf("GetExecutableName() got = %v, want %v", got, tt.want) } - if (err != nil) != tt.wantErr { - t.Errorf("GetExecutableName() error = %v, wantErr %v", err, tt.wantErr) + if got := l.GetAbsoluteLogFilePath(); got != tt.want { + t.Errorf("GetAbsoluteLogFilePath() = %v, want %v", got, tt.want) } }) } } -func TestCopyFile(t *testing.T) { +func TestLifeCycleDTO_GetAbsoluteSourceFileFolderPath(t *testing.T) { + sourceFileFolder := "sourceFileFolder" + absoluteSourceFileFolder, _ := filepath.Abs(sourceFileFolder) + type fields struct { - folderGlobs []string + PipelineId uuid.UUID Folder Folder Extension Extension ExecutableName func(uuid.UUID, string) (string, error) - pipelineId uuid.UUID - } - type args struct { - fileName string - sourceDir string - destinationDir string } tests := []struct { - name string - fields fields - args args - wantErr bool + name string + fields fields + want string }{ { - name: "file doesn't exist", + name: "GetAbsoluteSourceFileFolderPath", fields: fields{ - folderGlobs: nil, - Folder: Folder{}, - Extension: Extension{}, - ExecutableName: nil, - pipelineId: uuid.UUID{}, - }, - args: args{ - fileName: "file1.txt", - sourceDir: sourceDir, - destinationDir: destinationDir, + Folder: Folder{SourceFileFolder: sourceFileFolder}, }, - wantErr: true, + want: absoluteSourceFileFolder, }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + } + if got := l.GetAbsoluteSourceFileFolderPath(); got != tt.want { + t.Errorf("GetAbsoluteSourceFileFolderPath() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestLifeCycleDTO_GetAbsoluteSourceFilePath(t *testing.T) { + pipelineId := uuid.New() + sourceFileFolder := "sourceFileFolder" + sourceFileExtension := ".sourceFileExtension" + absoluteSourceFilePath, _ := filepath.Abs(filepath.Join(sourceFileFolder, pipelineId.String()+sourceFileExtension)) + + type fields struct { + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + } + tests := []struct { + name string + fields fields + want string + }{ { - name: "file exists", + name: "GetAbsoluteSourceFilePath", fields: fields{ - folderGlobs: nil, - Folder: Folder{}, - Extension: Extension{}, - ExecutableName: nil, - pipelineId: uuid.UUID{}, + PipelineId: pipelineId, + Folder: Folder{SourceFileFolder: sourceFileFolder}, + Extension: Extension{SourceFileExtension: sourceFileExtension}, }, - args: args{ - fileName: "file.txt", - sourceDir: sourceDir, - destinationDir: destinationDir, + want: absoluteSourceFilePath, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + } + if got := l.GetAbsoluteSourceFilePath(); got != tt.want { + t.Errorf("GetAbsoluteSourceFilePath() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestLifeCycleDTO_GetExecutableFileName(t *testing.T) { + pipelineId := uuid.New() + executableFileExtension := ".executableFileExtension" + + type fields struct { + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "GetExecutableFileName", + fields: fields{ + PipelineId: pipelineId, + Extension: Extension{ExecutableFileExtension: executableFileExtension}, }, - wantErr: false, + want: pipelineId.String() + executableFileExtension, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - l := &LifeCycle{ - folderGlobs: tt.fields.folderGlobs, + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, Folder: tt.fields.Folder, Extension: tt.fields.Extension, ExecutableName: tt.fields.ExecutableName, - pipelineId: tt.fields.pipelineId, } - err := l.CopyFile(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir) - if (err != nil) != tt.wantErr { - t.Errorf("CopyFile() error = %v, wantErr %v", err, tt.wantErr) + if got := l.GetExecutableFileName(); got != tt.want { + t.Errorf("GetExecutableFileName() = %v, want %v", got, tt.want) } - if err == nil && !tt.wantErr { - newFilePath := filepath.Join(destinationDir, tt.args.fileName) - _, err = os.Stat(newFilePath) - if os.IsNotExist(err) { - t.Errorf("CopyFile() should create a new file: %s", newFilePath) - } + }) + } +} + +func TestLifeCycleDTO_GetSourceFileName(t *testing.T) { + pipelineId := uuid.New() + sourceFileExtension := ".sourceFileExtension" + + type fields struct { + PipelineId uuid.UUID + Folder Folder + Extension Extension + ExecutableName func(uuid.UUID, string) (string, error) + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "GetSourceFileName", + fields: fields{ + PipelineId: pipelineId, + Extension: Extension{SourceFileExtension: sourceFileExtension}, + }, + want: pipelineId.String() + sourceFileExtension, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + l := &LifeCycleDTO{ + PipelineId: tt.fields.PipelineId, + Folder: tt.fields.Folder, + Extension: tt.fields.Extension, + ExecutableName: tt.fields.ExecutableName, + } + if got := l.GetSourceFileName(); got != tt.want { + t.Errorf("GetSourceFileName() = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/fs_tool/go_fs_test.go b/playground/backend/internal/fs_tool/go_fs_test.go index e026d1d9e199..d82eac2123c1 100644 --- a/playground/backend/internal/fs_tool/go_fs_test.go +++ b/playground/backend/internal/fs_tool/go_fs_test.go @@ -48,16 +48,18 @@ func Test_newGoLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, + Dto: LifeCycleDTO{ + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: srcFileFolder, + ExecutableFileFolder: binFileFolder, + }, + Extension: Extension{ + SourceFileExtension: goSourceFileExtension, + ExecutableFileExtension: goExecutableFileExtension, + }, + PipelineId: pipelineId, }, - Extension: Extension{ - SourceFileExtension: goSourceFileExtension, - ExecutableFileExtension: goExecutableFileExtension, - }, - pipelineId: pipelineId, }, }, } @@ -67,14 +69,14 @@ func Test_newGoLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newGoLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Folder, tt.want.Folder) { - t.Errorf("newGoLifeCycle() Folder = %v, want %v", got.Folder, tt.want.Folder) + if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { + t.Errorf("newGoLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) } - if !reflect.DeepEqual(got.Extension, tt.want.Extension) { - t.Errorf("newGoLifeCycle() Extension = %v, want %v", got.Extension, tt.want.Extension) + if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { + t.Errorf("newGoLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) } - if !reflect.DeepEqual(got.pipelineId, tt.want.pipelineId) { - t.Errorf("newGoLifeCycle() pipelineId = %v, want %v", got.pipelineId, tt.want.pipelineId) + if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { + t.Errorf("newGoLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) } }) } diff --git a/playground/backend/internal/fs_tool/java_fs.go b/playground/backend/internal/fs_tool/java_fs.go index 55e837346a06..dd3b376b5b58 100644 --- a/playground/backend/internal/fs_tool/java_fs.go +++ b/playground/backend/internal/fs_tool/java_fs.go @@ -31,7 +31,7 @@ const ( // newJavaLifeCycle creates LifeCycle with java SDK environment. func newJavaLifeCycle(pipelineId uuid.UUID, workingDir string) *LifeCycle { javaLifeCycle := newCompilingLifeCycle(pipelineId, workingDir, JavaSourceFileExtension, javaCompiledFileExtension) - javaLifeCycle.ExecutableName = executableName + javaLifeCycle.Dto.ExecutableName = executableName return javaLifeCycle } diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index 1bb4cc338f8d..1dfdb7061fc0 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -50,17 +50,19 @@ func Test_newJavaLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, + Dto: LifeCycleDTO{ + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: srcFileFolder, + ExecutableFileFolder: binFileFolder, + }, + Extension: Extension{ + SourceFileExtension: JavaSourceFileExtension, + ExecutableFileExtension: javaCompiledFileExtension, + }, + ExecutableName: executableName, + PipelineId: pipelineId, }, - Extension: Extension{ - SourceFileExtension: JavaSourceFileExtension, - ExecutableFileExtension: javaCompiledFileExtension, - }, - ExecutableName: executableName, - pipelineId: pipelineId, }, }, } @@ -70,14 +72,14 @@ func Test_newJavaLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newJavaLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Folder, tt.want.Folder) { - t.Errorf("newJavaLifeCycle() Folder = %v, want %v", got.Folder, tt.want.Folder) + if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { + t.Errorf("newJavaLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) } - if !reflect.DeepEqual(got.Extension, tt.want.Extension) { - t.Errorf("newJavaLifeCycle() Extension = %v, want %v", got.Extension, tt.want.Extension) + if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { + t.Errorf("newJavaLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) } - if !reflect.DeepEqual(got.pipelineId, tt.want.pipelineId) { - t.Errorf("newJavaLifeCycle() pipelineId = %v, want %v", got.pipelineId, tt.want.pipelineId) + if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { + t.Errorf("newJavaLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) } }) } diff --git a/playground/backend/internal/fs_tool/lc_constructor.go b/playground/backend/internal/fs_tool/lc_constructor.go index d2a6164a141b..1c810fec61ec 100644 --- a/playground/backend/internal/fs_tool/lc_constructor.go +++ b/playground/backend/internal/fs_tool/lc_constructor.go @@ -33,16 +33,18 @@ func newCompilingLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFileEx binFileFolder := filepath.Join(baseFileFolder, compiledFolderName) return &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, + Dto: LifeCycleDTO{ + PipelineId: pipelineId, + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: srcFileFolder, + ExecutableFileFolder: binFileFolder, + }, + Extension: Extension{ + SourceFileExtension: sourceFileExtension, + ExecutableFileExtension: compiledFileExtension, + }, }, - Extension: Extension{ - SourceFileExtension: sourceFileExtension, - ExecutableFileExtension: compiledFileExtension, - }, - pipelineId: pipelineId, } } @@ -51,15 +53,17 @@ func newInterpretedLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFile sourceFileFolder := filepath.Join(workingDir, baseFileFolder, pipelineId.String()) return &LifeCycle{ folderGlobs: []string{sourceFileFolder}, - Folder: Folder{ - BaseFolder: sourceFileFolder, - SourceFileFolder: sourceFileFolder, - ExecutableFileFolder: sourceFileFolder, - }, - Extension: Extension{ - ExecutableFileExtension: sourceFileExtension, - SourceFileExtension: sourceFileExtension, + Dto: LifeCycleDTO{ + PipelineId: pipelineId, + Folder: Folder{ + BaseFolder: sourceFileFolder, + SourceFileFolder: sourceFileFolder, + ExecutableFileFolder: sourceFileFolder, + }, + Extension: Extension{ + ExecutableFileExtension: sourceFileExtension, + SourceFileExtension: sourceFileExtension, + }, }, - pipelineId: pipelineId, } } diff --git a/playground/backend/internal/fs_tool/python_fs_test.go b/playground/backend/internal/fs_tool/python_fs_test.go index c0e6ded2755c..00d7773ea968 100644 --- a/playground/backend/internal/fs_tool/python_fs_test.go +++ b/playground/backend/internal/fs_tool/python_fs_test.go @@ -46,16 +46,18 @@ func Test_newPythonLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder}, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: baseFileFolder, - ExecutableFileFolder: baseFileFolder, + Dto: LifeCycleDTO{ + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: baseFileFolder, + ExecutableFileFolder: baseFileFolder, + }, + Extension: Extension{ + SourceFileExtension: pythonExecutableFileExtension, + ExecutableFileExtension: pythonExecutableFileExtension, + }, + PipelineId: pipelineId, }, - Extension: Extension{ - SourceFileExtension: pythonExecutableFileExtension, - ExecutableFileExtension: pythonExecutableFileExtension, - }, - pipelineId: pipelineId, }, }, } @@ -65,14 +67,14 @@ func Test_newPythonLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newPythonLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Folder, tt.want.Folder) { - t.Errorf("newPythonLifeCycle() Folder = %v, want %v", got.Folder, tt.want.Folder) + if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { + t.Errorf("newPythonLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) } - if !reflect.DeepEqual(got.Extension, tt.want.Extension) { - t.Errorf("newPythonLifeCycle() Extension = %v, want %v", got.Extension, tt.want.Extension) + if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { + t.Errorf("newPythonLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) } - if !reflect.DeepEqual(got.pipelineId, tt.want.pipelineId) { - t.Errorf("newPythonLifeCycle() pipelineId = %v, want %v", got.pipelineId, tt.want.pipelineId) + if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { + t.Errorf("newPythonLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) } }) } diff --git a/playground/backend/internal/preparators/java_preparators_test.go b/playground/backend/internal/preparators/java_preparators_test.go index b13da842a994..e964df09db54 100644 --- a/playground/backend/internal/preparators/java_preparators_test.go +++ b/playground/backend/internal/preparators/java_preparators_test.go @@ -40,7 +40,7 @@ func Test_replace(t *testing.T) { lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, uuid.New(), filepath.Join(path, "temp")) _ = lc.CreateFolders() defer os.RemoveAll(filepath.Join(path, "temp")) - _, _ = lc.CreateSourceCodeFile(codeWithPublicClass) + _ = lc.CreateSourceCodeFile(codeWithPublicClass) type args struct { args []interface{} @@ -58,14 +58,14 @@ func Test_replace(t *testing.T) { }, { name: "original file exists", - args: args{[]interface{}{lc.GetAbsoluteSourceFilePath(), classWithPublicModifierPattern, classWithoutPublicModifierPattern}}, + args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), classWithPublicModifierPattern, classWithoutPublicModifierPattern}}, wantCode: codeWithoutPublicClass, wantErr: false, }, { // Test that file where package is used changes to import all dependencies from this package name: "original file with package", - args: args{[]interface{}{lc.GetAbsoluteSourceFilePath(), packagePattern, importStringPattern}}, + args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), packagePattern, importStringPattern}}, wantCode: codeWithImportedPackage, wantErr: false, }, @@ -121,7 +121,7 @@ func Test_changeJavaTestFileName(t *testing.T) { lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, uuid.New(), filepath.Join(path, "temp")) _ = lc.CreateFolders() defer os.RemoveAll(filepath.Join(path, "temp")) - _, _ = lc.CreateSourceCodeFile(codeWithPublicClass) + _ = lc.CreateSourceCodeFile(codeWithPublicClass) validationResults := sync.Map{} validationResults.Store(validators.UnitTestValidatorName, true) @@ -137,7 +137,7 @@ func Test_changeJavaTestFileName(t *testing.T) { { // Test that file changes its name to the name of its public class name: "file with java unit test code to be renamed", - args: args{[]interface{}{lc.GetAbsoluteSourceFilePath(), &validationResults}}, + args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), &validationResults}}, wantErr: false, wantName: "Class.java", }, @@ -147,7 +147,7 @@ func Test_changeJavaTestFileName(t *testing.T) { if err := changeJavaTestFileName(tt.args.args...); (err != nil) != tt.wantErr { t.Errorf("changeJavaTestFileName() error = %v, wantErr %v", err, tt.wantErr) } - files, err := filepath.Glob(fmt.Sprintf("%s/*java", lc.GetAbsoluteSourceFolderPath())) + files, err := filepath.Glob(fmt.Sprintf("%s/*java", lc.Dto.GetAbsoluteSourceFileFolderPath())) if err != nil { t.Errorf("changeJavaTestFileName() error = %v, wantErr %v", err, tt.wantErr) } diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index cb92f04e49e7..f5635aa5ac4a 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -32,25 +32,25 @@ const ( ) // SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(lc *fs_tool.LifeCycle, pipelineOptions string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk if sdk == pb.Sdk_SDK_JAVA { pipelineOptions = utils.ReplaceSpacesWithEquals(pipelineOptions) } - val, err := utils.GetValidators(sdk, lc.GetAbsoluteSourceFilePath()) + val, err := utils.GetValidators(sdk, lcDto.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, lc.GetAbsoluteSourceFilePath()) + prep, err := utils.GetPreparators(sdk, lcDto.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). - WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). + WithExecutableFileName(lcDto.GetAbsoluteExecutableFilePath()). + WithWorkingDir(lcDto.GetAbsoluteBaseFolderPath()). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -58,7 +58,7 @@ func SetupExecutorBuilder(lc *fs_tool.LifeCycle, pipelineOptions string, sdkEnv WithCompiler(). WithCommand(executorConfig.CompileCmd). WithArgs(executorConfig.CompileArgs). - WithFileName(lc.GetAbsoluteSourceFilePath()). + WithFileName(lcDto.GetAbsoluteSourceFilePath()). WithRunner(). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). @@ -66,7 +66,7 @@ func SetupExecutorBuilder(lc *fs_tool.LifeCycle, pipelineOptions string, sdkEnv WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). - WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). + WithWorkingDir(lcDto.GetAbsoluteSourceFileFolderPath()). ExecutorBuilder switch sdk { @@ -74,20 +74,20 @@ func SetupExecutorBuilder(lc *fs_tool.LifeCycle, pipelineOptions string, sdkEnv args := make([]string, 0) for _, arg := range executorConfig.RunArgs { if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(lc.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) + logConfigFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) } args = append(args, arg) } builder = builder.WithRunner().WithArgs(args).ExecutorBuilder - builder = builder.WithTestRunner().WithWorkingDir(lc.GetAbsoluteBaseFolderPath()).ExecutorBuilder //change directory for unit test + builder = builder.WithTestRunner().WithWorkingDir(lcDto.GetAbsoluteBaseFolderPath()).ExecutorBuilder //change directory for unit test case pb.Sdk_SDK_GO: //go run command is executable file itself builder = builder. WithExecutableFileName(""). WithRunner(). - WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder + WithCommand(lcDto.GetAbsoluteExecutableFilePath()).ExecutorBuilder case pb.Sdk_SDK_PYTHON: - builder = *builder.WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()) + builder = *builder.WithExecutableFileName(lcDto.GetAbsoluteExecutableFilePath()) case pb.Sdk_SDK_SCIO: return nil, fmt.Errorf("SCIO is not supported yet") default: diff --git a/playground/backend/internal/setup_tools/builder/setup_builder_test.go b/playground/backend/internal/setup_tools/builder/setup_builder_test.go index 8c41afb41dcb..6ccad9dc2626 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -47,7 +47,7 @@ func TestSetupExecutor(t *testing.T) { panic(err) } - srcFilePath := lc.GetAbsoluteSourceFilePath() + srcFilePath := lc.Dto.GetAbsoluteSourceFilePath() sdkEnv := environment.NewBeamEnvs(sdk, executorConfig, "") val, err := utils.GetValidators(sdk, srcFilePath) @@ -60,8 +60,8 @@ func TestSetupExecutor(t *testing.T) { } wantExecutor := executors.NewExecutorBuilder(). - WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). - WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). + WithExecutableFileName(lc.Dto.GetAbsoluteExecutableFilePath()). + WithWorkingDir(lc.Dto.GetAbsoluteBaseFolderPath()). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -77,11 +77,11 @@ func TestSetupExecutor(t *testing.T) { WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). - WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). + WithWorkingDir(lc.Dto.GetAbsoluteBaseFolderPath()). ExecutorBuilder type args struct { - lc *fs_tool.LifeCycle + dto fs_tool.LifeCycleDTO pipelineOptions string sdkEnv *environment.BeamEnvs } @@ -95,7 +95,7 @@ func TestSetupExecutor(t *testing.T) { // Test case with calling Setup with incorrect SDK. // As a result, want to receive an error. name: "incorrect sdk", - args: args{lc, pipelineOptions, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, + args: args{lc.Dto, pipelineOptions, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, want: nil, wantErr: true, }, @@ -103,14 +103,14 @@ func TestSetupExecutor(t *testing.T) { // Test case with calling Setup with correct SDK. // As a result, want to receive an expected builder. name: "correct sdk", - args: args{lc, pipelineOptions, sdkEnv}, + args: args{lc.Dto, pipelineOptions, sdkEnv}, want: &wantExecutor, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := SetupExecutorBuilder(tt.args.lc, tt.args.pipelineOptions, tt.args.sdkEnv) + got, err := SetupExecutorBuilder(tt.args.dto, tt.args.pipelineOptions, tt.args.sdkEnv) if (err != nil) != tt.wantErr { t.Errorf("SetupExecutorBuilder() error = %v, wantErr %v", err, tt.wantErr) return diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go index 2db5246be75f..648760ef9b85 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go @@ -69,7 +69,7 @@ func Setup(sdk pb.Sdk, code string, pipelineId uuid.UUID, workingDir string, pre } // create file with code - _, err = lc.CreateSourceCodeFile(code) + err = lc.CreateSourceCodeFile(code) if err != nil { logger.Errorf("%s: RunCode(): CreateSourceCodeFile(): %s\n", pipelineId, err.Error()) lc.DeleteFolders() @@ -81,11 +81,11 @@ func Setup(sdk pb.Sdk, code string, pipelineId uuid.UUID, workingDir string, pre // prepareGoFiles prepares file for Go environment. // Copy go.mod and go.sum file from /path/to/preparedModDir to /path/to/workingDir/executable_files/{pipelineId} func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uuid.UUID) error { - if err := lc.CopyFile(goModFileName, preparedModDir, lc.Folder.BaseFolder); err != nil { + if err := lc.CopyFile(goModFileName, preparedModDir, lc.Dto.GetAbsoluteBaseFolderPath()); err != nil { logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goModFileName, err.Error()) return err } - if err := lc.CopyFile(goSumFileName, preparedModDir, lc.Folder.BaseFolder); err != nil { + if err := lc.CopyFile(goSumFileName, preparedModDir, lc.Dto.GetAbsoluteBaseFolderPath()); err != nil { logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goSumFileName, err.Error()) return err } @@ -96,12 +96,12 @@ func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uui // Copy log config file from /path/to/workingDir to /path/to/workingDir/executable_files/{pipelineId} // and update this file according to pipeline. func prepareJavaFiles(lc *fs_tool.LifeCycle, workingDir string, pipelineId uuid.UUID) error { - err := lc.CopyFile(javaLogConfigFileName, workingDir, lc.Folder.BaseFolder) + err := lc.CopyFile(javaLogConfigFileName, workingDir, lc.Dto.GetAbsoluteBaseFolderPath()) if err != nil { logger.Errorf("%s: error during copying logging.properties file: %s\n", pipelineId, err.Error()) return err } - err = updateJavaLogConfigFile(lc) + err = updateJavaLogConfigFile(lc.Dto) if err != nil { logger.Errorf("%s: error during updating logging.properties file: %s\n", pipelineId, err.Error()) return err @@ -110,9 +110,9 @@ func prepareJavaFiles(lc *fs_tool.LifeCycle, workingDir string, pipelineId uuid. } // updateJavaLogConfigFile updates java log config file according to pipeline -func updateJavaLogConfigFile(lc *fs_tool.LifeCycle) error { - logConfigFilePath := filepath.Join(lc.Folder.BaseFolder, javaLogConfigFileName) - logConfigUpdatedFilePath := filepath.Join(lc.Folder.BaseFolder, javaTmpLogConfigFile) +func updateJavaLogConfigFile(lcDto fs_tool.LifeCycleDTO) error { + logConfigFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) + logConfigUpdatedFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaTmpLogConfigFile) if _, err := os.Stat(logConfigFilePath); os.IsNotExist(err) { return err } @@ -130,7 +130,7 @@ func updateJavaLogConfigFile(lc *fs_tool.LifeCycle) error { for scanner.Scan() { line := scanner.Text() - line = strings.ReplaceAll(line, javaLogFilePlaceholder, lc.GetAbsoluteLogFilePath()) + line = strings.ReplaceAll(line, javaLogFilePlaceholder, lcDto.GetAbsoluteLogFilePath()) if _, err = io.WriteString(updatedFile, line+"\n"); err != nil { return err } diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go index be1d638dbfda..a4011546d074 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go @@ -122,11 +122,11 @@ func TestSetup(t *testing.T) { return } if got != nil { - if !reflect.DeepEqual(got.Folder, tt.want.Folder) { - t.Errorf("Setup() got.Folder = %v, want %v", got.Folder, tt.want.Folder) + if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { + t.Errorf("Setup() got.Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) } - if !reflect.DeepEqual(got.Extension, tt.want.Extension) { - t.Errorf("Setup() got.Extension = %v, want %v", got.Extension, tt.want.Extension) + if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { + t.Errorf("Setup() got.Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) } if !tt.check() { t.Errorf("Setup() doesn't prepare necessary files/folders") From ee769b1f5e67b4c9b5135422a305f4b2ea6f0ca7 Mon Sep 17 00:00:00 2001 From: AydarZaynutdinov Date: Tue, 11 Jan 2022 15:42:04 +0300 Subject: [PATCH 2/5] [BEAM-13398][Playground] Change LifeCycle structure --- .../code_processing/code_processing.go | 17 +- .../code_processing/code_processing_test.go | 6 +- playground/backend/internal/fs_tool/fs.go | 87 +-- .../backend/internal/fs_tool/fs_test.go | 701 +++++------------- .../backend/internal/fs_tool/go_fs_test.go | 40 +- .../backend/internal/fs_tool/java_fs.go | 8 +- .../backend/internal/fs_tool/java_fs_test.go | 43 +- .../internal/fs_tool/lc_constructor.go | 62 +- .../internal/fs_tool/python_fs_test.go | 36 +- .../internal/preparators/java_preparators.go | 3 +- .../preparators/java_preparators_test.go | 8 +- .../setup_tools/builder/setup_builder.go | 26 +- .../setup_tools/builder/setup_builder_test.go | 14 +- .../life_cycle/life_cycle_setuper.go | 18 +- .../life_cycle/life_cycle_setuper_test.go | 66 +- 15 files changed, 384 insertions(+), 751 deletions(-) diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 01622aceee7f..f5075cf87462 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -34,6 +34,7 @@ import ( "io" "os" "os/exec" + "path/filepath" "reflect" "sync" "time" @@ -69,7 +70,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl go cancelCheck(pipelineLifeCycleCtx, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc.Dto, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) + executorBuilder, err := builder.SetupExecutorBuilder(lc.Paths, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) if err != nil { _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) return @@ -131,7 +132,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Compile if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { executor = executorBuilder.WithCompiler(). - WithFileName(builder.GetFileNameFromFolder(lc.Dto.GetAbsoluteSourceFileFolderPath())).Build() // Need changed name for unit tests + WithFileName(builder.GetFileNameFromFolder(lc.Paths.AbsoluteSourceFileFolderPath, filepath.Ext(lc.Paths.SourceFileName))).Build() // Need changed name for unit tests } logger.Infof("%s: Compile() ...\n", pipelineId) compileCmd := executor.Compile(pipelineLifeCycleCtx) @@ -156,7 +157,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Run if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { - executor, err = setJavaExecutableFile(lc.Dto, pipelineId, cacheService, pipelineLifeCycleCtx, executorBuilder, appEnv.WorkingDir()) + executor, err = setJavaExecutableFile(lc.Paths, pipelineId, cacheService, pipelineLifeCycleCtx, executorBuilder, appEnv.WorkingDir()) if err != nil { return } @@ -165,11 +166,11 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl runCmd := getExecuteCmd(&validationResults, &executor, pipelineLifeCycleCtx) var runError bytes.Buffer runOutput := streaming.RunOutputWriter{Ctx: pipelineLifeCycleCtx, CacheService: cacheService, PipelineId: pipelineId} - go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, lc.Dto.GetAbsoluteLogFilePath(), pipelineId, stopReadLogsChannel, finishReadLogsChannel) + go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, lc.Paths.AbsoluteLogFilePath, pipelineId, stopReadLogsChannel, finishReadLogsChannel) if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO { // For go SDK all logs are placed to stdErr. - file, err := os.Create(lc.Dto.GetAbsoluteLogFilePath()) + file, err := os.Create(lc.Paths.AbsoluteLogFilePath) if err != nil { // If some error with creating a log file do the same as with other SDK. logger.Errorf("%s: error during create log file (go sdk): %s", pipelineId, err.Error()) @@ -195,7 +196,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Run step is finished, but code contains some error (divide by 0 for example) if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO { // For Go SDK stdErr was redirected to the log file. - errData, err := os.ReadFile(lc.Dto.GetAbsoluteLogFilePath()) + errData, err := os.ReadFile(lc.Paths.AbsoluteLogFilePath) if err != nil { logger.Errorf("%s: error during read errors from log file (go sdk): %s", pipelineId, err.Error()) } @@ -220,8 +221,8 @@ func getExecuteCmd(valRes *sync.Map, executor *executors.Executor, ctxWithTimeou } // setJavaExecutableFile sets executable file name to runner (JAVA class name is known after compilation step) -func setJavaExecutableFile(lcDto fs_tool.LifeCycleDTO, id uuid.UUID, service cache.Cache, ctx context.Context, executorBuilder *executors.ExecutorBuilder, dir string) (executors.Executor, error) { - className, err := lcDto.ExecutableName(id, dir) +func setJavaExecutableFile(paths fs_tool.LifeCyclePaths, id uuid.UUID, service cache.Cache, ctx context.Context, executorBuilder *executors.ExecutorBuilder, dir string) (executors.Executor, error) { + className, err := paths.ExecutableName(id, dir) if err != nil { if err = processSetupError(err, id, service, ctx); err != nil { return executorBuilder.Build(), err diff --git a/playground/backend/internal/code_processing/code_processing_test.go b/playground/backend/internal/code_processing/code_processing_test.go index 0c6df550dfef..8aff888b2760 100644 --- a/playground/backend/internal/code_processing/code_processing_test.go +++ b/playground/backend/internal/code_processing/code_processing_test.go @@ -269,7 +269,7 @@ func Test_Process(t *testing.T) { compileOutput, _ := cacheService.GetValue(tt.args.ctx, tt.args.pipelineId, cache.CompileOutput) if tt.expectedCompileOutput != nil && strings.Contains(tt.expectedCompileOutput.(string), "%s") { - tt.expectedCompileOutput = fmt.Sprintf(tt.expectedCompileOutput.(string), lc.Dto.GetAbsoluteSourceFilePath()) + tt.expectedCompileOutput = fmt.Sprintf(tt.expectedCompileOutput.(string), lc.Paths.AbsoluteSourceFilePath) } if !reflect.DeepEqual(compileOutput, tt.expectedCompileOutput) { t.Errorf("processCode() set compileOutput: %s, but expectes: %s", compileOutput, tt.expectedCompileOutput) @@ -535,7 +535,7 @@ func TestGetLastIndex(t *testing.T) { func Test_setJavaExecutableFile(t *testing.T) { pipelineId := uuid.New() lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, pipelineId, os.Getenv("APP_WORK_DIR")) - lc.Dto.ExecutableName = fakeExecutableName + lc.Paths.ExecutableName = fakeExecutableName executorBuilder := executors.NewExecutorBuilder().WithRunner().WithCommand("fake cmd").ExecutorBuilder type args struct { lc *fs_tool.LifeCycle @@ -572,7 +572,7 @@ func Test_setJavaExecutableFile(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := setJavaExecutableFile(tt.args.lc.Dto, tt.args.id, tt.args.service, tt.args.ctx, tt.args.executorBuilder, tt.args.dir) + got, err := setJavaExecutableFile(tt.args.lc.Paths, tt.args.id, tt.args.service, tt.args.ctx, tt.args.executorBuilder, tt.args.dir) if (err != nil) != tt.wantErr { t.Errorf("setJavaExecutableFile() error = %v, wantErr %v", err, tt.wantErr) } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 3d0c11a50bd2..ce2978e6f40d 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -30,80 +30,23 @@ const ( logFileName = "logs.log" ) -// Folder contains names of folders with executable and compiled files. -// For each SDK these values should be set depending on folders that need for the SDK. -type Folder struct { - BaseFolder string - SourceFileFolder string - ExecutableFileFolder string -} - -// Extension contains executable and compiled files' extensions. -// For each SDK these values should be set depending on SDK's extensions. -type Extension struct { - SourceFileExtension string - ExecutableFileExtension string -} - -// LifeCycleDTO contains methods to receive files/folders paths -type LifeCycleDTO struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) -} - -// GetSourceFileName returns name of the source file like {pipelineId}.{sourceFileExtension}. -func (l *LifeCycleDTO) GetSourceFileName() string { - return l.PipelineId.String() + l.Extension.SourceFileExtension -} - -// GetAbsoluteSourceFileFolderPath returns absolute filepath to source folder (/path/to/workingDir/executable_files/{pipelineId}/src). -func (l *LifeCycleDTO) GetAbsoluteSourceFileFolderPath() string { - absolutePath, _ := filepath.Abs(l.Folder.SourceFileFolder) - return absolutePath -} - -// GetAbsoluteSourceFilePath returns absolute filepath to source file (/path/to/workingDir/executable_files/{pipelineId}/src/{pipelineId}.{sourceFileExtension}). -func (l *LifeCycleDTO) GetAbsoluteSourceFilePath() string { - absolutePath, _ := filepath.Abs(filepath.Join(l.GetAbsoluteSourceFileFolderPath(), l.GetSourceFileName())) - return absolutePath -} - -// GetExecutableFileName returns name of the executable file like {pipelineId}.{executableFileExtension}. -func (l *LifeCycleDTO) GetExecutableFileName() string { - return l.PipelineId.String() + l.Extension.ExecutableFileExtension -} - -// GetAbsoluteExecutableFileFolderPath returns absolute filepath to executable folder (/path/to/workingDir/executable_files/{pipelineId}/bin). -func (l *LifeCycleDTO) GetAbsoluteExecutableFileFolderPath() string { - absolutePath, _ := filepath.Abs(l.Folder.ExecutableFileFolder) - return absolutePath -} - -// GetAbsoluteExecutableFilePath returns absolute filepath to executable file (/path/to/workingDir/executable_files/{pipelineId}/bin/{pipelineId}.{executableFileExtension}). -func (l *LifeCycleDTO) GetAbsoluteExecutableFilePath() string { - absolutePath, _ := filepath.Abs(filepath.Join(l.GetAbsoluteExecutableFileFolderPath(), l.GetExecutableFileName())) - return absolutePath -} - -// GetAbsoluteBaseFolderPath returns absolute path to executable folder (/path/to/workingDir/executable_files/{pipelineId}). -func (l *LifeCycleDTO) GetAbsoluteBaseFolderPath() string { - absolutePath, _ := filepath.Abs(l.Folder.BaseFolder) - return absolutePath -} - -// GetAbsoluteLogFilePath returns absolute path to the logs file (/path/to/workingDir/executable_files/{pipelineId}/logs.log) -func (l *LifeCycleDTO) GetAbsoluteLogFilePath() string { - filePath := filepath.Join(l.GetAbsoluteBaseFolderPath(), logFileName) - absoluteFilePath, _ := filepath.Abs(filePath) - return absoluteFilePath +// LifeCyclePaths contains all files/folders paths +type LifeCyclePaths struct { + SourceFileName string // {pipelineId}.{sourceFileExtension} + AbsoluteSourceFileFolderPath string // /path/to/workingDir/executable_files/{pipelineId}/src + AbsoluteSourceFilePath string // /path/to/workingDir/executable_files/{pipelineId}/src/{pipelineId}.{sourceFileExtension} + ExecutableFileName string // {pipelineId}.{executableFileExtension} + AbsoluteExecutableFileFolderPath string // /path/to/workingDir/executable_files/{pipelineId}/bin + AbsoluteExecutableFilePath string // /path/to/workingDir/executable_files/{pipelineId}/bin/{pipelineId}.{executableFileExtension} + AbsoluteBaseFolderPath string // /path/to/workingDir/executable_files/{pipelineId} + AbsoluteLogFilePath string // /path/to/workingDir/executable_files/{pipelineId}/logs.log + ExecutableName func(uuid.UUID, string) (string, error) } // LifeCycle is used for preparing folders and files to process code for one code processing request. type LifeCycle struct { - folderGlobs []string //folders that should be created to process code - Dto LifeCycleDTO + folderGlobs []string // folders that should be created to process code + Paths LifeCyclePaths } // NewLifeCycle returns a corresponding LifeCycle depending on the given SDK. @@ -145,11 +88,11 @@ func (l *LifeCycle) DeleteFolders() error { // CreateSourceCodeFile creates an executable file (i.e. file.{sourceFileExtension}). func (l *LifeCycle) CreateSourceCodeFile(code string) error { - if _, err := os.Stat(l.Dto.GetAbsoluteSourceFileFolderPath()); os.IsNotExist(err) { + if _, err := os.Stat(l.Paths.AbsoluteSourceFileFolderPath); os.IsNotExist(err) { return err } - filePath := l.Dto.GetAbsoluteSourceFilePath() + filePath := l.Paths.AbsoluteSourceFilePath err := os.WriteFile(filePath, []byte(code), fileMode) if err != nil { return err diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index d545e79437e8..150edce19db4 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -16,11 +16,9 @@ package fs_tool import ( - pb "beam.apache.org/playground/backend/internal/api/v1" - "beam.apache.org/playground/backend/internal/logger" + playground "beam.apache.org/playground/backend/internal/api/v1" "fmt" "github.com/google/uuid" - "io/fs" "os" "path/filepath" "reflect" @@ -30,125 +28,107 @@ import ( const ( sourceDir = "sourceDir" destinationDir = "destinationDir" + testFileMode = 0755 ) -func TestMain(m *testing.M) { - err := setupPreparedFiles() - if err != nil { - logger.Fatal(err) - } - defer teardown() - m.Run() -} - -func setupPreparedFiles() error { - err := os.Mkdir(sourceDir, 0755) +func prepareFiles() error { + err := os.Mkdir(sourceDir, testFileMode) if err != nil { return err } - err = os.Mkdir(destinationDir, 0755) + err = os.Mkdir(destinationDir, testFileMode) if err != nil { return err } filePath := filepath.Join(sourceDir, "file.txt") _, err = os.Create(filePath) - if err != nil { - return err - } - return nil + return err } -func teardown() { +func teardownFiles() error { err := os.RemoveAll(sourceDir) if err != nil { - logger.Fatal(err) - } - err = os.RemoveAll(destinationDir) - if err != nil { - logger.Fatal(err) + return err } + return os.RemoveAll(destinationDir) } -func TestLifeCycle_CreateSourceCodeFile(t *testing.T) { - pipelineId := uuid.New() - baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) - srcFileFolder := baseFileFolder + "/src" - binFileFolder := baseFileFolder + "/bin" +func prepareFolders(baseFileFolder string) error { + srcFileFolder := filepath.Join(baseFileFolder, "src") + + return os.MkdirAll(srcFileFolder, testFileMode) +} + +func teardownFolders(baseFileFolder string) error { + err := os.RemoveAll(baseFileFolder) + return err +} + +func TestLifeCycle_CopyFile(t *testing.T) { + if err := prepareFiles(); err != nil { + t.Fatalf("Error during preparing files for test: %s", err) + } + defer teardownFiles() type fields struct { folderGlobs []string - folder Folder - extension Extension - pipelineId uuid.UUID + Paths LifeCyclePaths } type args struct { - code string + fileName string + sourceDir string + destinationDir string } tests := []struct { - name string - createFolders []string - fields fields - args args - want string - wantErr bool + name string + fields fields + args args + wantErr bool }{ { - name: "executable folder doesn't exist", + name: "file doesn't exist", fields: fields{ - folder: Folder{ - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - pipelineId: pipelineId, + folderGlobs: nil, + }, + args: args{ + fileName: "file1.txt", + sourceDir: sourceDir, + destinationDir: destinationDir, }, - args: args{}, - want: "", wantErr: true, }, { - name: "executable folder exists", - createFolders: []string{srcFileFolder}, + name: "file exists", fields: fields{ - folder: Folder{SourceFileFolder: srcFileFolder}, - extension: Extension{SourceFileExtension: JavaSourceFileExtension}, - pipelineId: pipelineId, + folderGlobs: nil, + }, + args: args{ + fileName: "file.txt", + sourceDir: sourceDir, + destinationDir: destinationDir, }, - args: args{code: "TEST_CODE"}, wantErr: false, }, } for _, tt := range tests { - for _, folder := range tt.createFolders { - os.MkdirAll(folder, fs.ModePerm) - } t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Dto: LifeCycleDTO{ - Folder: tt.fields.folder, - Extension: tt.fields.extension, - PipelineId: tt.fields.pipelineId, - }, + Paths: tt.fields.Paths, } - err := l.CreateSourceCodeFile(tt.args.code) - if (err != nil) != tt.wantErr { - t.Errorf("CreateSourceCodeFile() error = %v, wantErr %v", err, tt.wantErr) - return + if err := l.CopyFile(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir); (err != nil) != tt.wantErr { + t.Errorf("CopyFile() error = %v, wantErr %v", err, tt.wantErr) } }) - os.RemoveAll(baseFileFolder) } } func TestLifeCycle_CreateFolders(t *testing.T) { pipelineId := uuid.New() - baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) + baseFileFolder := pipelineId.String() type fields struct { folderGlobs []string - folder Folder - extension Extension - pipelineId uuid.UUID } tests := []struct { name string @@ -165,527 +145,234 @@ func TestLifeCycle_CreateFolders(t *testing.T) { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ folderGlobs: tt.fields.folderGlobs, - Dto: LifeCycleDTO{ - Folder: tt.fields.folder, - Extension: tt.fields.extension, - PipelineId: tt.fields.pipelineId, - }, } if err := l.CreateFolders(); (err != nil) != tt.wantErr { t.Errorf("CreateFolders() error = %v, wantErr %v", err, tt.wantErr) } - for _, folder := range tt.fields.folderGlobs { - if _, err := os.Stat(folder); os.IsNotExist(err) { - t.Errorf("CreateFolders() should create folder %s, but it dosn't", folder) - } - } }) os.RemoveAll(baseFileFolder) } } -func TestLifeCycle_DeleteFolders(t *testing.T) { +func TestLifeCycle_CreateSourceCodeFile(t *testing.T) { pipelineId := uuid.New() - baseFileFolder := fmt.Sprintf("%s_%s", baseFileFolder, pipelineId) + baseFileFolder, _ := filepath.Abs(pipelineId.String()) + if err := prepareFolders(baseFileFolder); err != nil { + t.Fatalf("Error during preparing folders for test: %s", err) + } + defer teardownFolders(baseFileFolder) type fields struct { - folderGlobs []string - folder Folder - extension Extension - pipelineId uuid.UUID + Paths LifeCyclePaths + } + type args struct { + code string } tests := []struct { name string fields fields + args args wantErr bool }{ { - name: "DeleteFolders", + name: "source file folder path doesn't exist", fields: fields{ - folderGlobs: []string{baseFileFolder}, - pipelineId: pipelineId, + Paths: LifeCyclePaths{ + AbsoluteSourceFileFolderPath: "src", + }, + }, wantErr: true, + }, + { + name: "source file folder path exists", + fields: fields{ + Paths: LifeCyclePaths{ + AbsoluteSourceFileFolderPath: filepath.Join(baseFileFolder, "src"), + AbsoluteSourceFilePath: filepath.Join(baseFileFolder, "src", fmt.Sprintf("%s.%s", pipelineId.String(), "txt")), + }, }, + args: args{code: "TEST_CODE"}, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { l := &LifeCycle{ - folderGlobs: tt.fields.folderGlobs, - Dto: LifeCycleDTO{ - Folder: tt.fields.folder, - Extension: tt.fields.extension, - PipelineId: tt.fields.pipelineId, - }, + Paths: tt.fields.Paths, } - if err := l.DeleteFolders(); (err != nil) != tt.wantErr { - t.Errorf("DeleteFolders() error = %v, wantErr %v", err, tt.wantErr) + if err := l.CreateSourceCodeFile(tt.args.code); (err != nil) != tt.wantErr { + t.Errorf("CreateSourceCodeFile() error = %v, wantErr %v", err, tt.wantErr) + } + if !tt.wantErr { + if _, err := os.Stat(l.Paths.AbsoluteSourceFilePath); os.IsNotExist(err) { + t.Error("CreateSourceCodeFile() should create a new file, but it doesn't") + } else { + data, err := os.ReadFile(l.Paths.AbsoluteSourceFilePath) + if err != nil { + t.Errorf("CreateSourceCodeFile() error during open created file: %s", err) + } + if string(data) != tt.args.code { + t.Errorf("CreateSourceCodeFile() code = %s, want code %s", string(data), tt.args.code) + } + } } }) } } -func TestNewLifeCycle(t *testing.T) { +func TestLifeCycle_DeleteFolders(t *testing.T) { pipelineId := uuid.New() - workingDir := "workingDir" - baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) - srcFileFolder := baseFileFolder + "/src" - binFileFolder := baseFileFolder + "/bin" + baseFileFolder := pipelineId.String() + if err := prepareFolders(baseFileFolder); err != nil { + t.Fatalf("Error during preparing folders for test: %s", err) + } - type args struct { - sdk pb.Sdk - pipelineId uuid.UUID - workingDir string + type fields struct { + folderGlobs []string } tests := []struct { name string - args args - want *LifeCycle + fields fields wantErr bool }{ { - name: "Available SDK", - args: args{ - sdk: pb.Sdk_SDK_JAVA, - pipelineId: pipelineId, - workingDir: workingDir, - }, - want: &LifeCycle{ - folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Dto: LifeCycleDTO{ - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - Extension: Extension{ - SourceFileExtension: JavaSourceFileExtension, - ExecutableFileExtension: javaCompiledFileExtension, - }, - ExecutableName: executableName, - PipelineId: pipelineId, - }, - }, + name: "DeleteFolders", + fields: fields{folderGlobs: []string{baseFileFolder}}, wantErr: false, }, - { - name: "Unavailable SDK", - args: args{ - sdk: pb.Sdk_SDK_UNSPECIFIED, - pipelineId: pipelineId, - workingDir: workingDir, - }, - want: nil, - wantErr: true, - }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := NewLifeCycle(tt.args.sdk, tt.args.pipelineId, tt.args.workingDir) - if (err != nil) != tt.wantErr { - t.Errorf("NewLifeCycle() error = %v, wantErr %v", err, tt.wantErr) - return - } - if !tt.wantErr && !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { - t.Errorf("NewLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) - } - if !tt.wantErr && !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { - t.Errorf("NewLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) + l := &LifeCycle{ + folderGlobs: tt.fields.folderGlobs, } - if !tt.wantErr && !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { - t.Errorf("NewLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) + if err := l.DeleteFolders(); (err != nil) != tt.wantErr { + t.Errorf("DeleteFolders() error = %v, wantErr %v", err, tt.wantErr) } - if !tt.wantErr && !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { - t.Errorf("NewLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) + if !tt.wantErr { + if _, err := os.Stat(baseFileFolder); err == nil || !os.IsNotExist(err) { + t.Error("DeleteFolders() should remove folders, but it doesn't") + } } }) } } -func TestLifeCycle_CopyFile(t *testing.T) { - type fields struct { - folderGlobs []string - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - pipelineId uuid.UUID - } +func TestNewLifeCycle(t *testing.T) { + pipelineId := uuid.New() + workingDir, _ := filepath.Abs("workingDir") + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) + srcFileFolder := filepath.Join(baseFileFolder, "src") + execFileFolder := filepath.Join(baseFileFolder, "bin") + type args struct { - fileName string - sourceDir string - destinationDir string + sdk playground.Sdk + pipelineId uuid.UUID + workingDir string } tests := []struct { name string - fields fields args args + want *LifeCycle wantErr bool }{ { - name: "file doesn't exist", - fields: fields{ - folderGlobs: nil, - Folder: Folder{}, - Extension: Extension{}, - ExecutableName: nil, - pipelineId: uuid.UUID{}, - }, + name: "Java LifeCycle", args: args{ - fileName: "file1.txt", - sourceDir: sourceDir, - destinationDir: destinationDir, + sdk: playground.Sdk_SDK_JAVA, + pipelineId: pipelineId, + workingDir: workingDir, + }, + want: &LifeCycle{ + folderGlobs: []string{baseFileFolder, srcFileFolder, execFileFolder}, + Paths: LifeCyclePaths{ + SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), javaSourceFileExtension), + AbsoluteSourceFileFolderPath: srcFileFolder, + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), javaSourceFileExtension)), + ExecutableFileName: fmt.Sprintf("%s%s", pipelineId.String(), javaCompiledFileExtension), + AbsoluteExecutableFileFolderPath: execFileFolder, + AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), javaCompiledFileExtension)), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + }, }, - wantErr: true, }, { - name: "file exists", - fields: fields{ - folderGlobs: nil, - Folder: Folder{}, - Extension: Extension{}, - ExecutableName: nil, - pipelineId: uuid.UUID{}, - }, + name: "Go LifeCycle", args: args{ - fileName: "file.txt", - sourceDir: sourceDir, - destinationDir: destinationDir, + sdk: playground.Sdk_SDK_GO, + pipelineId: pipelineId, + workingDir: workingDir, }, - wantErr: false, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycle{ - folderGlobs: tt.fields.folderGlobs, - Dto: LifeCycleDTO{ - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - PipelineId: tt.fields.pipelineId, + want: &LifeCycle{ + folderGlobs: []string{baseFileFolder, srcFileFolder, execFileFolder}, + Paths: LifeCyclePaths{ + SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), goSourceFileExtension), + AbsoluteSourceFileFolderPath: srcFileFolder, + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), goSourceFileExtension)), + ExecutableFileName: fmt.Sprintf("%s%s", pipelineId.String(), goExecutableFileExtension), + AbsoluteExecutableFileFolderPath: execFileFolder, + AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), goExecutableFileExtension)), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), }, - } - err := l.CopyFile(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir) - if (err != nil) != tt.wantErr { - t.Errorf("CopyFile() error = %v, wantErr %v", err, tt.wantErr) - } - if err == nil && !tt.wantErr { - newFilePath := filepath.Join(destinationDir, tt.args.fileName) - _, err = os.Stat(newFilePath) - if os.IsNotExist(err) { - t.Errorf("CopyFile() should create a new file: %s", newFilePath) - } - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteBaseFolderPath(t *testing.T) { - baseFolder := "baseFolder" - absoluteBaseFileFolder, _ := filepath.Abs(baseFolder) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetAbsoluteBaseFolderPath", - fields: fields{Folder: Folder{BaseFolder: baseFolder}}, - want: absoluteBaseFileFolder, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteBaseFolderPath(); got != tt.want { - t.Errorf("GetAbsoluteBaseFolderPath() = %v, want %v", got, tt.want) - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteExecutableFileFolderPath(t *testing.T) { - executableFileFolder := "executableFileFolder" - absoluteExecutableFileFolder, _ := filepath.Abs(executableFileFolder) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetAbsoluteExecutableFileFolderPath", - fields: fields{Folder: Folder{ExecutableFileFolder: executableFileFolder}}, - want: absoluteExecutableFileFolder, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteExecutableFileFolderPath(); got != tt.want { - t.Errorf("GetAbsoluteExecutableFileFolderPath() = %v, want %v", got, tt.want) - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteExecutableFilePath(t *testing.T) { - pipelineId := uuid.New() - executableFileFolder := "executableFileFolder" - executableFileExtension := ".executableFileExtension" - absoluteExecutableFilePath, _ := filepath.Abs(filepath.Join(executableFileFolder, pipelineId.String()+executableFileExtension)) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetAbsoluteExecutableFilePath", - fields: fields{ - PipelineId: pipelineId, - Folder: Folder{ExecutableFileFolder: executableFileFolder}, - Extension: Extension{ExecutableFileExtension: executableFileExtension}, }, - want: absoluteExecutableFilePath, }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteExecutableFilePath(); got != tt.want { - t.Errorf("GetAbsoluteExecutableFilePath() = %v, want %v", got, tt.want) - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteLogFilePath(t *testing.T) { - baseFileFolder := "baseFileFolder" - absoluteLogFilePath, _ := filepath.Abs(filepath.Join(baseFileFolder, logFileName)) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ { - name: "GetAbsoluteLogFilePath", - fields: fields{ - Folder: Folder{BaseFolder: baseFileFolder}, + name: "Python LifeCycle", + args: args{ + sdk: playground.Sdk_SDK_PYTHON, + pipelineId: pipelineId, + workingDir: workingDir, }, - want: absoluteLogFilePath, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteLogFilePath(); got != tt.want { - t.Errorf("GetAbsoluteLogFilePath() = %v, want %v", got, tt.want) - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteSourceFileFolderPath(t *testing.T) { - sourceFileFolder := "sourceFileFolder" - absoluteSourceFileFolder, _ := filepath.Abs(sourceFileFolder) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetAbsoluteSourceFileFolderPath", - fields: fields{ - Folder: Folder{SourceFileFolder: sourceFileFolder}, + want: &LifeCycle{ + folderGlobs: []string{baseFileFolder}, + Paths: LifeCyclePaths{ + SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), pythonExecutableFileExtension), + AbsoluteSourceFileFolderPath: baseFileFolder, + AbsoluteSourceFilePath: filepath.Join(baseFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), pythonExecutableFileExtension)), + ExecutableFileName: fmt.Sprintf("%s%s", pipelineId.String(), pythonExecutableFileExtension), + AbsoluteExecutableFileFolderPath: baseFileFolder, + AbsoluteExecutableFilePath: filepath.Join(baseFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), pythonExecutableFileExtension)), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + }, }, - want: absoluteSourceFileFolder, }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteSourceFileFolderPath(); got != tt.want { - t.Errorf("GetAbsoluteSourceFileFolderPath() = %v, want %v", got, tt.want) - } - }) - } -} - -func TestLifeCycleDTO_GetAbsoluteSourceFilePath(t *testing.T) { - pipelineId := uuid.New() - sourceFileFolder := "sourceFileFolder" - sourceFileExtension := ".sourceFileExtension" - absoluteSourceFilePath, _ := filepath.Abs(filepath.Join(sourceFileFolder, pipelineId.String()+sourceFileExtension)) - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ { - name: "GetAbsoluteSourceFilePath", - fields: fields{ - PipelineId: pipelineId, - Folder: Folder{SourceFileFolder: sourceFileFolder}, - Extension: Extension{SourceFileExtension: sourceFileExtension}, + name: "Unavailable SDK", + args: args{ + sdk: playground.Sdk_SDK_UNSPECIFIED, + pipelineId: pipelineId, + workingDir: workingDir, }, - want: absoluteSourceFilePath, + want: nil, + wantErr: true, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetAbsoluteSourceFilePath(); got != tt.want { - t.Errorf("GetAbsoluteSourceFilePath() = %v, want %v", got, tt.want) + got, err := NewLifeCycle(tt.args.sdk, tt.args.pipelineId, tt.args.workingDir) + if (err != nil) != tt.wantErr { + t.Errorf("NewLifeCycle() error = %v, wantErr %v", err, tt.wantErr) + return } - }) - } -} - -func TestLifeCycleDTO_GetExecutableFileName(t *testing.T) { - pipelineId := uuid.New() - executableFileExtension := ".executableFileExtension" - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetExecutableFileName", - fields: fields{ - PipelineId: pipelineId, - Extension: Extension{ExecutableFileExtension: executableFileExtension}, - }, - want: pipelineId.String() + executableFileExtension, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, + if !tt.wantErr && !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { + t.Errorf("NewLifeCycle() got folderGlobs = %v, want folderGlobs %v", got.folderGlobs, tt.want.folderGlobs) } - if got := l.GetExecutableFileName(); got != tt.want { - t.Errorf("GetExecutableFileName() = %v, want %v", got, tt.want) + if !tt.wantErr && !checkPathsEqual(got.Paths, tt.want.Paths) { + t.Errorf("NewLifeCycle() got Paths = %v, want Paths %v", got.Paths, tt.want.Paths) } }) } } -func TestLifeCycleDTO_GetSourceFileName(t *testing.T) { - pipelineId := uuid.New() - sourceFileExtension := ".sourceFileExtension" - - type fields struct { - PipelineId uuid.UUID - Folder Folder - Extension Extension - ExecutableName func(uuid.UUID, string) (string, error) - } - tests := []struct { - name string - fields fields - want string - }{ - { - name: "GetSourceFileName", - fields: fields{ - PipelineId: pipelineId, - Extension: Extension{SourceFileExtension: sourceFileExtension}, - }, - want: pipelineId.String() + sourceFileExtension, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - l := &LifeCycleDTO{ - PipelineId: tt.fields.PipelineId, - Folder: tt.fields.Folder, - Extension: tt.fields.Extension, - ExecutableName: tt.fields.ExecutableName, - } - if got := l.GetSourceFileName(); got != tt.want { - t.Errorf("GetSourceFileName() = %v, want %v", got, tt.want) - } - }) - } +func checkPathsEqual(paths1, paths2 LifeCyclePaths) bool { + return paths1.SourceFileName == paths2.SourceFileName && + paths1.AbsoluteSourceFileFolderPath == paths2.AbsoluteSourceFileFolderPath && + paths1.AbsoluteSourceFilePath == paths2.AbsoluteSourceFilePath && + paths1.ExecutableFileName == paths2.ExecutableFileName && + paths1.AbsoluteExecutableFileFolderPath == paths2.AbsoluteExecutableFileFolderPath && + paths1.AbsoluteExecutableFilePath == paths2.AbsoluteExecutableFilePath && + paths1.AbsoluteBaseFolderPath == paths2.AbsoluteBaseFolderPath && + paths1.AbsoluteLogFilePath == paths2.AbsoluteLogFilePath } diff --git a/playground/backend/internal/fs_tool/go_fs_test.go b/playground/backend/internal/fs_tool/go_fs_test.go index d82eac2123c1..9ed1320d561d 100644 --- a/playground/backend/internal/fs_tool/go_fs_test.go +++ b/playground/backend/internal/fs_tool/go_fs_test.go @@ -16,18 +16,18 @@ package fs_tool import ( - "fmt" "github.com/google/uuid" + "path/filepath" "reflect" "testing" ) func Test_newGoLifeCycle(t *testing.T) { pipelineId := uuid.New() - workingDir := "workingDir" - baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) - srcFileFolder := baseFileFolder + "/src" - binFileFolder := baseFileFolder + "/bin" + workingDir, _ := filepath.Abs("workingDir") + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) + srcFileFolder := filepath.Join(baseFileFolder, "src") + binFileFolder := filepath.Join(baseFileFolder, "bin") type args struct { pipelineId uuid.UUID @@ -48,17 +48,15 @@ func Test_newGoLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Dto: LifeCycleDTO{ - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - Extension: Extension{ - SourceFileExtension: goSourceFileExtension, - ExecutableFileExtension: goExecutableFileExtension, - }, - PipelineId: pipelineId, + Paths: LifeCyclePaths{ + SourceFileName: pipelineId.String() + goSourceFileExtension, + AbsoluteSourceFileFolderPath: srcFileFolder, + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, pipelineId.String()+goSourceFileExtension), + ExecutableFileName: pipelineId.String() + goExecutableFileExtension, + AbsoluteExecutableFileFolderPath: binFileFolder, + AbsoluteExecutableFilePath: filepath.Join(binFileFolder, pipelineId.String()+goExecutableFileExtension), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), }, }, }, @@ -69,14 +67,8 @@ func Test_newGoLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newGoLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { - t.Errorf("newGoLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) - } - if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { - t.Errorf("newGoLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) - } - if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { - t.Errorf("newGoLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) + if !reflect.DeepEqual(got.Paths, tt.want.Paths) { + t.Errorf("newGoLifeCycle() Paths = %v, want %v", got.Paths, tt.want.Paths) } }) } diff --git a/playground/backend/internal/fs_tool/java_fs.go b/playground/backend/internal/fs_tool/java_fs.go index dd3b376b5b58..751f5c610174 100644 --- a/playground/backend/internal/fs_tool/java_fs.go +++ b/playground/backend/internal/fs_tool/java_fs.go @@ -24,20 +24,20 @@ import ( ) const ( - JavaSourceFileExtension = ".java" + javaSourceFileExtension = ".java" javaCompiledFileExtension = ".class" ) // newJavaLifeCycle creates LifeCycle with java SDK environment. func newJavaLifeCycle(pipelineId uuid.UUID, workingDir string) *LifeCycle { - javaLifeCycle := newCompilingLifeCycle(pipelineId, workingDir, JavaSourceFileExtension, javaCompiledFileExtension) - javaLifeCycle.Dto.ExecutableName = executableName + javaLifeCycle := newCompilingLifeCycle(pipelineId, workingDir, javaSourceFileExtension, javaCompiledFileExtension) + javaLifeCycle.Paths.ExecutableName = executableName return javaLifeCycle } // executableName returns name that should be executed (HelloWorld for HelloWorld.class for java SDK) func executableName(pipelineId uuid.UUID, workingDir string) (string, error) { - baseFileFolder := filepath.Join(workingDir, baseFileFolder, pipelineId.String()) + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) binFileFolder := filepath.Join(baseFileFolder, compiledFolderName) dirEntries, err := os.ReadDir(binFileFolder) if err != nil { diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index 1dfdb7061fc0..fe50c58c3302 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -16,7 +16,6 @@ package fs_tool import ( - "fmt" "github.com/google/uuid" "os" "path/filepath" @@ -26,10 +25,10 @@ import ( func Test_newJavaLifeCycle(t *testing.T) { pipelineId := uuid.New() - workingDir := "workingDir" - baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) - srcFileFolder := baseFileFolder + "/src" - binFileFolder := baseFileFolder + "/bin" + workingDir, _ := filepath.Abs("workingDir") + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) + srcFileFolder := filepath.Join(baseFileFolder, "src") + binFileFolder := filepath.Join(baseFileFolder, "bin") type args struct { pipelineId uuid.UUID @@ -50,18 +49,16 @@ func Test_newJavaLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Dto: LifeCycleDTO{ - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - Extension: Extension{ - SourceFileExtension: JavaSourceFileExtension, - ExecutableFileExtension: javaCompiledFileExtension, - }, - ExecutableName: executableName, - PipelineId: pipelineId, + Paths: LifeCyclePaths{ + SourceFileName: pipelineId.String() + javaSourceFileExtension, + AbsoluteSourceFileFolderPath: srcFileFolder, + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, pipelineId.String()+javaSourceFileExtension), + ExecutableFileName: pipelineId.String() + javaCompiledFileExtension, + AbsoluteExecutableFileFolderPath: binFileFolder, + AbsoluteExecutableFilePath: filepath.Join(binFileFolder, pipelineId.String()+javaCompiledFileExtension), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + ExecutableName: executableName, }, }, }, @@ -72,14 +69,8 @@ func Test_newJavaLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newJavaLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { - t.Errorf("newJavaLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) - } - if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { - t.Errorf("newJavaLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) - } - if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { - t.Errorf("newJavaLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) + if !checkPathsEqual(got.Paths, tt.want.Paths) { + t.Errorf("newJavaLifeCycle() Paths = %v, want %v", got.Paths, tt.want.Paths) } }) } @@ -109,7 +100,7 @@ func Test_executableName(t *testing.T) { // As a result, want to receive a name that should be executed name: "get executable name", prepare: func() { - compiled := filepath.Join(workDir, baseFileFolder, pipelineId.String(), compiledFolderName) + compiled := filepath.Join(workDir, executableFiles, pipelineId.String(), compiledFolderName) filePath := filepath.Join(compiled, "temp.class") err := os.WriteFile(filePath, []byte("TEMP_DATA"), 0600) if err != nil { diff --git a/playground/backend/internal/fs_tool/lc_constructor.go b/playground/backend/internal/fs_tool/lc_constructor.go index 1c810fec61ec..cdd224895728 100644 --- a/playground/backend/internal/fs_tool/lc_constructor.go +++ b/playground/backend/internal/fs_tool/lc_constructor.go @@ -21,49 +21,61 @@ import ( ) const ( - baseFileFolder = "executable_files" + executableFiles = "executable_files" sourceFolderName = "src" compiledFolderName = "bin" ) // newCompilingLifeCycle creates LifeCycle for compiled SDK environment. func newCompilingLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFileExtension string, compiledFileExtension string) *LifeCycle { - baseFileFolder := filepath.Join(workingDir, baseFileFolder, pipelineId.String()) + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) srcFileFolder := filepath.Join(baseFileFolder, sourceFolderName) binFileFolder := filepath.Join(baseFileFolder, compiledFolderName) + + srcFileName := pipelineId.String() + sourceFileExtension + absSrcFileFolderPath, _ := filepath.Abs(srcFileFolder) + absSrcFilePath, _ := filepath.Abs(filepath.Join(absSrcFileFolderPath, srcFileName)) + execFileName := pipelineId.String() + compiledFileExtension + absExecFileFolderPath, _ := filepath.Abs(binFileFolder) + absExecFilePath, _ := filepath.Abs(filepath.Join(absExecFileFolderPath, execFileName)) + absBaseFolderPath, _ := filepath.Abs(baseFileFolder) + absLogFilePath, _ := filepath.Abs(filepath.Join(absBaseFolderPath, logFileName)) + return &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, - Dto: LifeCycleDTO{ - PipelineId: pipelineId, - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: srcFileFolder, - ExecutableFileFolder: binFileFolder, - }, - Extension: Extension{ - SourceFileExtension: sourceFileExtension, - ExecutableFileExtension: compiledFileExtension, - }, + Paths: LifeCyclePaths{ + SourceFileName: srcFileName, + AbsoluteSourceFileFolderPath: absSrcFileFolderPath, + AbsoluteSourceFilePath: absSrcFilePath, + ExecutableFileName: execFileName, + AbsoluteExecutableFileFolderPath: absExecFileFolderPath, + AbsoluteExecutableFilePath: absExecFilePath, + AbsoluteBaseFolderPath: absBaseFolderPath, + AbsoluteLogFilePath: absLogFilePath, }, } } // newInterpretedLifeCycle creates LifeCycle for interpreted SDK environment. func newInterpretedLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFileExtension string) *LifeCycle { - sourceFileFolder := filepath.Join(workingDir, baseFileFolder, pipelineId.String()) + sourceFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) + + fileName := pipelineId.String() + sourceFileExtension + absFileFolderPath, _ := filepath.Abs(sourceFileFolder) + absFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, fileName)) + absLogFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, logFileName)) + return &LifeCycle{ folderGlobs: []string{sourceFileFolder}, - Dto: LifeCycleDTO{ - PipelineId: pipelineId, - Folder: Folder{ - BaseFolder: sourceFileFolder, - SourceFileFolder: sourceFileFolder, - ExecutableFileFolder: sourceFileFolder, - }, - Extension: Extension{ - ExecutableFileExtension: sourceFileExtension, - SourceFileExtension: sourceFileExtension, - }, + Paths: LifeCyclePaths{ + SourceFileName: fileName, + AbsoluteSourceFileFolderPath: absFileFolderPath, + AbsoluteSourceFilePath: absFilePath, + ExecutableFileName: fileName, + AbsoluteExecutableFileFolderPath: absFileFolderPath, + AbsoluteExecutableFilePath: absFilePath, + AbsoluteBaseFolderPath: absFileFolderPath, + AbsoluteLogFilePath: absLogFilePath, }, } } diff --git a/playground/backend/internal/fs_tool/python_fs_test.go b/playground/backend/internal/fs_tool/python_fs_test.go index 00d7773ea968..83aa641ea9ef 100644 --- a/playground/backend/internal/fs_tool/python_fs_test.go +++ b/playground/backend/internal/fs_tool/python_fs_test.go @@ -16,16 +16,16 @@ package fs_tool import ( - "fmt" "github.com/google/uuid" + "path/filepath" "reflect" "testing" ) func Test_newPythonLifeCycle(t *testing.T) { pipelineId := uuid.New() - workingDir := "workingDir" - baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) + workingDir, _ := filepath.Abs("workingDir") + baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) type args struct { pipelineId uuid.UUID @@ -46,17 +46,15 @@ func Test_newPythonLifeCycle(t *testing.T) { }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder}, - Dto: LifeCycleDTO{ - Folder: Folder{ - BaseFolder: baseFileFolder, - SourceFileFolder: baseFileFolder, - ExecutableFileFolder: baseFileFolder, - }, - Extension: Extension{ - SourceFileExtension: pythonExecutableFileExtension, - ExecutableFileExtension: pythonExecutableFileExtension, - }, - PipelineId: pipelineId, + Paths: LifeCyclePaths{ + SourceFileName: pipelineId.String() + pythonExecutableFileExtension, + AbsoluteSourceFileFolderPath: baseFileFolder, + AbsoluteSourceFilePath: filepath.Join(baseFileFolder, pipelineId.String()+pythonExecutableFileExtension), + ExecutableFileName: pipelineId.String() + pythonExecutableFileExtension, + AbsoluteExecutableFileFolderPath: baseFileFolder, + AbsoluteExecutableFilePath: filepath.Join(baseFileFolder, pipelineId.String()+pythonExecutableFileExtension), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), }, }, }, @@ -67,14 +65,8 @@ func Test_newPythonLifeCycle(t *testing.T) { if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { t.Errorf("newPythonLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) } - if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { - t.Errorf("newPythonLifeCycle() Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) - } - if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { - t.Errorf("newPythonLifeCycle() Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) - } - if !reflect.DeepEqual(got.Dto.PipelineId, tt.want.Dto.PipelineId) { - t.Errorf("newPythonLifeCycle() pipelineId = %v, want %v", got.Dto.PipelineId, tt.want.Dto.PipelineId) + if !checkPathsEqual(got.Paths, tt.want.Paths) { + t.Errorf("newPythonLifeCycle() Paths = %v, want %v", got.Paths, tt.want.Paths) } }) } diff --git a/playground/backend/internal/preparators/java_preparators.go b/playground/backend/internal/preparators/java_preparators.go index 26e5d6ecc0f7..07b0a6ab7015 100644 --- a/playground/backend/internal/preparators/java_preparators.go +++ b/playground/backend/internal/preparators/java_preparators.go @@ -16,7 +16,6 @@ package preparators import ( - "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/logger" "beam.apache.org/playground/backend/internal/validators" "bufio" @@ -203,7 +202,7 @@ func changeJavaTestFileName(args ...interface{}) error { func renameJavaFile(filePath string, className string) error { currentFileName := filepath.Base(filePath) - newFilePath := strings.Replace(filePath, currentFileName, fmt.Sprintf("%s%s", className, fs_tool.JavaSourceFileExtension), 1) + newFilePath := strings.Replace(filePath, currentFileName, fmt.Sprintf("%s%s", className, filepath.Ext(currentFileName)), 1) err := os.Rename(filePath, newFilePath) return err } diff --git a/playground/backend/internal/preparators/java_preparators_test.go b/playground/backend/internal/preparators/java_preparators_test.go index e964df09db54..145d703a4c26 100644 --- a/playground/backend/internal/preparators/java_preparators_test.go +++ b/playground/backend/internal/preparators/java_preparators_test.go @@ -58,14 +58,14 @@ func Test_replace(t *testing.T) { }, { name: "original file exists", - args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), classWithPublicModifierPattern, classWithoutPublicModifierPattern}}, + args: args{[]interface{}{lc.Paths.AbsoluteSourceFilePath, classWithPublicModifierPattern, classWithoutPublicModifierPattern}}, wantCode: codeWithoutPublicClass, wantErr: false, }, { // Test that file where package is used changes to import all dependencies from this package name: "original file with package", - args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), packagePattern, importStringPattern}}, + args: args{[]interface{}{lc.Paths.AbsoluteSourceFilePath, packagePattern, importStringPattern}}, wantCode: codeWithImportedPackage, wantErr: false, }, @@ -137,7 +137,7 @@ func Test_changeJavaTestFileName(t *testing.T) { { // Test that file changes its name to the name of its public class name: "file with java unit test code to be renamed", - args: args{[]interface{}{lc.Dto.GetAbsoluteSourceFilePath(), &validationResults}}, + args: args{[]interface{}{lc.Paths.AbsoluteSourceFilePath, &validationResults}}, wantErr: false, wantName: "Class.java", }, @@ -147,7 +147,7 @@ func Test_changeJavaTestFileName(t *testing.T) { if err := changeJavaTestFileName(tt.args.args...); (err != nil) != tt.wantErr { t.Errorf("changeJavaTestFileName() error = %v, wantErr %v", err, tt.wantErr) } - files, err := filepath.Glob(fmt.Sprintf("%s/*java", lc.Dto.GetAbsoluteSourceFileFolderPath())) + files, err := filepath.Glob(fmt.Sprintf("%s/*java", lc.Paths.AbsoluteSourceFileFolderPath)) if err != nil { t.Errorf("changeJavaTestFileName() error = %v, wantErr %v", err, tt.wantErr) } diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index f5635aa5ac4a..19847de0fd02 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -32,25 +32,25 @@ const ( ) // SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +func SetupExecutorBuilder(paths fs_tool.LifeCyclePaths, pipelineOptions string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk if sdk == pb.Sdk_SDK_JAVA { pipelineOptions = utils.ReplaceSpacesWithEquals(pipelineOptions) } - val, err := utils.GetValidators(sdk, lcDto.GetAbsoluteSourceFilePath()) + val, err := utils.GetValidators(sdk, paths.AbsoluteSourceFilePath) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, lcDto.GetAbsoluteSourceFilePath()) + prep, err := utils.GetPreparators(sdk, paths.AbsoluteSourceFilePath) if err != nil { return nil, err } executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(lcDto.GetAbsoluteExecutableFilePath()). - WithWorkingDir(lcDto.GetAbsoluteBaseFolderPath()). + WithExecutableFileName(paths.AbsoluteExecutableFilePath). + WithWorkingDir(paths.AbsoluteBaseFolderPath). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -58,7 +58,7 @@ func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sd WithCompiler(). WithCommand(executorConfig.CompileCmd). WithArgs(executorConfig.CompileArgs). - WithFileName(lcDto.GetAbsoluteSourceFilePath()). + WithFileName(paths.AbsoluteSourceFilePath). WithRunner(). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). @@ -66,7 +66,7 @@ func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sd WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). - WithWorkingDir(lcDto.GetAbsoluteSourceFileFolderPath()). + WithWorkingDir(paths.AbsoluteSourceFileFolderPath). ExecutorBuilder switch sdk { @@ -74,20 +74,20 @@ func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sd args := make([]string, 0) for _, arg := range executorConfig.RunArgs { if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) + logConfigFilePath := filepath.Join(paths.AbsoluteBaseFolderPath, javaLogConfigFileName) arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) } args = append(args, arg) } builder = builder.WithRunner().WithArgs(args).ExecutorBuilder - builder = builder.WithTestRunner().WithWorkingDir(lcDto.GetAbsoluteBaseFolderPath()).ExecutorBuilder //change directory for unit test + builder = builder.WithTestRunner().WithWorkingDir(paths.AbsoluteBaseFolderPath).ExecutorBuilder //change directory for unit test case pb.Sdk_SDK_GO: //go run command is executable file itself builder = builder. WithExecutableFileName(""). WithRunner(). - WithCommand(lcDto.GetAbsoluteExecutableFilePath()).ExecutorBuilder + WithCommand(paths.AbsoluteExecutableFilePath).ExecutorBuilder case pb.Sdk_SDK_PYTHON: - builder = *builder.WithExecutableFileName(lcDto.GetAbsoluteExecutableFilePath()) + builder = *builder.WithExecutableFileName(paths.AbsoluteExecutableFilePath) case pb.Sdk_SDK_SCIO: return nil, fmt.Errorf("SCIO is not supported yet") default: @@ -97,7 +97,7 @@ func SetupExecutorBuilder(lcDto fs_tool.LifeCycleDTO, pipelineOptions string, sd } // GetFileNameFromFolder return a name of the first file in a specified folder -func GetFileNameFromFolder(folderAbsolutePath string) string { - files, _ := filepath.Glob(fmt.Sprintf("%s/*%s", folderAbsolutePath, fs_tool.JavaSourceFileExtension)) +func GetFileNameFromFolder(folderAbsolutePath, extension string) string { + files, _ := filepath.Glob(fmt.Sprintf("%s/*%s", folderAbsolutePath, extension)) return files[0] } diff --git a/playground/backend/internal/setup_tools/builder/setup_builder_test.go b/playground/backend/internal/setup_tools/builder/setup_builder_test.go index 653823b04614..f862c7c304a4 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -47,7 +47,7 @@ func TestSetupExecutor(t *testing.T) { panic(err) } - srcFilePath := lc.Dto.GetAbsoluteSourceFilePath() + srcFilePath := lc.Paths.AbsoluteSourceFilePath sdkEnv := environment.NewBeamEnvs(sdk, executorConfig, "", 0) val, err := utils.GetValidators(sdk, srcFilePath) @@ -60,8 +60,8 @@ func TestSetupExecutor(t *testing.T) { } wantExecutor := executors.NewExecutorBuilder(). - WithExecutableFileName(lc.Dto.GetAbsoluteExecutableFilePath()). - WithWorkingDir(lc.Dto.GetAbsoluteBaseFolderPath()). + WithExecutableFileName(lc.Paths.AbsoluteExecutableFilePath). + WithWorkingDir(lc.Paths.AbsoluteBaseFolderPath). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -77,11 +77,11 @@ func TestSetupExecutor(t *testing.T) { WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). - WithWorkingDir(lc.Dto.GetAbsoluteBaseFolderPath()). + WithWorkingDir(lc.Paths.AbsoluteBaseFolderPath). ExecutorBuilder type args struct { - dto fs_tool.LifeCycleDTO + dto fs_tool.LifeCyclePaths pipelineOptions string sdkEnv *environment.BeamEnvs } @@ -95,7 +95,7 @@ func TestSetupExecutor(t *testing.T) { // Test case with calling Setup with incorrect SDK. // As a result, want to receive an error. name: "incorrect sdk", - args: args{lc.Dto, pipelineOptions, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "", 0)}, + args: args{lc.Paths, pipelineOptions, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "", 0)}, want: nil, wantErr: true, }, @@ -103,7 +103,7 @@ func TestSetupExecutor(t *testing.T) { // Test case with calling Setup with correct SDK. // As a result, want to receive an expected builder. name: "correct sdk", - args: args{lc.Dto, pipelineOptions, sdkEnv}, + args: args{lc.Paths, pipelineOptions, sdkEnv}, want: &wantExecutor, wantErr: false, }, diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go index 648760ef9b85..6f778ece70cc 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go @@ -34,7 +34,7 @@ const ( javaLogFilePlaceholder = "{logFilePath}" goModFileName = "go.mod" goSumFileName = "go.sum" - baseFileFolder = "executable_files" + executableFiles = "executable_files" ) // Setup returns fs_tool.LifeCycle. @@ -81,11 +81,11 @@ func Setup(sdk pb.Sdk, code string, pipelineId uuid.UUID, workingDir string, pre // prepareGoFiles prepares file for Go environment. // Copy go.mod and go.sum file from /path/to/preparedModDir to /path/to/workingDir/executable_files/{pipelineId} func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uuid.UUID) error { - if err := lc.CopyFile(goModFileName, preparedModDir, lc.Dto.GetAbsoluteBaseFolderPath()); err != nil { + if err := lc.CopyFile(goModFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil { logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goModFileName, err.Error()) return err } - if err := lc.CopyFile(goSumFileName, preparedModDir, lc.Dto.GetAbsoluteBaseFolderPath()); err != nil { + if err := lc.CopyFile(goSumFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil { logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goSumFileName, err.Error()) return err } @@ -96,12 +96,12 @@ func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uui // Copy log config file from /path/to/workingDir to /path/to/workingDir/executable_files/{pipelineId} // and update this file according to pipeline. func prepareJavaFiles(lc *fs_tool.LifeCycle, workingDir string, pipelineId uuid.UUID) error { - err := lc.CopyFile(javaLogConfigFileName, workingDir, lc.Dto.GetAbsoluteBaseFolderPath()) + err := lc.CopyFile(javaLogConfigFileName, workingDir, lc.Paths.AbsoluteBaseFolderPath) if err != nil { logger.Errorf("%s: error during copying logging.properties file: %s\n", pipelineId, err.Error()) return err } - err = updateJavaLogConfigFile(lc.Dto) + err = updateJavaLogConfigFile(lc.Paths) if err != nil { logger.Errorf("%s: error during updating logging.properties file: %s\n", pipelineId, err.Error()) return err @@ -110,9 +110,9 @@ func prepareJavaFiles(lc *fs_tool.LifeCycle, workingDir string, pipelineId uuid. } // updateJavaLogConfigFile updates java log config file according to pipeline -func updateJavaLogConfigFile(lcDto fs_tool.LifeCycleDTO) error { - logConfigFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) - logConfigUpdatedFilePath := filepath.Join(lcDto.GetAbsoluteBaseFolderPath(), javaTmpLogConfigFile) +func updateJavaLogConfigFile(paths fs_tool.LifeCyclePaths) error { + logConfigFilePath := filepath.Join(paths.AbsoluteBaseFolderPath, javaLogConfigFileName) + logConfigUpdatedFilePath := filepath.Join(paths.AbsoluteBaseFolderPath, javaTmpLogConfigFile) if _, err := os.Stat(logConfigFilePath); os.IsNotExist(err) { return err } @@ -130,7 +130,7 @@ func updateJavaLogConfigFile(lcDto fs_tool.LifeCycleDTO) error { for scanner.Scan() { line := scanner.Text() - line = strings.ReplaceAll(line, javaLogFilePlaceholder, lcDto.GetAbsoluteLogFilePath()) + line = strings.ReplaceAll(line, javaLogFilePlaceholder, paths.AbsoluteLogFilePath) if _, err = io.WriteString(updatedFile, line+"\n"); err != nil { return err } diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go index a4011546d074..831f6445b885 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go @@ -18,24 +18,29 @@ package life_cycle import ( playground "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/fs_tool" + "fmt" "github.com/google/uuid" "io/fs" "os" "path/filepath" - "reflect" "testing" ) const ( - workingDir = "workingDir" - sourceFolder = "src" - executableFolder = "bin" - javaSourceFileExtension = ".java" + workingDir = "workingDir" + sourceFolder = "src" + executableFolder = "bin" + javaSourceFileExtension = ".java" + javaCompiledFileExtension = ".class" + logFileName = "logs.log" ) func TestSetup(t *testing.T) { errorPipelineId := uuid.New() successPipelineId := uuid.New() + baseFileFolder, _ := filepath.Abs(filepath.Join(workingDir, executableFiles, successPipelineId.String())) + srcFileFolder := filepath.Join(baseFileFolder, sourceFolder) + execFileFolder := filepath.Join(baseFileFolder, executableFolder) err := os.MkdirAll(workingDir, fs.ModePerm) if err != nil { @@ -46,11 +51,6 @@ func TestSetup(t *testing.T) { panic(err) } defer os.RemoveAll(workingDir) - - lc, err := fs_tool.NewLifeCycle(playground.Sdk_SDK_JAVA, successPipelineId, workingDir) - if err != nil { - panic(err) - } type args struct { sdk playground.Sdk code string @@ -76,7 +76,7 @@ func TestSetup(t *testing.T) { workingDir: workingDir, }, check: func() bool { - if _, err := os.Stat(filepath.Join(baseFileFolder, errorPipelineId.String())); os.IsNotExist(err) { + if _, err := os.Stat(filepath.Join(executableFiles, errorPipelineId.String())); os.IsNotExist(err) { return true } return false @@ -96,21 +96,32 @@ func TestSetup(t *testing.T) { preparedModDir: "", }, check: func() bool { - if _, err := os.Stat(filepath.Join(workingDir, baseFileFolder, successPipelineId.String())); os.IsNotExist(err) { + if _, err := os.Stat(filepath.Join(workingDir, executableFiles, successPipelineId.String())); os.IsNotExist(err) { return false } - if _, err := os.Stat(filepath.Join(workingDir, baseFileFolder, successPipelineId.String(), sourceFolder)); os.IsNotExist(err) { + if _, err := os.Stat(filepath.Join(workingDir, executableFiles, successPipelineId.String(), sourceFolder)); os.IsNotExist(err) { return false } - if _, err := os.Stat(filepath.Join(workingDir, baseFileFolder, successPipelineId.String(), executableFolder)); os.IsNotExist(err) { + if _, err := os.Stat(filepath.Join(workingDir, executableFiles, successPipelineId.String(), executableFolder)); os.IsNotExist(err) { return false } - if _, err := os.Stat(filepath.Join(workingDir, baseFileFolder, successPipelineId.String(), sourceFolder, successPipelineId.String()+javaSourceFileExtension)); os.IsNotExist(err) { + if _, err := os.Stat(filepath.Join(workingDir, executableFiles, successPipelineId.String(), sourceFolder, successPipelineId.String()+javaSourceFileExtension)); os.IsNotExist(err) { return false } return true }, - want: lc, + want: &fs_tool.LifeCycle{ + Paths: fs_tool.LifeCyclePaths{ + SourceFileName: fmt.Sprintf("%s%s", successPipelineId.String(), javaSourceFileExtension), + AbsoluteSourceFileFolderPath: srcFileFolder, + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, fmt.Sprintf("%s%s", successPipelineId.String(), javaSourceFileExtension)), + ExecutableFileName: fmt.Sprintf("%s%s", successPipelineId.String(), javaCompiledFileExtension), + AbsoluteExecutableFileFolderPath: execFileFolder, + AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", successPipelineId.String(), javaCompiledFileExtension)), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + }, + }, wantErr: false, }, } @@ -122,17 +133,22 @@ func TestSetup(t *testing.T) { return } if got != nil { - if !reflect.DeepEqual(got.Dto.Folder, tt.want.Dto.Folder) { - t.Errorf("Setup() got.Folder = %v, want %v", got.Dto.Folder, tt.want.Dto.Folder) - } - if !reflect.DeepEqual(got.Dto.Extension, tt.want.Dto.Extension) { - t.Errorf("Setup() got.Extension = %v, want %v", got.Dto.Extension, tt.want.Dto.Extension) - } - if !tt.check() { - t.Errorf("Setup() doesn't prepare necessary files/folders") + if !checkPathsEqual(got.Paths, tt.want.Paths) { + t.Errorf("Setup() got.Paths = %v, want %v", got.Paths, tt.want.Paths) } } - os.RemoveAll("executable_files") + os.RemoveAll(executableFiles) }) } } + +func checkPathsEqual(paths1, paths2 fs_tool.LifeCyclePaths) bool { + return paths1.SourceFileName == paths2.SourceFileName && + paths1.AbsoluteSourceFileFolderPath == paths2.AbsoluteSourceFileFolderPath && + paths1.AbsoluteSourceFilePath == paths2.AbsoluteSourceFilePath && + paths1.ExecutableFileName == paths2.ExecutableFileName && + paths1.AbsoluteExecutableFileFolderPath == paths2.AbsoluteExecutableFileFolderPath && + paths1.AbsoluteExecutableFilePath == paths2.AbsoluteExecutableFilePath && + paths1.AbsoluteBaseFolderPath == paths2.AbsoluteBaseFolderPath && + paths1.AbsoluteLogFilePath == paths2.AbsoluteLogFilePath +} From e7b2afc1e8bb01795bf8796fbcb308eaa2b7b009 Mon Sep 17 00:00:00 2001 From: AydarZaynutdinov Date: Tue, 11 Jan 2022 16:19:10 +0300 Subject: [PATCH 3/5] [BEAM-13398][Playground] Merge with master --- .../internal/environment/application_test.go | 2 - .../backend/internal/fs_tool/fs_test.go | 43 ++++++++++--------- .../backend/internal/fs_tool/java_fs_test.go | 8 ++-- .../preparators/java_preparators_test.go | 2 - 4 files changed, 26 insertions(+), 29 deletions(-) diff --git a/playground/backend/internal/environment/application_test.go b/playground/backend/internal/environment/application_test.go index 7b4eab10c0db..92f2aeb1ec8d 100644 --- a/playground/backend/internal/environment/application_test.go +++ b/playground/backend/internal/environment/application_test.go @@ -348,8 +348,6 @@ func TestApplicationEnvs_PipelinesFolder(t *testing.T) { workingDir string cacheEnvs *CacheEnvs pipelineExecuteTimeout time.Duration - launchSite string - projectId string pipelinesFolder string } tests := []struct { diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index 20e2d003dfb4..58ddbf3a0aea 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -26,9 +26,10 @@ import ( ) const ( - sourceDir = "sourceDir" - destinationDir = "destinationDir" - testFileMode = 0755 + sourceDir = "sourceDir" + destinationDir = "destinationDir" + testFileMode = 0755 + pipelinesFolder = "executable_files" ) func prepareFiles() error { @@ -259,15 +260,15 @@ func TestLifeCycle_DeleteFolders(t *testing.T) { func TestNewLifeCycle(t *testing.T) { pipelineId := uuid.New() - workingDir, _ := filepath.Abs("workingDir") - baseFileFolder := filepath.Join(workingDir, executableFiles, pipelineId.String()) + pipelinesFolder, _ := filepath.Abs(pipelinesFolder) + baseFileFolder := filepath.Join(pipelinesFolder, pipelineId.String()) srcFileFolder := filepath.Join(baseFileFolder, "src") execFileFolder := filepath.Join(baseFileFolder, "bin") type args struct { - sdk pb.Sdk - pipelineId uuid.UUID - workingDir string + sdk pb.Sdk + pipelineId uuid.UUID + pipelinesFolder string } tests := []struct { name string @@ -278,9 +279,9 @@ func TestNewLifeCycle(t *testing.T) { { name: "Java LifeCycle", args: args{ - sdk: pb.Sdk_SDK_JAVA, - pipelineId: pipelineId, - workingDir: workingDir, + sdk: pb.Sdk_SDK_JAVA, + pipelineId: pipelineId, + pipelinesFolder: pipelinesFolder, }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, execFileFolder}, @@ -299,9 +300,9 @@ func TestNewLifeCycle(t *testing.T) { { name: "Go LifeCycle", args: args{ - sdk: pb.Sdk_SDK_GO, - pipelineId: pipelineId, - workingDir: workingDir, + sdk: pb.Sdk_SDK_GO, + pipelineId: pipelineId, + pipelinesFolder: pipelinesFolder, }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, execFileFolder}, @@ -320,9 +321,9 @@ func TestNewLifeCycle(t *testing.T) { { name: "Python LifeCycle", args: args{ - sdk: pb.Sdk_SDK_PYTHON, - pipelineId: pipelineId, - workingDir: workingDir, + sdk: pb.Sdk_SDK_PYTHON, + pipelineId: pipelineId, + pipelinesFolder: pipelinesFolder, }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder}, @@ -341,9 +342,9 @@ func TestNewLifeCycle(t *testing.T) { { name: "Unavailable SDK", args: args{ - sdk: pb.Sdk_SDK_UNSPECIFIED, - pipelineId: pipelineId, - workingDir: workingDir, + sdk: pb.Sdk_SDK_UNSPECIFIED, + pipelineId: pipelineId, + pipelinesFolder: pipelinesFolder, }, want: nil, wantErr: true, @@ -351,7 +352,7 @@ func TestNewLifeCycle(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := NewLifeCycle(tt.args.sdk, tt.args.pipelineId, tt.args.workingDir) + got, err := NewLifeCycle(tt.args.sdk, tt.args.pipelineId, tt.args.pipelinesFolder) if (err != nil) != tt.wantErr { t.Errorf("NewLifeCycle() error = %v, wantErr %v", err, tt.wantErr) return diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index 14c64c525494..6a414cf8db0b 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -31,8 +31,8 @@ func Test_newJavaLifeCycle(t *testing.T) { binFileFolder := filepath.Join(baseFileFolder, "bin") type args struct { - pipelineId uuid.UUID - workingDir string + pipelineId uuid.UUID + pipelinesFolder string } tests := []struct { name string @@ -44,8 +44,8 @@ func Test_newJavaLifeCycle(t *testing.T) { // As a result, want to receive an expected java life cycle. name: "newJavaLifeCycle", args: args{ - pipelineId: pipelineId, - workingDir: workingDir, + pipelineId: pipelineId, + pipelinesFolder: filepath.Join(workingDir, pipelinesFolder), }, want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, diff --git a/playground/backend/internal/preparators/java_preparators_test.go b/playground/backend/internal/preparators/java_preparators_test.go index 9ca7361b3434..145d703a4c26 100644 --- a/playground/backend/internal/preparators/java_preparators_test.go +++ b/playground/backend/internal/preparators/java_preparators_test.go @@ -28,8 +28,6 @@ import ( "testing" ) -const pipelinesFolder = "executable_files" - func Test_replace(t *testing.T) { codeWithPublicClass := "package org.apache.beam.sdk.transforms; \n public class Class {\n public static void main(String[] args) {\n System.out.println(\"Hello World!\");\n }\n}" codeWithoutPublicClass := "package org.apache.beam.sdk.transforms; \n class Class {\n public static void main(String[] args) {\n System.out.println(\"Hello World!\");\n }\n}" From 3028d5b50b4445a664c2b355803adbf1048cd88f Mon Sep 17 00:00:00 2001 From: AydarZaynutdinov Date: Tue, 11 Jan 2022 16:26:08 +0300 Subject: [PATCH 4/5] [BEAM-13398][Playground] Merge with master Fix using `executable_files` value Fix comments --- playground/backend/internal/fs_tool/fs.go | 12 ++++++------ playground/backend/internal/utils/system_utils.go | 3 +-- .../backend/internal/utils/system_utils_test.go | 9 ++++----- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index c2fb7680d788..05b1b902a3e5 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -33,13 +33,13 @@ const ( // LifeCyclePaths contains all files/folders paths type LifeCyclePaths struct { SourceFileName string // {pipelineId}.{sourceFileExtension} - AbsoluteSourceFileFolderPath string // /path/to/workingDir/executable_files/{pipelineId}/src - AbsoluteSourceFilePath string // /path/to/workingDir/executable_files/{pipelineId}/src/{pipelineId}.{sourceFileExtension} + AbsoluteSourceFileFolderPath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/src + AbsoluteSourceFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/src/{pipelineId}.{sourceFileExtension} ExecutableFileName string // {pipelineId}.{executableFileExtension} - AbsoluteExecutableFileFolderPath string // /path/to/workingDir/executable_files/{pipelineId}/bin - AbsoluteExecutableFilePath string // /path/to/workingDir/executable_files/{pipelineId}/bin/{pipelineId}.{executableFileExtension} - AbsoluteBaseFolderPath string // /path/to/workingDir/executable_files/{pipelineId} - AbsoluteLogFilePath string // /path/to/workingDir/executable_files/{pipelineId}/logs.log + AbsoluteExecutableFileFolderPath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/bin + AbsoluteExecutableFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/bin/{pipelineId}.{executableFileExtension} + AbsoluteBaseFolderPath string // /path/to/workingDir/pipelinesFolder/{pipelineId} + AbsoluteLogFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/logs.log ExecutableName func(uuid.UUID, string) (string, error) } diff --git a/playground/backend/internal/utils/system_utils.go b/playground/backend/internal/utils/system_utils.go index 2a357055febc..51628fec1c8f 100644 --- a/playground/backend/internal/utils/system_utils.go +++ b/playground/backend/internal/utils/system_utils.go @@ -56,11 +56,10 @@ func GetReadinessFunction(envs *environment.Environment) func(writer http.Respon } // checkNumOfTheParallelJobs checks the number of currently working code executions. -// It counts by the number of the /path/to/workingDir/executable_files/{pipelineId} folders. +// It counts by the number of the /path/to/workingDir/executableFiles/{pipelineId} folders. // If it is equals or more than numOfParallelJobs, then returns false. // If it is less than numOfParallelJobs, then returns true. func checkNumOfTheParallelJobs(workingDir string, numOfParallelJobs int) bool { - // TODO [BEAM-13308] add getting of dir executable_files from environments. baseFileFolder := filepath.Join(workingDir, executableFiles) _, err := os.Stat(baseFileFolder) if os.IsNotExist(err) { diff --git a/playground/backend/internal/utils/system_utils_test.go b/playground/backend/internal/utils/system_utils_test.go index b745934952c3..34b6d3335ef7 100644 --- a/playground/backend/internal/utils/system_utils_test.go +++ b/playground/backend/internal/utils/system_utils_test.go @@ -47,7 +47,6 @@ func TestGetFuncName(t *testing.T) { } func Test_checkNumOfTheParallelJobs(t *testing.T) { - baseFileFolder := "executable_files" type args struct { workingDir string numOfParallelJobs int @@ -78,7 +77,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { numOfParallelJobs: 2, }, prepareFunc: func() { - err := os.MkdirAll(filepath.Join(baseFileFolder, "1"), fs.ModePerm) + err := os.MkdirAll(filepath.Join(executableFiles, "1"), fs.ModePerm) if err != nil { panic(err) } @@ -94,7 +93,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { numOfParallelJobs: 1, }, prepareFunc: func() { - err := os.MkdirAll(filepath.Join(baseFileFolder, "1"), fs.ModePerm) + err := os.MkdirAll(filepath.Join(executableFiles, "1"), fs.ModePerm) if err != nil { panic(err) } @@ -110,7 +109,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { numOfParallelJobs: 0, }, prepareFunc: func() { - err := os.MkdirAll(filepath.Join(baseFileFolder, "1"), fs.ModePerm) + err := os.MkdirAll(filepath.Join(executableFiles, "1"), fs.ModePerm) if err != nil { panic(err) } @@ -124,7 +123,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { if got := checkNumOfTheParallelJobs(tt.args.workingDir, tt.args.numOfParallelJobs); got != tt.want { t.Errorf("checkNumOfTheParallelJobs() = %v, want %v", got, tt.want) } - os.RemoveAll(baseFileFolder) + os.RemoveAll(executableFiles) }) } } From 2fc425f2537333189a085af5f19ddeb1fefa0ae5 Mon Sep 17 00:00:00 2001 From: AydarZaynutdinov Date: Wed, 12 Jan 2022 10:47:01 +0300 Subject: [PATCH 5/5] [BEAM-13398][Playground] renaming --- playground/backend/internal/fs_tool/fs.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 05b1b902a3e5..0d0b5231d0ce 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -64,8 +64,8 @@ func NewLifeCycle(sdk pb.Sdk, pipelineId uuid.UUID, pipelinesFolder string) (*Li } // CreateFolders creates all folders which will be used for code execution. -func (l *LifeCycle) CreateFolders() error { - for _, folder := range l.folderGlobs { +func (lc *LifeCycle) CreateFolders() error { + for _, folder := range lc.folderGlobs { err := os.MkdirAll(folder, fs.ModePerm) if err != nil { return err @@ -75,8 +75,8 @@ func (l *LifeCycle) CreateFolders() error { } // DeleteFolders deletes all previously provisioned folders. -func (l *LifeCycle) DeleteFolders() error { - for _, folder := range l.folderGlobs { +func (lc *LifeCycle) DeleteFolders() error { + for _, folder := range lc.folderGlobs { err := os.RemoveAll(folder) if err != nil { return err @@ -86,12 +86,12 @@ func (l *LifeCycle) DeleteFolders() error { } // CreateSourceCodeFile creates an executable file (i.e. file.{sourceFileExtension}). -func (l *LifeCycle) CreateSourceCodeFile(code string) error { - if _, err := os.Stat(l.Paths.AbsoluteSourceFileFolderPath); os.IsNotExist(err) { +func (lc *LifeCycle) CreateSourceCodeFile(code string) error { + if _, err := os.Stat(lc.Paths.AbsoluteSourceFileFolderPath); os.IsNotExist(err) { return err } - filePath := l.Paths.AbsoluteSourceFilePath + filePath := lc.Paths.AbsoluteSourceFilePath err := os.WriteFile(filePath, []byte(code), fileMode) if err != nil { return err @@ -100,7 +100,7 @@ func (l *LifeCycle) CreateSourceCodeFile(code string) error { } // CopyFile copies a file with fileName from sourceDir to destinationDir. -func (l *LifeCycle) CopyFile(fileName, sourceDir, destinationDir string) error { +func (lc *LifeCycle) CopyFile(fileName, sourceDir, destinationDir string) error { absSourcePath := filepath.Join(sourceDir, fileName) absDestinationPath := filepath.Join(destinationDir, fileName) sourceFileStat, err := os.Stat(absSourcePath)