diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 8e6d59ec9148..f02171a20543 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -34,7 +34,6 @@ import ( "io" "os" "os/exec" - "path/filepath" "reflect" "sync" "time" @@ -61,117 +60,62 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl DeleteFolders(pipelineId, lc) }(lc) - errorChannel := make(chan error, 1) - successChannel := make(chan bool, 1) cancelChannel := make(chan bool, 1) - stopReadLogsChannel := make(chan bool, 1) - finishReadLogsChannel := make(chan bool, 1) + var validationResults sync.Map go cancelCheck(pipelineLifeCycleCtx, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc.Paths, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) - if err != nil { - _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) + executor := validateStep(ctx, cacheService, &lc.Paths, pipelineId, sdkEnv, pipelineLifeCycleCtx, &validationResults, cancelChannel) + if executor == nil { return } - executor := executorBuilder.Build() - // Validate - logger.Infof("%s: Validate() ...\n", pipelineId) - validateFunc := executor.Validate() - // Run validate function - go validateFunc(successChannel, errorChannel, &validationResults) - // Start of the monitoring of background tasks (validate function/cancellation/timeout) - ok, err := reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) - if err != nil { - return - } - if !ok { - // Validate step is finished, but code isn't valid - err := <-errorChannel - _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, []byte(err.Error()), pipelineId, cache.ValidationOutput, cacheService, "Validate", pb.Status_STATUS_VALIDATION_ERROR) - return - } - // Validate step is finished and code is valid - if err := processSuccess(pipelineLifeCycleCtx, pipelineId, cacheService, "Validate", pb.Status_STATUS_PREPARING); err != nil { + executor = prepareStep(ctx, cacheService, &lc.Paths, pipelineId, sdkEnv, pipelineLifeCycleCtx, &validationResults, cancelChannel) + if executor == nil { return } - // Prepare - logger.Infof("%s: Prepare() ...\n", pipelineId) - prepareFunc := executor.Prepare() - // Run prepare function - go prepareFunc(successChannel, errorChannel, &validationResults) + // Check if is unit test + validateIsUnitTest, _ := validationResults.Load(validators.UnitTestValidatorName) + isUnitTest := validateIsUnitTest.(bool) - // Start of the monitoring of background tasks (prepare function/cancellation/timeout) - ok, err = reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) - if err != nil { - return - } - if !ok { - // Prepare step is finished, but code couldn't be prepared (some error during prepare step) - err := <-errorChannel - _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, []byte(err.Error()), pipelineId, cache.PreparationOutput, cacheService, "Prepare", pb.Status_STATUS_PREPARATION_ERROR) - return - } - // Prepare step is finished and code is prepared - if err := processSuccess(pipelineLifeCycleCtx, pipelineId, cacheService, "Prepare", pb.Status_STATUS_COMPILING); err != nil { + executor = compileStep(ctx, cacheService, &lc.Paths, pipelineId, sdkEnv, isUnitTest, pipelineLifeCycleCtx, cancelChannel) + if executor == nil { return } - // Check if unit test - validateIsUnitTest, _ := validationResults.Load(validators.UnitTestValidatorName) - isUnitTest := validateIsUnitTest.(bool) + // Run/RunTest + runStep(ctx, cacheService, &lc.Paths, pipelineId, isUnitTest, sdkEnv, pipelineOptions, pipelineLifeCycleCtx, cancelChannel) +} - // 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 - } - } else { // in case of Java, Go (not unit test), Scala - need compile step - // Compile - if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { - executor = executorBuilder.WithCompiler(). - 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) - var compileError bytes.Buffer - var compileOutput bytes.Buffer - runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) +func runStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.LifeCyclePaths, pipelineId uuid.UUID, isUnitTest bool, sdkEnv *environment.BeamEnvs, pipelineOptions string, pipelineLifeCycleCtx context.Context, cancelChannel chan bool) { + errorChannel, successChannel := createStatusChannels() + stopReadLogsChannel := make(chan bool, 1) + finishReadLogsChannel := make(chan bool, 1) - // Start of the monitoring of background tasks (compile step/cancellation/timeout) - ok, err = reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) - if err != nil { - return - } - if !ok { // Compile step is finished, but code couldn't be compiled (some typos for example) - err := <-errorChannel - _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, compileError.Bytes(), pipelineId, cache.CompileOutput, cacheService, "Compile", pb.Status_STATUS_COMPILE_ERROR) - return - } // Compile step is finished and code is compiled - if err := processCompileSuccess(pipelineLifeCycleCtx, compileOutput.Bytes(), pipelineId, cacheService); err != nil { - return - } + executorBuilder := executors.NewExecutorBuilder() + err := error(nil) + if isUnitTest { + executorBuilder, err = builder.TestRunner(paths, sdkEnv) + } else { + executorBuilder, err = builder.Runner(paths, utils.ReduceWhiteSpacesToSinge(pipelineOptions), sdkEnv) } - - // Run - if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_JAVA { - executor, err = setJavaExecutableFile(lc.Paths, pipelineId, cacheService, pipelineLifeCycleCtx, executorBuilder, filepath.Join(appEnv.WorkingDir(), appEnv.PipelinesFolder())) - if err != nil { - return - } + if err != nil { + _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) + return } - logger.Infof("%s: Run() ...\n", pipelineId) - runCmd := getExecuteCmd(&validationResults, &executor, pipelineLifeCycleCtx) + + executor := executorBuilder.Build() + logger.Infof("%s: Run()/Test() ...\n", pipelineId) + runCmd := getExecuteCmd(isUnitTest, &executor, pipelineLifeCycleCtx) var runError bytes.Buffer runOutput := streaming.RunOutputWriter{Ctx: pipelineLifeCycleCtx, CacheService: cacheService, PipelineId: pipelineId} - go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, lc.Paths.AbsoluteLogFilePath, pipelineId, stopReadLogsChannel, finishReadLogsChannel) + go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, 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.Paths.AbsoluteLogFilePath) + file, err := os.Create(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()) @@ -186,7 +130,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl } // Start of the monitoring of background tasks (run step/cancellation/timeout) - ok, err = reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) + ok, err := reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) if err != nil { return } @@ -202,7 +146,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.Paths.AbsoluteLogFilePath) + errData, err := os.ReadFile(paths.AbsoluteLogFilePath) if err != nil { logger.Errorf("%s: error during read errors from log file (go sdk): %s", pipelineId, err.Error()) } @@ -215,30 +159,117 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl _ = processRunSuccess(pipelineLifeCycleCtx, pipelineId, cacheService, stopReadLogsChannel, finishReadLogsChannel) } +func compileStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.LifeCyclePaths, pipelineId uuid.UUID, sdkEnv *environment.BeamEnvs, isUnitTest bool, pipelineLifeCycleCtx context.Context, cancelChannel chan bool) *executors.Executor { + errorChannel, successChannel := createStatusChannels() + var executor = executors.Executor{} + // 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 nil + } + } else { // in case of Java, Go (not unit test), Scala - need compile step + executorBuilder := builder.Compiler(paths, sdkEnv) + executor := executorBuilder.Build() + logger.Infof("%s: Compile() ...\n", pipelineId) + compileCmd := executor.Compile(pipelineLifeCycleCtx) + var compileError bytes.Buffer + var compileOutput bytes.Buffer + runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) + + // Start of the monitoring of background tasks (compile step/cancellation/timeout) + ok, err := reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) + if err != nil { + return nil + } + if !ok { // Compile step is finished, but code couldn't be compiled (some typos for example) + err := <-errorChannel + _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, compileError.Bytes(), pipelineId, cache.CompileOutput, cacheService, "Compile", pb.Status_STATUS_COMPILE_ERROR) + return nil + } // Compile step is finished and code is compiled + if err := processCompileSuccess(pipelineLifeCycleCtx, compileOutput.Bytes(), pipelineId, cacheService); err != nil { + return nil + } + } + return &executor +} + +func prepareStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.LifeCyclePaths, pipelineId uuid.UUID, sdkEnv *environment.BeamEnvs, pipelineLifeCycleCtx context.Context, validationResults *sync.Map, cancelChannel chan bool) *executors.Executor { + errorChannel, successChannel := createStatusChannels() + executorBuilder, err := builder.Preparer(paths, sdkEnv, validationResults) + if err != nil { + _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) + return nil + } + executor := executorBuilder.Build() + logger.Infof("%s: Prepare() ...\n", pipelineId) + prepareFunc := executor.Prepare() + go prepareFunc(successChannel, errorChannel, validationResults) + + // Start of the monitoring of background tasks (prepare function/cancellation/timeout) + ok, err := reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) + if err != nil { + return nil + } + if !ok { + err := <-errorChannel + // Prepare step is finished, but code couldn't be prepared (some error during prepare step) + _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, []byte(err.Error()), pipelineId, cache.PreparationOutput, cacheService, "Prepare", pb.Status_STATUS_PREPARATION_ERROR) + return nil + } + // Prepare step is finished and code is prepared + if err := processSuccess(pipelineLifeCycleCtx, pipelineId, cacheService, "Prepare", pb.Status_STATUS_COMPILING); err != nil { + return nil + } + return &executor +} + +func validateStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.LifeCyclePaths, pipelineId uuid.UUID, sdkEnv *environment.BeamEnvs, pipelineLifeCycleCtx context.Context, validationResults *sync.Map, cancelChannel chan bool) *executors.Executor { + errorChannel, successChannel := createStatusChannels() + executorBuilder, err := builder.Validator(paths, sdkEnv) + if err != nil { + _ = processSetupError(err, pipelineId, cacheService, pipelineLifeCycleCtx) + return nil + } + executor := executorBuilder.Build() + logger.Infof("%s: Validate() ...\n", pipelineId) + validateFunc := executor.Validate() + go validateFunc(successChannel, errorChannel, validationResults) + + // Start of the monitoring of background tasks (validate function/cancellation/timeout) + ok, err := reconcileBackgroundTask(pipelineLifeCycleCtx, ctx, pipelineId, cacheService, cancelChannel, successChannel) + if err != nil { + return nil + } + if !ok { + err := <-errorChannel + // Validate step is finished, but code isn't valid + _ = processErrorWithSavingOutput(pipelineLifeCycleCtx, err, []byte(err.Error()), pipelineId, cache.ValidationOutput, cacheService, "Validate", pb.Status_STATUS_VALIDATION_ERROR) + return nil + } + + // Validate step is finished and code is valid + if err := processSuccess(pipelineLifeCycleCtx, pipelineId, cacheService, "Validate", pb.Status_STATUS_PREPARING); err != nil { + return nil + } + return &executor +} + +func createStatusChannels() (chan error, chan bool) { + errorChannel := make(chan error, 1) + successChannel := make(chan bool, 1) + return errorChannel, successChannel +} + // getExecuteCmd return cmd instance based on the code type: unit test or example code -func getExecuteCmd(valRes *sync.Map, executor *executors.Executor, ctxWithTimeout context.Context) *exec.Cmd { - isUnitTest, ok := valRes.Load(validators.UnitTestValidatorName) +func getExecuteCmd(isUnitTest bool, executor *executors.Executor, ctxWithTimeout context.Context) *exec.Cmd { runType := executors.Run - if ok && isUnitTest.(bool) { + if isUnitTest { runType = executors.Test } cmdReflect := reflect.ValueOf(executor).MethodByName(string(runType)).Call([]reflect.Value{reflect.ValueOf(ctxWithTimeout)}) return cmdReflect[0].Interface().(*exec.Cmd) } -// setJavaExecutableFile sets executable file name to runner (JAVA class name is known after compilation step) -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 - } - } - return executorBuilder. - WithExecutableFileName(className). - Build(), nil -} - // processSetupError processes errors during the setting up an executor builder func processSetupError(err error, pipelineId uuid.UUID, cacheService cache.Cache, ctxWithTimeout context.Context) error { logger.Errorf("%s: error during setup builder: %s\n", pipelineId, err.Error()) diff --git a/playground/backend/internal/code_processing/code_processing_test.go b/playground/backend/internal/code_processing/code_processing_test.go index f70065f28227..5fe8e48f781d 100644 --- a/playground/backend/internal/code_processing/code_processing_test.go +++ b/playground/backend/internal/code_processing/code_processing_test.go @@ -93,10 +93,6 @@ func teardown() { os.Clearenv() } -func fakeExecutableName(uuid.UUID, string) (string, error) { - return fileName, nil -} - func Test_Process(t *testing.T) { defer goleak.VerifyNone(t, opt) appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() @@ -532,64 +528,7 @@ func TestGetLastIndex(t *testing.T) { } } -func Test_setJavaExecutableFile(t *testing.T) { - pipelineId := uuid.New() - lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) - lc.Paths.ExecutableName = fakeExecutableName - executorBuilder := executors.NewExecutorBuilder().WithRunner().WithCommand("fake cmd").ExecutorBuilder - type args struct { - lc *fs_tool.LifeCycle - id uuid.UUID - service cache.Cache - ctx context.Context - executorBuilder *executors.ExecutorBuilder - dir string - } - tests := []struct { - name string - args args - want executors.Executor - wantErr bool - }{ - { - name: "set executable name to runner", - args: args{ - lc: lc, - id: pipelineId, - service: cacheService, - ctx: context.Background(), - executorBuilder: &executorBuilder, - dir: pipelinesFolder, - }, - want: executors.NewExecutorBuilder(). - WithExecutableFileName(fileName). - WithRunner(). - WithCommand("fake cmd"). - WithTestRunner(). - Build(), - wantErr: false, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - 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) - } - if !reflect.DeepEqual(got, tt.want) { - t.Errorf("setJavaExecutableFile() = %v, want %v", got, tt.want) - } - }) - } -} - func Test_getRunOrTestCmd(t *testing.T) { - unitTests := sync.Map{} - unitTests.Store(validators.UnitTestValidatorName, true) - - notUnitTests := sync.Map{} - notUnitTests.Store(validators.UnitTestValidatorName, false) - runEx := executors.NewExecutorBuilder(). WithRunner(). WithCommand("runCommand"). @@ -607,7 +546,7 @@ func Test_getRunOrTestCmd(t *testing.T) { wantTestExec := exec.CommandContext(context.Background(), "testCommand", "arg1", "") type args struct { - valResult *sync.Map + isUnitTest bool executor *executors.Executor ctxWithTimeout context.Context } @@ -621,7 +560,7 @@ func Test_getRunOrTestCmd(t *testing.T) { //Get cmd objects with set run executor name: "get run cmd", args: args{ - valResult: ¬UnitTests, + isUnitTest: false, executor: &runEx, ctxWithTimeout: context.Background(), }, @@ -631,7 +570,7 @@ func Test_getRunOrTestCmd(t *testing.T) { //Get cmd objects with set test executor name: "get test cmd", args: args{ - valResult: &unitTests, + isUnitTest: true, executor: &testEx, ctxWithTimeout: context.Background(), }, @@ -640,7 +579,7 @@ func Test_getRunOrTestCmd(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := getExecuteCmd(tt.args.valResult, tt.args.executor, tt.args.ctxWithTimeout); !reflect.DeepEqual(got, tt.want) { + if got := getExecuteCmd(tt.args.isUnitTest, tt.args.executor, tt.args.ctxWithTimeout); !reflect.DeepEqual(got, tt.want) { t.Errorf("getExecuteCmd() = %v, want %v", got, tt.want) } }) @@ -836,3 +775,232 @@ func Benchmark_GetLastIndex(b *testing.B) { _, _ = GetLastIndex(ctx, cacheService, pipelineId, subKey, "") } } + +func Test_validateStep(t *testing.T) { + appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + if err != nil { + panic(err) + } + sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + if err != nil { + panic(err) + } + type args struct { + ctx context.Context + cacheService cache.Cache + pipelineId uuid.UUID + sdkEnv *environment.BeamEnvs + pipelineLifeCycleCtx context.Context + validationResults *sync.Map + cancelChannel chan bool + } + tests := []struct { + name string + args args + want int + code string + }{ + { + name: "Test validation step by checking number of validators", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: sdkEnv, + pipelineLifeCycleCtx: context.Background(), + validationResults: &sync.Map{}, + cancelChannel: make(chan bool, 1), + }, + want: 3, + code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + err := lc.CreateFolders() + if err != nil { + t.Fatalf("error during prepare folders: %s", err.Error()) + } + _ = lc.CreateSourceCodeFile(tt.code) + executor := validateStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.pipelineLifeCycleCtx, tt.args.validationResults, tt.args.cancelChannel) + got := syncMapLen(tt.args.validationResults) + if executor != nil && !reflect.DeepEqual(got, tt.want) { + t.Errorf("validateStep() = %d, want %d", got, tt.want) + } + }) + } +} + +func Test_prepareStep(t *testing.T) { + appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + if err != nil { + panic(err) + } + sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + if err != nil { + panic(err) + } + validationResults := sync.Map{} + validationResults.Store(validators.UnitTestValidatorName, false) + validationResults.Store(validators.KatasValidatorName, false) + type args struct { + ctx context.Context + cacheService cache.Cache + pipelineId uuid.UUID + sdkEnv *environment.BeamEnvs + pipelineLifeCycleCtx context.Context + validationResults *sync.Map + cancelChannel chan bool + } + tests := []struct { + name string + args args + want *executors.Executor + code string + }{ + { + name: "Test preparer step working without an error", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: sdkEnv, + pipelineLifeCycleCtx: context.Background(), + validationResults: &validationResults, + cancelChannel: make(chan bool, 1), + }, + code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + err := lc.CreateFolders() + if err != nil { + t.Fatalf("error during prepare folders: %s", err.Error()) + } + _ = lc.CreateSourceCodeFile(tt.code) + if got := prepareStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.pipelineLifeCycleCtx, tt.args.validationResults, tt.args.cancelChannel); got == nil { + t.Errorf("prepareStep(): got nil instead of preparer executor") + } + }) + } +} + +func Test_compileStep(t *testing.T) { + appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + if err != nil { + panic(err) + } + sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + if err != nil { + panic(err) + } + type args struct { + ctx context.Context + cacheService cache.Cache + paths *fs_tool.LifeCyclePaths + pipelineId uuid.UUID + sdkEnv *environment.BeamEnvs + isUnitTest bool + pipelineLifeCycleCtx context.Context + cancelChannel chan bool + } + tests := []struct { + name string + args args + want *executors.Executor + code string + }{ + { + name: "Test compilation step working without an error", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: sdkEnv, + isUnitTest: false, + pipelineLifeCycleCtx: context.Background(), + cancelChannel: make(chan bool, 1), + }, + want: nil, + code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + err := lc.CreateFolders() + if err != nil { + t.Fatalf("error during prepare folders: %s", err.Error()) + } + _ = lc.CreateSourceCodeFile(tt.code) + if got := compileStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.isUnitTest, tt.args.pipelineLifeCycleCtx, tt.args.cancelChannel); got == nil { + t.Errorf("compileStep: got nil instead of compiler executor") + } + }) + } +} + +func Test_runStep(t *testing.T) { + appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + if err != nil { + panic(err) + } + sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + if err != nil { + panic(err) + } + sdkEnv.ApacheBeamSdk = pb.Sdk_SDK_PYTHON + type args struct { + ctx context.Context + cacheService cache.Cache + pipelineId uuid.UUID + isUnitTest bool + sdkEnv *environment.BeamEnvs + pipelineOptions string + pipelineLifeCycleCtx context.Context + cancelChannel chan bool + } + tests := []struct { + name string + args args + code string + }{ + { + name: "Test run step working without an error", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.UUID{}, + isUnitTest: false, + sdkEnv: sdkEnv, + pipelineOptions: "", + pipelineLifeCycleCtx: context.Background(), + cancelChannel: make(chan bool, 1), + }, + code: "if __name__ == \"__main__\":\n print(\"Hello world!\")\n", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_PYTHON, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + err := lc.CreateFolders() + if err != nil { + t.Fatalf("error during prepare folders: %s", err.Error()) + } + _ = lc.CreateSourceCodeFile(tt.code) + runStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.isUnitTest, tt.args.sdkEnv, tt.args.pipelineOptions, tt.args.pipelineLifeCycleCtx, tt.args.cancelChannel) + }) + } +} + +func syncMapLen(syncMap *sync.Map) int { + length := 0 + syncMap.Range(func(_, _ interface{}) bool { + length++ + return true + }) + return length +} diff --git a/playground/backend/internal/executors/executor.go b/playground/backend/internal/executors/executor.go index e2262b846dbf..15c95946f752 100644 --- a/playground/backend/internal/executors/executor.go +++ b/playground/backend/internal/executors/executor.go @@ -16,7 +16,7 @@ package executors import ( - "beam.apache.org/playground/backend/internal/preparators" + "beam.apache.org/playground/backend/internal/preparers" "beam.apache.org/playground/backend/internal/validators" "context" "os/exec" @@ -45,7 +45,7 @@ type Executor struct { runArgs CmdConfiguration testArgs CmdConfiguration validators []validators.Validator - preparators []preparators.Preparator + preparers []preparers.Preparer } // Validate returns the function that applies all validators of executor @@ -78,9 +78,9 @@ func (ex *Executor) Validate() func(chan bool, chan error, *sync.Map) { // Prepare returns the function that applies all preparations of executor func (ex *Executor) Prepare() func(chan bool, chan error, *sync.Map) { return func(doneCh chan bool, errCh chan error, validationResults *sync.Map) { - for _, preparator := range ex.preparators { - preparator.Args = append(preparator.Args, validationResults) - err := preparator.Prepare(preparator.Args...) + for _, preparer := range ex.preparers { + preparer.Args = append(preparer.Args, validationResults) + err := preparer.Prepare(preparer.Args...) if err != nil { errCh <- err doneCh <- false diff --git a/playground/backend/internal/executors/executor_builder.go b/playground/backend/internal/executors/executor_builder.go index 544a10472f57..7c4eb2f5784f 100644 --- a/playground/backend/internal/executors/executor_builder.go +++ b/playground/backend/internal/executors/executor_builder.go @@ -16,7 +16,7 @@ package executors import ( - "beam.apache.org/playground/backend/internal/preparators" + "beam.apache.org/playground/backend/internal/preparers" "beam.apache.org/playground/backend/internal/validators" ) @@ -42,8 +42,8 @@ type ValidatorBuilder struct { ExecutorBuilder } -//PreparatorBuilder facet of ExecutorBuilder -type PreparatorBuilder struct { +//PreparerBuilder facet of ExecutorBuilder +type PreparerBuilder struct { ExecutorBuilder } @@ -57,25 +57,6 @@ func NewExecutorBuilder() *ExecutorBuilder { return &ExecutorBuilder{} } -//WithExecutableFileName adds file name to executor -func (b *ExecutorBuilder) WithExecutableFileName(name string) *ExecutorBuilder { - b.actions = append(b.actions, func(e *Executor) { - e.runArgs.fileName = name - e.testArgs.fileName = name - }) - return b -} - -//WithWorkingDir adds dir path to executor -func (b *ExecutorBuilder) WithWorkingDir(dir string) *ExecutorBuilder { - b.actions = append(b.actions, func(e *Executor) { - e.compileArgs.workingDir = dir - e.runArgs.workingDir = dir - e.testArgs.workingDir = dir - }) - return b -} - // WithCompiler - Lives chains to type *ExecutorBuilder and returns a *CompileBuilder func (b *ExecutorBuilder) WithCompiler() *CompileBuilder { return &CompileBuilder{*b} @@ -91,9 +72,9 @@ func (b *ExecutorBuilder) WithValidator() *ValidatorBuilder { return &ValidatorBuilder{*b} } -// WithPreparator - Lives chains to type *ExecutorBuilder and returns a *PreparatorBuilder -func (b *ExecutorBuilder) WithPreparator() *PreparatorBuilder { - return &PreparatorBuilder{*b} +// WithPreparer - Lives chains to type *ExecutorBuilder and returns a *PreparerBuilder +func (b *ExecutorBuilder) WithPreparer() *PreparerBuilder { + return &PreparerBuilder{*b} } // WithTestRunner - Lives chains to type *ExecutorBuilder and returns a *UnitTestExecutorBuilder @@ -109,6 +90,14 @@ func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder { return b } +//WithWorkingDir adds dir path to executor +func (b *CompileBuilder) WithWorkingDir(dir string) *CompileBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.compileArgs.workingDir = dir + }) + return b +} + //WithArgs adds compile args to executor func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder { b.actions = append(b.actions, func(e *Executor) { @@ -125,6 +114,22 @@ func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder { return b } +//WithExecutableFileName adds file name to executor +func (b *RunBuilder) WithExecutableFileName(name string) *RunBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.runArgs.fileName = name + }) + return b +} + +//WithWorkingDir adds dir path to executor +func (b *RunBuilder) WithWorkingDir(dir string) *RunBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.runArgs.workingDir = dir + }) + return b +} + //WithCommand adds run command to executor func (b *RunBuilder) WithCommand(runCmd string) *RunBuilder { b.actions = append(b.actions, func(e *Executor) { @@ -181,6 +186,14 @@ func (b *UnitTestExecutorBuilder) WithGraphOutput() *UnitTestExecutorBuilder { return b } +//WithExecutableFileName adds file name to executor +func (b *UnitTestExecutorBuilder) WithExecutableFileName(name string) *UnitTestExecutorBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.testArgs.fileName = name + }) + return b +} + //WithSdkValidators sets validators to executor func (b *ValidatorBuilder) WithSdkValidators(validators *[]validators.Validator) *ValidatorBuilder { b.actions = append(b.actions, func(e *Executor) { @@ -189,10 +202,18 @@ func (b *ValidatorBuilder) WithSdkValidators(validators *[]validators.Validator) return b } -//WithSdkPreparators sets preparators to executor -func (b *PreparatorBuilder) WithSdkPreparators(preparators *[]preparators.Preparator) *PreparatorBuilder { +//WithSdkPreparers sets preparers to executor +func (b *PreparerBuilder) WithSdkPreparers(preparers *[]preparers.Preparer) *PreparerBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.preparers = *preparers + }) + return b +} + +//WithPipelineOptions adds pipeline options to executor +func (b *RunBuilder) WithPipelineOptions(pipelineOptions []string) *RunBuilder { b.actions = append(b.actions, func(e *Executor) { - e.preparators = *preparators + e.runArgs.pipelineOptions = pipelineOptions }) return b } @@ -205,11 +226,3 @@ func (b *ExecutorBuilder) Build() Executor { } return executor } - -//WithPipelineOptions adds pipeline options to executor -func (b *RunBuilder) WithPipelineOptions(pipelineOptions []string) *RunBuilder { - b.actions = append(b.actions, func(e *Executor) { - e.runArgs.pipelineOptions = pipelineOptions - }) - return b -} diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index b4f8759033c7..42725aa3e990 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -16,61 +16,14 @@ package executors import ( - pb "beam.apache.org/playground/backend/internal/api/v1" - "beam.apache.org/playground/backend/internal/environment" - "beam.apache.org/playground/backend/internal/preparators" + "beam.apache.org/playground/backend/internal/preparers" "beam.apache.org/playground/backend/internal/validators" "context" - "os" "os/exec" "reflect" "testing" ) -const defaultBeamJarsPath = "pathToJars" - -var ( - executorConfig = environment.NewExecutorConfig( - "javac", "java", "java", - []string{"-d", "bin", "-classpath", defaultBeamJarsPath}, - []string{"-cp", "bin:" + defaultBeamJarsPath}, - []string{"-cp", "bin:" + defaultBeamJarsPath, "JUnit"}, - ) - env = environment.NewEnvironment(environment.NetworkEnvs{}, *environment.NewBeamEnvs(pb.Sdk_SDK_JAVA, executorConfig, "", 0), environment.ApplicationEnvs{}) -) - -// BaseExecutorBuilder fills up an executor with base parameters -func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath string, validatorsFuncs *[]validators.Validator, preparatorsFuncs *[]preparators.Preparator) *ExecutorBuilder { - if validatorsFuncs == nil { - v := make([]validators.Validator, 0) - validatorsFuncs = &v - } - if preparatorsFuncs == nil { - v := make([]preparators.Preparator, 0) - preparatorsFuncs = &v - } - builder := NewExecutorBuilder(). - WithExecutableFileName("HelloWorld"). - WithWorkingDir(workingDir). - WithValidator(). - WithSdkValidators(validatorsFuncs). - WithPreparator(). - WithSdkPreparators(preparatorsFuncs). - WithCompiler(). - WithCommand(envs.ExecutorConfig.CompileCmd). - WithArgs(envs.ExecutorConfig.CompileArgs). - WithFileName(filePath). - WithRunner(). - WithCommand(envs.ExecutorConfig.RunCmd). - WithArgs(envs.ExecutorConfig.RunArgs). - WithTestRunner(). - WithCommand(envs.ExecutorConfig.TestCmd). - WithArgs(envs.ExecutorConfig.TestArgs). - WithWorkingDir(workingDir). - ExecutorBuilder - return &builder -} - func TestExecutor_Compile(t *testing.T) { type fields struct { compileArgs CmdConfiguration @@ -126,7 +79,9 @@ func TestExecutor_Run(t *testing.T) { type fields struct { compileArgs CmdConfiguration runArgs CmdConfiguration + testArgs CmdConfiguration validators []validators.Validator + preparers []preparers.Preparer } tests := []struct { name string @@ -139,14 +94,14 @@ func TestExecutor_Run(t *testing.T) { runArgs: CmdConfiguration{ fileName: "HelloWorld", workingDir: "./", - commandName: "testCommand", + commandName: "runCommand", commandArgs: []string{"-cp", "bin:/opt/apache/beam/jars/beam-sdks-java-harness.jar:" + "/opt/apache/beam/jars/beam-runners-direct.jar:/opt/apache/beam/jars/slf4j-jdk14.jar"}, pipelineOptions: []string{""}, }, }, want: &exec.Cmd{ - Path: "testCommand", + Path: "runCommand", Args: []string{"java", "-cp", "bin:/opt/apache/beam/jars/beam-sdks-java-harness.jar:" + "/opt/apache/beam/jars/beam-runners-direct.jar:/opt/apache/beam/jars/slf4j-jdk14.jar", "HelloWorld"}, Env: nil, @@ -166,7 +121,9 @@ func TestExecutor_Run(t *testing.T) { ex := &Executor{ compileArgs: tt.fields.compileArgs, runArgs: tt.fields.runArgs, + testArgs: tt.fields.testArgs, validators: tt.fields.validators, + preparers: tt.fields.preparers, } if got := ex.Run(context.Background()); !reflect.DeepEqual(got.String(), tt.want.String()) { t.Errorf("WithRunner() = %v, want %v", got, tt.want) @@ -175,64 +132,62 @@ func TestExecutor_Run(t *testing.T) { } } -func TestBaseExecutorBuilder(t *testing.T) { - validatorsFuncs := validators.GetJavaValidators("filePath") - preparatorsFuncs := preparators.GetJavaPreparators("filePath") - +func TestExecutor_RunTest(t *testing.T) { + type fields struct { + compileArgs CmdConfiguration + runArgs CmdConfiguration + testArgs CmdConfiguration + validators []validators.Validator + preparers []preparers.Preparer + } type args struct { - envs environment.BeamEnvs - workingDir string - filePath string - validatorsFuncs *[]validators.Validator - preparatorsFuncs *[]preparators.Preparator + ctx context.Context } tests := []struct { - name string - args args - want Executor + name string + fields fields + args args + want *exec.Cmd }{ { - name: "NewCmdProvider", - args: args{ - envs: env.BeamSdkEnvs, - workingDir: "./", - filePath: "filePath", - validatorsFuncs: validatorsFuncs, - preparatorsFuncs: preparatorsFuncs, - }, - want: Executor{ - compileArgs: CmdConfiguration{ - fileName: "filePath", - workingDir: "./", - commandName: "javac", - commandArgs: []string{"-d", "bin", "-classpath", defaultBeamJarsPath}, - }, - runArgs: CmdConfiguration{ - fileName: "HelloWorld", - workingDir: "./", - commandName: "java", - commandArgs: []string{"-cp", "bin:" + defaultBeamJarsPath}, - }, + name: "TestRunTest", + fields: fields{ testArgs: CmdConfiguration{ - fileName: "HelloWorld", - workingDir: "./", - commandName: "java", - commandArgs: []string{"-cp", "bin:" + defaultBeamJarsPath, "JUnit"}, + fileName: "HelloWorld", + workingDir: "./", + commandName: "testCommand", + commandArgs: []string{"-cp", "option1:option2"}, + pipelineOptions: []string{""}, }, - validators: *validatorsFuncs, - preparators: *preparatorsFuncs, + }, + args: args{context.Background()}, + want: &exec.Cmd{ + Path: "testCommand", + Args: []string{"java", "-cp", "option1:option2", "HelloWorld"}, + Env: nil, + Dir: "./", + Stdin: nil, + Stdout: nil, + Stderr: nil, + ExtraFiles: nil, + SysProcAttr: nil, + Process: nil, + ProcessState: nil, }, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := BaseExecutorBuilder(tt.args.envs, tt.args.workingDir, tt.args.filePath, tt.args.validatorsFuncs, tt.args.preparatorsFuncs).Build(); !reflect.DeepEqual(got, tt.want) { - t.Errorf("BaseExecutorBuilder() = %v, want %v", got, tt.want) + ex := &Executor{ + compileArgs: tt.fields.compileArgs, + runArgs: tt.fields.runArgs, + testArgs: tt.fields.testArgs, + validators: tt.fields.validators, + preparers: tt.fields.preparers, + } + if got := ex.RunTest(tt.args.ctx); !reflect.DeepEqual(got.String(), tt.want.String()) { + t.Errorf("RunTest() = %v, want %v", got, tt.want) } }) } - err := os.RemoveAll("configs") - if err != nil { - return - } } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 0d0b5231d0ce..6a897cb8ff69 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -40,7 +40,7 @@ type LifeCyclePaths struct { 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) + ExecutableName func(string) (string, error) } // LifeCycle is used for preparing folders and files to process code for one code processing request. diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index 58ddbf3a0aea..0379466e222e 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -286,9 +286,9 @@ func TestNewLifeCycle(t *testing.T) { want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, execFileFolder}, Paths: LifeCyclePaths{ - SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), javaSourceFileExtension), + SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), JavaSourceFileExtension), AbsoluteSourceFileFolderPath: srcFileFolder, - AbsoluteSourceFilePath: filepath.Join(srcFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), javaSourceFileExtension)), + 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)), diff --git a/playground/backend/internal/fs_tool/java_fs.go b/playground/backend/internal/fs_tool/java_fs.go index 1e815537e0ff..2fd361292eee 100644 --- a/playground/backend/internal/fs_tool/java_fs.go +++ b/playground/backend/internal/fs_tool/java_fs.go @@ -19,27 +19,24 @@ import ( "errors" "github.com/google/uuid" "os" - "path/filepath" "strings" ) const ( - javaSourceFileExtension = ".java" + JavaSourceFileExtension = ".java" javaCompiledFileExtension = ".class" ) // newJavaLifeCycle creates LifeCycle with java SDK environment. func newJavaLifeCycle(pipelineId uuid.UUID, pipelinesFolder string) *LifeCycle { - javaLifeCycle := newCompilingLifeCycle(pipelineId, pipelinesFolder, javaSourceFileExtension, javaCompiledFileExtension) + javaLifeCycle := newCompilingLifeCycle(pipelineId, pipelinesFolder, 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, pipelinesFolder string) (string, error) { - baseFileFolder := filepath.Join(pipelinesFolder, pipelineId.String()) - binFileFolder := filepath.Join(baseFileFolder, compiledFolderName) - dirEntries, err := os.ReadDir(binFileFolder) +func executableName(executableFileFolderPath string) (string, error) { + dirEntries, err := os.ReadDir(executableFileFolderPath) if err != nil { return "", err } diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index 6a414cf8db0b..d0d9aa72b369 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -50,9 +50,9 @@ func Test_newJavaLifeCycle(t *testing.T) { want: &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, Paths: LifeCyclePaths{ - SourceFileName: pipelineId.String() + javaSourceFileExtension, + SourceFileName: pipelineId.String() + JavaSourceFileExtension, AbsoluteSourceFileFolderPath: srcFileFolder, - AbsoluteSourceFilePath: filepath.Join(srcFileFolder, pipelineId.String()+javaSourceFileExtension), + AbsoluteSourceFilePath: filepath.Join(srcFileFolder, pipelineId.String()+JavaSourceFileExtension), ExecutableFileName: pipelineId.String() + javaCompiledFileExtension, AbsoluteExecutableFileFolderPath: binFileFolder, AbsoluteExecutableFilePath: filepath.Join(binFileFolder, pipelineId.String()+javaCompiledFileExtension), @@ -85,8 +85,7 @@ func Test_executableName(t *testing.T) { defer os.RemoveAll(workDir) type args struct { - pipelineId uuid.UUID - pipelinesFolder string + executableFolder string } tests := []struct { name string @@ -108,19 +107,18 @@ func Test_executableName(t *testing.T) { } }, args: args{ - pipelineId: pipelineId, - pipelinesFolder: preparedPipelinesFolder, + executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"), }, want: "temp", wantErr: false, }, { - // Test case with calling sourceFileName method with correct pipelineId and workingDir. + // Test case with calling sourceFileName method with wrong directory. // As a result, want to receive an error. name: "directory doesn't exist", prepare: func() {}, args: args{ - pipelineId: uuid.New(), + executableFolder: filepath.Join(workDir, pipelineId.String()), }, want: "", wantErr: true, @@ -129,7 +127,7 @@ func Test_executableName(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { tt.prepare() - got, err := executableName(tt.args.pipelineId, tt.args.pipelinesFolder) + got, err := executableName(tt.args.executableFolder) if (err != nil) != tt.wantErr { t.Errorf("sourceFileName() error = %v, wantErr %v", err, tt.wantErr) return diff --git a/playground/backend/internal/preparators/go_preparators.go b/playground/backend/internal/preparers/go_preparers.go similarity index 50% rename from playground/backend/internal/preparators/go_preparators.go rename to playground/backend/internal/preparers/go_preparers.go index 286dd8c2ec40..d76ac708eb4c 100644 --- a/playground/backend/internal/preparators/go_preparators.go +++ b/playground/backend/internal/preparers/go_preparers.go @@ -13,33 +13,61 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( - "beam.apache.org/playground/backend/internal/validators" "errors" "fmt" "os" "os/exec" "path/filepath" "strings" - "sync" ) const ( goName = "go" fmtArgs = "fmt" - mvCmd = "mv" sep = "." ) -// GetGoPreparators returns reparation methods that should be applied to Go code -func GetGoPreparators(filePath string) *[]Preparator { - preparatorArgs := make([]interface{}, 1) - preparatorArgs[0] = filePath - formatCodePreparator := Preparator{Prepare: formatCode, Args: preparatorArgs} - changeNamePreparator := Preparator{Prepare: changeGoTestFileName, Args: preparatorArgs} - return &[]Preparator{formatCodePreparator, changeNamePreparator} +//GoPreparersBuilder facet of PreparersBuilder +type GoPreparersBuilder struct { + PreparersBuilder +} + +//GoPreparers chains to type *PreparersBuilder and returns a *GoPreparersBuilder +func (builder *PreparersBuilder) GoPreparers() *GoPreparersBuilder { + return &GoPreparersBuilder{*builder} +} + +//WithCodeFormatter adds code formatter preparer +func (builder *GoPreparersBuilder) WithCodeFormatter() *GoPreparersBuilder { + formatCodePreparer := Preparer{ + Prepare: formatCode, + Args: []interface{}{builder.filePath}, + } + builder.AddPreparer(formatCodePreparer) + return builder +} + +//WithFileNameChanger adds preparer to change file name +func (builder *GoPreparersBuilder) WithFileNameChanger() *GoPreparersBuilder { + changeTestFileName := Preparer{ + Prepare: changeGoTestFileName, + Args: []interface{}{builder.filePath}, + } + builder.AddPreparer(changeTestFileName) + return builder +} + +// GetGoPreparers returns reparation methods that should be applied to Go code +func GetGoPreparers(builder *PreparersBuilder, isUnitTest bool) { + builder. + GoPreparers(). + WithCodeFormatter() + if isUnitTest { + builder.GoPreparers().WithFileNameChanger() + } } // formatCode formats go code @@ -56,14 +84,10 @@ func formatCode(args ...interface{}) error { func changeGoTestFileName(args ...interface{}) error { filePath := args[0].(string) - validationResults := args[1].(*sync.Map) - isUnitTest, ok := validationResults.Load(validators.UnitTestValidatorName) - if ok && isUnitTest.(bool) { - testFileName := fmt.Sprintf("%s_test.%s", strings.Split(filePath, sep)[0], goName) - err := os.Rename(filePath, testFileName) - if err != nil { - return err - } + testFileName := fmt.Sprintf("%s_test.%s", strings.Split(filePath, sep)[0], goName) + err := os.Rename(filePath, testFileName) + if err != nil { + return err } return nil } diff --git a/playground/backend/internal/preparators/go_preparators_test.go b/playground/backend/internal/preparers/go_preparers_test.go similarity index 86% rename from playground/backend/internal/preparators/go_preparators_test.go rename to playground/backend/internal/preparers/go_preparers_test.go index 0e13b39a4f26..00ae43c1e12c 100644 --- a/playground/backend/internal/preparators/go_preparators_test.go +++ b/playground/backend/internal/preparers/go_preparers_test.go @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( "beam.apache.org/playground/backend/internal/logger" @@ -88,26 +88,28 @@ func getPreparedArgs(args ...interface{}) []interface{} { return preparedArgs } -func TestGetGoPreparators(t *testing.T) { +func TestGetGoPreparers(t *testing.T) { type args struct { filePath string } tests := []struct { name string args args - want *[]Preparator + want *[]Preparer }{ { - // getting the expected preparator - name: "get expected preparator", + // getting the expected preparer + name: "get expected preparer", args: args{filePath: ""}, - want: &[]Preparator{{Prepare: formatCode, Args: nil}, {Prepare: changeGoTestFileName, Args: nil}}, + want: &[]Preparer{{Prepare: formatCode, Args: nil}, {Prepare: changeGoTestFileName, Args: nil}}, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := GetGoPreparators(tt.args.filePath); !reflect.DeepEqual(fmt.Sprint(got), fmt.Sprint(tt.want)) { - t.Errorf("GetGoPreparators() = %v, want %v", got, tt.want) + builder := NewPreparersBuilder(tt.args.filePath) + GetGoPreparers(builder, true) + if got := builder.Build().GetPreparers(); !reflect.DeepEqual(fmt.Sprint(got), fmt.Sprint(tt.want)) { + t.Errorf("GetGoPreparers() = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/preparators/java_preparators.go b/playground/backend/internal/preparers/java_preparers.go similarity index 69% rename from playground/backend/internal/preparators/java_preparators.go rename to playground/backend/internal/preparers/java_preparers.go index 07b0a6ab7015..bf604c2885a3 100644 --- a/playground/backend/internal/preparators/java_preparators.go +++ b/playground/backend/internal/preparers/java_preparers.go @@ -13,11 +13,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( "beam.apache.org/playground/backend/internal/logger" - "beam.apache.org/playground/backend/internal/validators" "bufio" "fmt" "io" @@ -26,7 +25,6 @@ import ( "path/filepath" "regexp" "strings" - "sync" ) const ( @@ -40,47 +38,73 @@ const ( publicClassNamePattern = "public class (.*?) [{|implements(.*)]" ) -// GetJavaPreparators returns preparation methods that should be applied to Java code -func GetJavaPreparators(filePath string) *[]Preparator { - removePublicClassPreparator := Preparator{ +//JavaPreparersBuilder facet of PreparersBuilder +type JavaPreparersBuilder struct { + PreparersBuilder +} + +//JavaPreparers chains to type *PreparersBuilder and returns a *JavaPreparersBuilder +func (builder *PreparersBuilder) JavaPreparers() *JavaPreparersBuilder { + return &JavaPreparersBuilder{*builder} +} + +//WithPublicClassRemover adds preparer to remove public class +func (builder *JavaPreparersBuilder) WithPublicClassRemover() *JavaPreparersBuilder { + removePublicClassPreparer := Preparer{ Prepare: removePublicClassModifier, - Args: []interface{}{filePath, classWithPublicModifierPattern, classWithoutPublicModifierPattern}, + Args: []interface{}{builder.filePath, classWithPublicModifierPattern, classWithoutPublicModifierPattern}, } - changePackagePreparator := Preparator{ - Prepare: changePackage, - Args: []interface{}{filePath, packagePattern, importStringPattern}, - } - removePackagePreparator := Preparator{ - Prepare: removePackage, - Args: []interface{}{filePath, packagePattern, newLinePattern}, + builder.AddPreparer(removePublicClassPreparer) + return builder +} + +//WithPackageChanger adds preparer to change package +func (builder *JavaPreparersBuilder) WithPackageChanger() *JavaPreparersBuilder { + changePackagePreparer := Preparer{ + Prepare: replace, + Args: []interface{}{builder.filePath, packagePattern, importStringPattern}, } - unitTestFileNameChanger := Preparator{ - Prepare: changeJavaTestFileName, - Args: []interface{}{filePath}, + builder.AddPreparer(changePackagePreparer) + return builder +} + +//WithPackageRemover adds preparer to remove package +func (builder *JavaPreparersBuilder) WithPackageRemover() *JavaPreparersBuilder { + removePackagePreparer := Preparer{ + Prepare: replace, + Args: []interface{}{builder.filePath, packagePattern, newLinePattern}, } - return &[]Preparator{removePublicClassPreparator, changePackagePreparator, removePackagePreparator, unitTestFileNameChanger} + builder.AddPreparer(removePackagePreparer) + return builder } -//changePackage changes the 'package' to 'import' and the last directory in the package value to '*' -func changePackage(args ...interface{}) error { - validationResults := args[3].(*sync.Map) - isKata, ok := validationResults.Load(validators.KatasValidatorName) - if ok && isKata.(bool) { - return nil +//WithFileNameChanger adds preparer to remove package +func (builder *JavaPreparersBuilder) WithFileNameChanger() *JavaPreparersBuilder { + unitTestFileNameChanger := Preparer{ + Prepare: changeJavaTestFileName, + Args: []interface{}{builder.filePath}, } - err := replace(args...) - return err + builder.AddPreparer(unitTestFileNameChanger) + return builder } -//removePackage remove the package line in the katas. -func removePackage(args ...interface{}) error { - validationResults := args[3].(*sync.Map) - isKata, ok := validationResults.Load(validators.KatasValidatorName) - if ok && isKata.(bool) { - err := replace(args...) - return err +// GetJavaPreparers returns preparation methods that should be applied to Java code +func GetJavaPreparers(builder *PreparersBuilder, isUnitTest bool, isKata bool) { + if !isUnitTest && !isKata { + builder.JavaPreparers(). + WithPublicClassRemover(). + WithPackageChanger() + } + if isUnitTest { + builder.JavaPreparers(). + WithPackageChanger(). + WithFileNameChanger() + } + if isKata { + builder.JavaPreparers(). + WithPublicClassRemover(). + WithPackageRemover() } - return nil } // replace processes file by filePath and replaces all patterns to newPattern @@ -119,11 +143,6 @@ func replace(args ...interface{}) error { } func removePublicClassModifier(args ...interface{}) error { - validationResults := args[3].(*sync.Map) - isUnitTest, ok := validationResults.Load(validators.UnitTestValidatorName) - if ok && isUnitTest.(bool) { - return nil - } err := replace(args...) return err } @@ -185,17 +204,13 @@ func addNewLine(newLine bool, file *os.File) error { func changeJavaTestFileName(args ...interface{}) error { filePath := args[0].(string) - validationResults := args[1].(*sync.Map) - isUnitTest, ok := validationResults.Load(validators.UnitTestValidatorName) - if ok && isUnitTest.(bool) { - className, err := getPublicClassName(filePath) - if err != nil { - return err - } - err = renameJavaFile(filePath, className) - if err != nil { - return err - } + className, err := getPublicClassName(filePath) + if err != nil { + return err + } + err = renameJavaFile(filePath, className) + if err != nil { + return err } return nil } @@ -210,7 +225,7 @@ func renameJavaFile(filePath string, className string) error { func getPublicClassName(filePath string) (string, error) { code, err := ioutil.ReadFile(filePath) if err != nil { - logger.Errorf("Preparator: Error during open file: %s, err: %s\n", filePath, err.Error()) + logger.Errorf("Preparer: Error during open file: %s, err: %s\n", filePath, err.Error()) return "", err } re := regexp.MustCompile(publicClassNamePattern) diff --git a/playground/backend/internal/preparators/java_preparators_test.go b/playground/backend/internal/preparers/java_preparers_test.go similarity index 87% rename from playground/backend/internal/preparators/java_preparators_test.go rename to playground/backend/internal/preparers/java_preparers_test.go index 145d703a4c26..23720b444495 100644 --- a/playground/backend/internal/preparators/java_preparators_test.go +++ b/playground/backend/internal/preparers/java_preparers_test.go @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( pb "beam.apache.org/playground/backend/internal/api/v1" @@ -88,9 +88,11 @@ func Test_replace(t *testing.T) { } } -func TestGetJavaPreparators(t *testing.T) { +func TestGetJavaPreparers(t *testing.T) { type args struct { - filePath string + filePath string + isUnitTest bool + isKata bool } tests := []struct { name string @@ -98,15 +100,27 @@ func TestGetJavaPreparators(t *testing.T) { want int }{ { - name: "all success", - args: args{"MOCK_FILEPATH"}, - want: 4, + name: "Test number of preparers for code", + args: args{"MOCK_FILEPATH", false, false}, + want: 2, + }, + { + name: "Test number of preparers for unit test", + args: args{"MOCK_FILEPATH", true, false}, + want: 2, + }, + { + name: "Test number of preparers for kata", + args: args{"MOCK_FILEPATH", false, true}, + want: 2, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := GetJavaPreparators(tt.args.filePath); len(*got) != tt.want { - t.Errorf("GetJavaPreparation() returns %v Preparators, want %v", len(*got), tt.want) + builder := NewPreparersBuilder(tt.args.filePath) + GetJavaPreparers(builder, tt.args.isUnitTest, tt.args.isKata) + if got := builder.Build().GetPreparers(); len(*got) != tt.want { + t.Errorf("GetJavaPreparation() returns %v Preparers, want %v", len(*got), tt.want) } }) } diff --git a/playground/backend/internal/preparators/preparator.go b/playground/backend/internal/preparers/preparer.go similarity index 50% rename from playground/backend/internal/preparators/preparator.go rename to playground/backend/internal/preparers/preparer.go index 7bc590be3fbb..7a758333577e 100644 --- a/playground/backend/internal/preparators/preparator.go +++ b/playground/backend/internal/preparers/preparer.go @@ -13,10 +13,38 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers -// Preparator is used to make preparations with file with code. -type Preparator struct { +// Preparer is used to make preparations with file with code. +type Preparer struct { Prepare func(args ...interface{}) error Args []interface{} } + +type Preparers struct { + functions *[]Preparer +} + +func (preparers *Preparers) GetPreparers() *[]Preparer { + return preparers.functions +} + +//PreparersBuilder struct +type PreparersBuilder struct { + preparers *Preparers + filePath string +} + +//NewPreparersBuilder constructor for PreparersBuilder +func NewPreparersBuilder(filePath string) *PreparersBuilder { + return &PreparersBuilder{preparers: &Preparers{functions: &[]Preparer{}}, filePath: filePath} +} + +//Build builds preparers from PreparersBuilder +func (builder *PreparersBuilder) Build() *Preparers { + return builder.preparers +} + +func (builder *PreparersBuilder) AddPreparer(newPreparer Preparer) { + *builder.preparers.functions = append(*builder.preparers.functions, newPreparer) +} diff --git a/playground/backend/internal/preparators/python_preparators.go b/playground/backend/internal/preparers/python_preparers.go similarity index 78% rename from playground/backend/internal/preparators/python_preparators.go rename to playground/backend/internal/preparers/python_preparers.go index 3a95bd3b9d70..d86c3dd82bd5 100644 --- a/playground/backend/internal/preparators/python_preparators.go +++ b/playground/backend/internal/preparers/python_preparers.go @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( "beam.apache.org/playground/backend/internal/logger" @@ -26,13 +26,31 @@ const ( addLogHandlerCode = "import logging\nlogging.basicConfig(\n level=logging.DEBUG,\n format=\"%(asctime)s [%(levelname)s] %(message)s\",\n handlers=[\n logging.FileHandler(\"logs.log\"),\n ]\n)\n" ) -// GetPythonPreparators returns preparation methods that should be applied to Python code -func GetPythonPreparators(filePath string) *[]Preparator { - addLogHandler := Preparator{ +// GetPythonPreparers returns preparation methods that should be applied to Python code +func GetPythonPreparers(builder *PreparersBuilder) { + builder. + PythonPreparers(). + WithLogHandler() +} + +//PythonPreparersBuilder facet of PreparersBuilder +type PythonPreparersBuilder struct { + PreparersBuilder +} + +//PythonPreparers chains to type *PreparersBuilder and returns a *GoPreparersBuilder +func (builder *PreparersBuilder) PythonPreparers() *PythonPreparersBuilder { + return &PythonPreparersBuilder{*builder} +} + +//WithLogHandler adds code for logging +func (builder *PythonPreparersBuilder) WithLogHandler() *PythonPreparersBuilder { + addLogHandler := Preparer{ Prepare: addCodeToFile, - Args: []interface{}{filePath, addLogHandlerCode}, + Args: []interface{}{builder.filePath, addLogHandlerCode}, } - return &[]Preparator{addLogHandler} + builder.AddPreparer(addLogHandler) + return builder } // addCodeToFile processes file by filePath and adds additional code diff --git a/playground/backend/internal/preparators/python_preparators_test.go b/playground/backend/internal/preparers/python_preparers_test.go similarity index 86% rename from playground/backend/internal/preparators/python_preparators_test.go rename to playground/backend/internal/preparers/python_preparers_test.go index 71519251bd9d..523b75f52912 100644 --- a/playground/backend/internal/preparators/python_preparators_test.go +++ b/playground/backend/internal/preparers/python_preparers_test.go @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package preparators +package preparers import ( "os" @@ -21,7 +21,7 @@ import ( "testing" ) -func TestGetPythonPreparators(t *testing.T) { +func TestGetPythonPreparers(t *testing.T) { type args struct { filePath string } @@ -31,17 +31,19 @@ func TestGetPythonPreparators(t *testing.T) { want int }{ { - // Test case with calling GetPythonPreparators method. - // As a result, want to receive slice of preparators with len = 1 - name: "get python preparators", + // Test case with calling GetPythonPreparers method. + // As a result, want to receive slice of preparers with len = 1 + name: "get python preparers", args: args{"MOCK_FILEPATH"}, want: 1, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := GetPythonPreparators(tt.args.filePath); len(*got) != tt.want { - t.Errorf("GetPythonPreparators() returns %v Preparators, want %v", len(*got), tt.want) + builder := NewPreparersBuilder(tt.args.filePath) + GetPythonPreparers(builder) + if got := builder.Build().GetPreparers(); len(*got) != tt.want { + t.Errorf("GetPythonPreparers() returns %v Preparers, want %v", len(*got), tt.want) } }) } diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index 19847de0fd02..5209ff0830cc 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -24,6 +24,7 @@ import ( "fmt" "path/filepath" "strings" + "sync" ) const ( @@ -31,73 +32,139 @@ const ( javaLogConfigFilePlaceholder = "{logConfigFile}" ) -// SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(paths fs_tool.LifeCyclePaths, pipelineOptions string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +// Validator return executor with set args for validator +func Validator(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk - - if sdk == pb.Sdk_SDK_JAVA { - pipelineOptions = utils.ReplaceSpacesWithEquals(pipelineOptions) - } - val, err := utils.GetValidators(sdk, paths.AbsoluteSourceFilePath) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, paths.AbsoluteSourceFilePath) + builder := executors.NewExecutorBuilder(). + WithValidator(). + WithSdkValidators(val). + ExecutorBuilder + return &builder, err +} + +// Preparer return executor with set args for preparer +func Preparer(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs, valResults *sync.Map) (*executors.ExecutorBuilder, error) { + sdk := sdkEnv.ApacheBeamSdk + prep, err := utils.GetPreparers(sdk, paths.AbsoluteSourceFilePath, valResults) if err != nil { return nil, err } + builder := executors.NewExecutorBuilder(). + WithPreparer(). + WithSdkPreparers(prep). + ExecutorBuilder + return &builder, err +} + +// Compiler return executor with set args for compiler +func Compiler(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs) *executors.ExecutorBuilder { + sdk := sdkEnv.ApacheBeamSdk executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(paths.AbsoluteExecutableFilePath). - WithWorkingDir(paths.AbsoluteBaseFolderPath). - WithValidator(). - WithSdkValidators(val). - WithPreparator(). - WithSdkPreparators(prep). WithCompiler(). WithCommand(executorConfig.CompileCmd). + WithWorkingDir(paths.AbsoluteBaseFolderPath). WithArgs(executorConfig.CompileArgs). WithFileName(paths.AbsoluteSourceFilePath). + ExecutorBuilder + + switch sdk { + case pb.Sdk_SDK_JAVA: + builder. + WithCompiler(). + WithFileName(GetFirstFileFromFolder(paths.AbsoluteSourceFileFolderPath)) + } + return &builder +} + +// Runner return executor with set args for runner +func Runner(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) + } + executorConfig := sdkEnv.ExecutorConfig + builder := executors.NewExecutorBuilder(). WithRunner(). + WithWorkingDir(paths.AbsoluteBaseFolderPath). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). WithPipelineOptions(strings.Split(pipelineOptions, " ")). - WithTestRunner(). - WithCommand(executorConfig.TestCmd). - WithArgs(executorConfig.TestArgs). - WithWorkingDir(paths.AbsoluteSourceFileFolderPath). ExecutorBuilder switch sdk { - case pb.Sdk_SDK_JAVA: // Executable name for java class will be known after compilation - args := make([]string, 0) - for _, arg := range executorConfig.RunArgs { - if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(paths.AbsoluteBaseFolderPath, javaLogConfigFileName) - arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) - } - args = append(args, arg) + case pb.Sdk_SDK_JAVA: // Executable name for java class is known after compilation + args := replaceLogPlaceholder(paths, executorConfig) + className, err := paths.ExecutableName(paths.AbsoluteExecutableFileFolderPath) + if err != nil { + return nil, fmt.Errorf("no executable file name found for JAVA pipeline at %s", paths.AbsoluteExecutableFileFolderPath) } - builder = builder.WithRunner().WithArgs(args).ExecutorBuilder - builder = builder.WithTestRunner().WithWorkingDir(paths.AbsoluteBaseFolderPath).ExecutorBuilder //change directory for unit test + builder = builder. + WithRunner(). + WithArgs(args). + WithExecutableFileName(className). + ExecutorBuilder case pb.Sdk_SDK_GO: //go run command is executable file itself builder = builder. - WithExecutableFileName(""). WithRunner(). - WithCommand(paths.AbsoluteExecutableFilePath).ExecutorBuilder + WithExecutableFileName(""). + WithCommand(paths.AbsoluteExecutableFilePath). + ExecutorBuilder case pb.Sdk_SDK_PYTHON: - builder = *builder.WithExecutableFileName(paths.AbsoluteExecutableFilePath) - case pb.Sdk_SDK_SCIO: - return nil, fmt.Errorf("SCIO is not supported yet") - default: - return nil, fmt.Errorf("incorrect sdk: %s", sdkEnv.ApacheBeamSdk) + builder = builder. + WithRunner(). + WithExecutableFileName(paths.AbsoluteExecutableFilePath). + ExecutorBuilder } return &builder, nil } -// GetFileNameFromFolder return a name of the first file in a specified folder -func GetFileNameFromFolder(folderAbsolutePath, extension string) string { - files, _ := filepath.Glob(fmt.Sprintf("%s/*%s", folderAbsolutePath, extension)) +// TestRunner return executor with set args for runner +func TestRunner(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { + sdk := sdkEnv.ApacheBeamSdk + executorConfig := sdkEnv.ExecutorConfig + builder := executors.NewExecutorBuilder(). + WithTestRunner(). + WithExecutableFileName(paths.AbsoluteExecutableFilePath). + WithCommand(executorConfig.TestCmd). + WithArgs(executorConfig.TestArgs). + WithWorkingDir(paths.AbsoluteSourceFileFolderPath). + ExecutorBuilder + + switch sdk { + case pb.Sdk_SDK_JAVA: // Executable name for java class is known after compilation + className, err := paths.ExecutableName(paths.AbsoluteExecutableFileFolderPath) + if err != nil { + return nil, fmt.Errorf("no executable file name found for JAVA pipeline at %s", paths.AbsoluteExecutableFileFolderPath) + } + builder = builder.WithTestRunner(). + WithExecutableFileName(className). + WithWorkingDir(paths.AbsoluteBaseFolderPath). + ExecutorBuilder //change directory for unit test + } + return &builder, nil +} + +// replaceLogPlaceholder replaces placeholder for log for JAVA SDK +func replaceLogPlaceholder(paths *fs_tool.LifeCyclePaths, executorConfig *environment.ExecutorConfig) []string { + args := make([]string, 0) + for _, arg := range executorConfig.RunArgs { + if strings.Contains(arg, javaLogConfigFilePlaceholder) { + logConfigFilePath := filepath.Join(paths.AbsoluteBaseFolderPath, javaLogConfigFileName) + arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) + } + args = append(args, arg) + } + return args +} + +// GetFirstFileFromFolder return a name of the first file in a specified folder +func GetFirstFileFromFolder(folderAbsolutePath string) string { + files, _ := filepath.Glob(fmt.Sprintf("%s/*%s", folderAbsolutePath, fs_tool.JavaSourceFileExtension)) 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 f862c7c304a4..89c577c76675 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -21,69 +21,118 @@ import ( "beam.apache.org/playground/backend/internal/executors" "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/utils" + "beam.apache.org/playground/backend/internal/validators" "fmt" "github.com/google/uuid" + "reflect" "strings" + "sync" "testing" ) -func TestSetupExecutor(t *testing.T) { +var paths *fs_tool.LifeCyclePaths +var sdkEnv *environment.BeamEnvs + +func TestMain(m *testing.M) { + setup() + m.Run() +} + +func setup() { pipelineId := uuid.New() - sdk := pb.Sdk_SDK_JAVA - lc, err := fs_tool.NewLifeCycle(sdk, pipelineId, "") - if err != nil { - t.Error(err) - } - pipelineOptions := "" + sdk := pb.Sdk_SDK_PYTHON + lc, _ := fs_tool.NewLifeCycle(sdk, pipelineId, "") + paths = &lc.Paths executorConfig := &environment.ExecutorConfig{ CompileCmd: "MOCK_COMPILE_CMD", - RunCmd: "MOCK_RUN_CMD", - TestCmd: "MOCK_TEST_CMD", CompileArgs: []string{"MOCK_COMPILE_ARG"}, - RunArgs: []string{"MOCK_RUN_ARG"}, - TestArgs: []string{"MOCK_TEST_ARG"}, } + sdkEnv = environment.NewBeamEnvs(sdk, executorConfig, "", 0) +} + +func TestValidator(t *testing.T) { + vals, err := utils.GetValidators(sdkEnv.ApacheBeamSdk, paths.AbsoluteSourceFilePath) if err != nil { panic(err) } + wantExecutor := executors.NewExecutorBuilder(). + WithValidator(). + WithSdkValidators(vals) - srcFilePath := lc.Paths.AbsoluteSourceFilePath + wrongSdkEnv := environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, sdkEnv.ExecutorConfig, "", 0) - sdkEnv := environment.NewBeamEnvs(sdk, executorConfig, "", 0) - val, err := utils.GetValidators(sdk, srcFilePath) - if err != nil { - panic(err) + type args struct { + paths *fs_tool.LifeCyclePaths + sdkEnv *environment.BeamEnvs + } + tests := []struct { + name string + args args + want *executors.ExecutorBuilder + wantErr bool + }{ + { + // Test case with calling Setup with correct data. + // As a result, want to receive an expected validator builder. + name: "Test correct validator builder", + args: args{ + paths: paths, + sdkEnv: sdkEnv, + }, + want: &wantExecutor.ExecutorBuilder, + wantErr: false, + }, + { + // Test case with calling Setup with incorrect SDK. + // As a result, want to receive an error. + name: "incorrect sdk", + args: args{ + paths: paths, + sdkEnv: wrongSdkEnv, + }, + want: nil, + wantErr: true, + }, } - prep, err := utils.GetPreparators(sdk, srcFilePath) + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := Validator(tt.args.paths, tt.args.sdkEnv) + if (err != nil) != tt.wantErr { + t.Errorf("Validator() error = %v, wantErr %v", err, tt.wantErr) + return + } + if err != nil && !reflect.DeepEqual(got, tt.want) { + t.Errorf("Validator() got = %v, want %v", got, tt.want) + return + } + if err == nil && !reflect.DeepEqual(fmt.Sprint(got.Build()), fmt.Sprint(tt.want.Build())) { + t.Errorf("Validator() got = %v\n, want %v", got.Build(), tt.want.Build()) + } + }) + } +} + +func TestPreparer(t *testing.T) { + validationResults := sync.Map{} + validationResults.Store(validators.UnitTestValidatorName, false) + validationResults.Store(validators.KatasValidatorName, false) + + prep, err := utils.GetPreparers(sdkEnv.ApacheBeamSdk, paths.AbsoluteSourceFilePath, &validationResults) if err != nil { panic(err) } - + pipelineOptions := "" wantExecutor := executors.NewExecutorBuilder(). - WithExecutableFileName(lc.Paths.AbsoluteExecutableFilePath). - WithWorkingDir(lc.Paths.AbsoluteBaseFolderPath). - WithValidator(). - WithSdkValidators(val). - WithPreparator(). - WithSdkPreparators(prep). - WithCompiler(). - WithCommand(executorConfig.CompileCmd). - WithArgs(executorConfig.CompileArgs). - WithFileName(srcFilePath). - WithRunner(). - WithCommand(executorConfig.RunCmd). - WithArgs(executorConfig.RunArgs). - WithPipelineOptions(strings.Split(pipelineOptions, " ")). - WithTestRunner(). - WithCommand(executorConfig.TestCmd). - WithArgs(executorConfig.TestArgs). - WithWorkingDir(lc.Paths.AbsoluteBaseFolderPath). - ExecutorBuilder + WithPreparer(). + WithSdkPreparers(prep) + + wrongSdkEnv := environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, sdkEnv.ExecutorConfig, "", 0) type args struct { - dto fs_tool.LifeCyclePaths + paths fs_tool.LifeCyclePaths pipelineOptions string sdkEnv *environment.BeamEnvs + valResults *sync.Map } tests := []struct { name string @@ -95,28 +144,176 @@ 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.Paths, pipelineOptions, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "", 0)}, + args: args{*paths, pipelineOptions, wrongSdkEnv, &validationResults}, want: nil, wantErr: true, }, { // Test case with calling Setup with correct SDK. - // As a result, want to receive an expected builder. + // As a result, want to receive an expected preparer builder. name: "correct sdk", - args: args{lc.Paths, pipelineOptions, sdkEnv}, - want: &wantExecutor, + args: args{*paths, pipelineOptions, sdkEnv, &validationResults}, + want: &wantExecutor.ExecutorBuilder, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := SetupExecutorBuilder(tt.args.dto, tt.args.pipelineOptions, tt.args.sdkEnv) + got, err := Preparer(&tt.args.paths, tt.args.sdkEnv, tt.args.valResults) if (err != nil) != tt.wantErr { - t.Errorf("SetupExecutorBuilder() error = %v, wantErr %v", err, tt.wantErr) + t.Errorf("Preparer() error = %v, wantErr %v", err, tt.wantErr) return } - if err == nil && fmt.Sprint(got.Build()) != fmt.Sprint(tt.want.Build()) { - t.Errorf("SetupExecutorBuilder() got = %v\n, want %v", got.Build(), tt.want.Build()) + if err != nil && !reflect.DeepEqual(got, tt.want) { + t.Errorf("Preparer() got = %v, want %v", got, tt.want) + return + } + if err == nil && !reflect.DeepEqual(fmt.Sprint(got.Build()), fmt.Sprint(tt.want.Build())) { + t.Errorf("Preparer() got = %v, want %v", got.Build(), tt.want.Build()) + } + }) + } +} + +func TestCompiler(t *testing.T) { + wantExecutor := executors.NewExecutorBuilder(). + WithCompiler(). + WithCommand(sdkEnv.ExecutorConfig.CompileCmd). + WithWorkingDir(paths.AbsoluteBaseFolderPath). + WithArgs(sdkEnv.ExecutorConfig.CompileArgs). + WithFileName(paths.AbsoluteSourceFilePath) + + type args struct { + paths *fs_tool.LifeCyclePaths + sdkEnv *environment.BeamEnvs + } + tests := []struct { + name string + args args + want *executors.ExecutorBuilder + }{ + { + // Test case with calling Setup with correct data. + // As a result, want to receive an expected compiler builder. + name: "Test correct compiler builder", + args: args{ + paths: paths, + sdkEnv: sdkEnv, + }, + want: &wantExecutor.ExecutorBuilder, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := Compiler(tt.args.paths, tt.args.sdkEnv) + if !reflect.DeepEqual(fmt.Sprint(got.Build()), fmt.Sprint(tt.want.Build())) { + t.Errorf("Compiler() = %v, want %v", got.Build(), tt.want.Build()) + } + }) + } +} + +func TestRunnerBuilder(t *testing.T) { + wantExecutor := executors.NewExecutorBuilder(). + WithRunner(). + WithExecutableFileName(paths.AbsoluteExecutableFilePath). + WithWorkingDir(paths.AbsoluteBaseFolderPath). + WithCommand(sdkEnv.ExecutorConfig.RunCmd). + WithArgs(sdkEnv.ExecutorConfig.RunArgs). + WithPipelineOptions(strings.Split("", " ")) + + type args struct { + paths *fs_tool.LifeCyclePaths + pipelineOptions string + sdkEnv *environment.BeamEnvs + } + tests := []struct { + name string + args args + want *executors.ExecutorBuilder + }{ + { + // Test case with calling Setup with correct data. + // As a result, want to receive an expected run builder. + name: "Test correct run builder", + args: args{ + paths: paths, + sdkEnv: sdkEnv, + }, + want: &wantExecutor.ExecutorBuilder, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, _ := Runner(tt.args.paths, tt.args.pipelineOptions, tt.args.sdkEnv) + if !reflect.DeepEqual(fmt.Sprint(got.Build()), fmt.Sprint(tt.want.Build())) { + t.Errorf("Runner() got = %v, want %v", got.Build(), tt.want.Build()) + } + }) + } +} + +func TestTestRunner(t *testing.T) { + wantExecutor := executors.NewExecutorBuilder(). + WithTestRunner(). + WithExecutableFileName(paths.AbsoluteExecutableFilePath). + WithCommand(sdkEnv.ExecutorConfig.TestCmd). + WithArgs(sdkEnv.ExecutorConfig.TestArgs). + WithWorkingDir(paths.AbsoluteSourceFileFolderPath) + + type args struct { + paths *fs_tool.LifeCyclePaths + sdkEnv *environment.BeamEnvs + } + tests := []struct { + name string + args args + want *executors.ExecutorBuilder + }{ + { + // Test case with calling Setup with correct data. + // As a result, want to receive an expected test builder. + name: "Test correct test builder", + args: args{ + paths: paths, + sdkEnv: sdkEnv, + }, + want: &wantExecutor.ExecutorBuilder, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, _ := TestRunner(tt.args.paths, tt.args.sdkEnv) + if !reflect.DeepEqual(fmt.Sprint(got.Build()), fmt.Sprint(tt.want.Build())) { + t.Errorf("TestRunner() got = %v, want %v", got.Build(), tt.want.Build()) + } + }) + } +} + +func Test_replaceLogPlaceholder(t *testing.T) { + type args struct { + paths *fs_tool.LifeCyclePaths + executorConfig *environment.ExecutorConfig + } + tests := []struct { + name string + args args + want []string + }{ + { + name: "Test to check the replacement of log work with no error", + args: args{ + paths: paths, + executorConfig: sdkEnv.ExecutorConfig, + }, + want: []string{}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := replaceLogPlaceholder(tt.args.paths, tt.args.executorConfig); !reflect.DeepEqual(got, tt.want) { + t.Errorf("replaceLogPlaceholder() = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/utils/preparators_utils.go b/playground/backend/internal/utils/preparators_utils.go index b8b3d419a532..876225ee1d79 100644 --- a/playground/backend/internal/utils/preparators_utils.go +++ b/playground/backend/internal/utils/preparators_utils.go @@ -17,25 +17,35 @@ package utils import ( pb "beam.apache.org/playground/backend/internal/api/v1" - "beam.apache.org/playground/backend/internal/preparators" + "beam.apache.org/playground/backend/internal/preparers" + "beam.apache.org/playground/backend/internal/validators" "fmt" "regexp" + "sync" ) -// GetPreparators returns slice of preparators.Preparator according to sdk -func GetPreparators(sdk pb.Sdk, filepath string) (*[]preparators.Preparator, error) { - var prep *[]preparators.Preparator +// GetPreparers returns slice of preparers.Preparer according to sdk +func GetPreparers(sdk pb.Sdk, filepath string, valResults *sync.Map) (*[]preparers.Preparer, error) { + isUnitTest, ok := valResults.Load(validators.UnitTestValidatorName) + if !ok { + return nil, fmt.Errorf("GetPreparers:: No information about unit test validation result") + } + builder := preparers.NewPreparersBuilder(filepath) switch sdk { case pb.Sdk_SDK_JAVA: - prep = preparators.GetJavaPreparators(filepath) + isKata, ok := valResults.Load(validators.KatasValidatorName) + if !ok { + return nil, fmt.Errorf("GetPreparers:: No information about katas validation result") + } + preparers.GetJavaPreparers(builder, isUnitTest.(bool), isKata.(bool)) case pb.Sdk_SDK_GO: - prep = preparators.GetGoPreparators(filepath) + preparers.GetGoPreparers(builder, isUnitTest.(bool)) case pb.Sdk_SDK_PYTHON: - prep = preparators.GetPythonPreparators(filepath) + preparers.GetPythonPreparers(builder) default: return nil, fmt.Errorf("incorrect sdk: %s", sdk) } - return prep, nil + return builder.Build().GetPreparers(), nil } // ReplaceSpacesWithEquals prepares pipelineOptions by replacing spaces between option and them value to equals.