From 9d96a2e980c5d9383f8a73f067ecdb38497ebc74 Mon Sep 17 00:00:00 2001 From: Ilya Date: Tue, 16 Nov 2021 15:07:49 +0300 Subject: [PATCH 01/12] Revert "[Playground][BEAM-12941][Bugfix] Fix workflows for playground applications (#83)" (#88) This reverts commit b73f5f70ac4184b56a0a03922731c5f2f69b9566. --- playground/backend/containers/java/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index cc95e19cd2f5..38303052e40b 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG BASE_IMAGE=apache/beam_java8_sdk:latest +ARG BASE_IMAGE FROM golang:1.17-buster AS build # Setup Go Environment From 9cc287ee5fc5a541676889bebdc59fc6218d30cd Mon Sep 17 00:00:00 2001 From: Ilya Date: Tue, 16 Nov 2021 15:07:49 +0300 Subject: [PATCH 02/12] Revert "[Playground][BEAM-12941][Bugfix] Fix workflows for playground applications (#83)" (#88) This reverts commit b73f5f70ac4184b56a0a03922731c5f2f69b9566. --- playground/backend/containers/java/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index 06571a77d068..cfd2f016e109 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG BASE_IMAGE=apache/beam_java8_sdk:latest +ARG BASE_IMAGE FROM golang:1.17-buster AS build # Setup Go Environment From 61fb10d669a9ae2a84d3a3748880a5738a658325 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Thu, 25 Nov 2021 22:45:36 +0300 Subject: [PATCH 03/12] Add go unit tests support --- playground/backend/configs/SDK_GO.json | 6 +- playground/backend/containers/go/Dockerfile | 2 +- playground/backend/containers/java/Dockerfile | 2 +- .../code_processing/code_processing.go | 2 +- .../internal/executors/executor_builder.go | 8 ++ .../internal/executors/executor_test.go | 10 +-- playground/backend/internal/fs_tool/fs.go | 6 ++ .../setup_tools/builder/setup_builder.go | 18 +++-- .../setup_tools/builder/setup_builder_test.go | 13 ++-- .../internal/utils/validators_utils.go | 2 +- .../internal/validators/go_validators.go | 31 +++++++- .../internal/validators/go_validators_test.go | 76 +++++++++++++++++++ .../internal/validators/java_validators.go | 4 +- .../validators/java_validators_test.go | 52 +++---------- .../internal/validators/validator_test.go | 56 ++++++++++++++ 15 files changed, 218 insertions(+), 70 deletions(-) create mode 100644 playground/backend/internal/validators/go_validators_test.go create mode 100644 playground/backend/internal/validators/validator_test.go diff --git a/playground/backend/configs/SDK_GO.json b/playground/backend/configs/SDK_GO.json index 68a5a27f93ba..0c4775be5e3b 100644 --- a/playground/backend/configs/SDK_GO.json +++ b/playground/backend/configs/SDK_GO.json @@ -1,11 +1,15 @@ { "compile_cmd": "go", "run_cmd": "", + "test_cmd": "go", "compile_args": [ "build", "-o", "bin" ], "run_args": [ + ], + "test_args": [ + "test" ] -} \ No newline at end of file +} diff --git a/playground/backend/containers/go/Dockerfile b/playground/backend/containers/go/Dockerfile index 493b8b20f8e8..bb130f6da776 100644 --- a/playground/backend/containers/go/Dockerfile +++ b/playground/backend/containers/go/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ############################################################################### #Dokerfile to set up the Beam Go SDK -ARG BASE_IMAGE golang:1.16-buster +ARG BASE_IMAGE=golang:1.16-buster FROM $BASE_IMAGE # Setup Go Environment diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index cfd2f016e109..06571a77d068 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG BASE_IMAGE +ARG BASE_IMAGE=apache/beam_java8_sdk:latest FROM golang:1.17-buster AS build # Setup Go Environment diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 917eae66dd31..48f088db985f 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(ctxWithTimeout, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv) + executorBuilder, err := builder.SetupExecutorBuilder(lc, sdkEnv) if err != nil { _ = processSetupError(err, pipelineId, cacheService, ctxWithTimeout) return diff --git a/playground/backend/internal/executors/executor_builder.go b/playground/backend/internal/executors/executor_builder.go index 44fabff48ee3..aee96ee3de18 100644 --- a/playground/backend/internal/executors/executor_builder.go +++ b/playground/backend/internal/executors/executor_builder.go @@ -165,6 +165,14 @@ func (b *UnitTestExecutorBuilder) WithArgs(testArgs []string) *UnitTestExecutorB return b } +// WithWorkingDir adds dir path to executor +func (b *UnitTestExecutorBuilder) WithWorkingDir(dir string) *UnitTestExecutorBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.testArgs.workingDir = dir + }) + return b +} + //WithGraphOutput adds the need of graph output to executor func (b *UnitTestExecutorBuilder) WithGraphOutput() *UnitTestExecutorBuilder { b.actions = append(b.actions, func(e *Executor) { diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index b1ac6c94e020..5210667cb230 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -51,7 +51,10 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath } builder := NewExecutorBuilder(). WithExecutableFileName("HelloWorld"). - WithWorkingDir(workingDir). + WithWorkingDir(workingDir).WithValidator(). + WithSdkValidators(validatorsFuncs). + WithPreparator(). + WithSdkPreparators(preparatorsFuncs). WithCompiler(). WithCommand(envs.ExecutorConfig.CompileCmd). WithArgs(envs.ExecutorConfig.CompileArgs). @@ -62,10 +65,7 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath WithTestRunner(). WithCommand(envs.ExecutorConfig.TestCmd). WithArgs(envs.ExecutorConfig.TestArgs). - WithValidator(). - WithSdkValidators(validatorsFuncs). - WithPreparator(). - WithSdkPreparators(preparatorsFuncs). + WithWorkingDir(workingDir). ExecutorBuilder return &builder } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 07c78ddfa2ce..c1cb75a97598 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -166,3 +166,9 @@ func (l *LifeCycle) GetAbsoluteLogFilePath() string { 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/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index fb65aa312a43..e0fa12486838 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -19,6 +19,7 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/environment" "beam.apache.org/playground/backend/internal/executors" + "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/utils" "fmt" "path/filepath" @@ -31,21 +32,21 @@ const ( ) // SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +func SetupExecutorBuilder(lc *fs_tool.LifeCycle, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk - val, err := utils.GetValidators(sdk, srcFilePath) + val, err := utils.GetValidators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, srcFilePath) + prep, err := utils.GetPreparators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(execFilePath). - WithWorkingDir(baseFolderPath). + WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). + WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -53,13 +54,14 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE WithCompiler(). WithCommand(executorConfig.CompileCmd). WithArgs(executorConfig.CompileArgs). - WithFileName(srcFilePath). + WithFileName(lc.GetAbsoluteSourceFilePath()). WithRunner(). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder switch sdk { @@ -67,7 +69,7 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE args := make([]string, 0) for _, arg := range executorConfig.RunArgs { if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(baseFolderPath, javaLogConfigFileName) + logConfigFilePath := filepath.Join(lc.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) } args = append(args, arg) @@ -77,7 +79,7 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE builder = builder. WithExecutableFileName(""). WithRunner(). - WithCommand(execFilePath).ExecutorBuilder + WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder case pb.Sdk_SDK_PYTHON: // Nothing is needed for Python case pb.Sdk_SDK_SCIO: 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 46a64ccf6f43..c9a8feec7380 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -71,13 +71,12 @@ func TestSetupExecutor(t *testing.T) { WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder type args struct { - srcFilePath string - baseFolderPath string - execFilePath string - sdkEnv *environment.BeamEnvs + lc *fs_tool.LifeCycle + sdkEnv *environment.BeamEnvs } tests := []struct { name string @@ -89,7 +88,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, + args: args{lc, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, want: nil, wantErr: true, }, @@ -97,14 +96,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv}, + args: args{lc, sdkEnv}, want: &wantExecutor, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := SetupExecutorBuilder(tt.args.srcFilePath, tt.args.baseFolderPath, tt.args.execFilePath, tt.args.sdkEnv) + got, err := SetupExecutorBuilder(tt.args.lc, 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/utils/validators_utils.go b/playground/backend/internal/utils/validators_utils.go index 8cbe367c9b4b..539ac9496391 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/playground/backend/internal/utils/validators_utils.go @@ -28,7 +28,7 @@ func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) case pb.Sdk_SDK_JAVA: val = validators.GetJavaValidators(filepath) case pb.Sdk_SDK_GO: - val = validators.GetGoValidators() + val = validators.GetGoValidators(filepath) case pb.Sdk_SDK_PYTHON: val = validators.GetPythonValidators() default: diff --git a/playground/backend/internal/validators/go_validators.go b/playground/backend/internal/validators/go_validators.go index 4258971bbae8..1694f0705afc 100644 --- a/playground/backend/internal/validators/go_validators.go +++ b/playground/backend/internal/validators/go_validators.go @@ -15,7 +15,34 @@ package validators +import ( + "beam.apache.org/playground/backend/internal/logger" + "io/ioutil" + "strings" +) + +const goUnitTestPattern = "*testing.T" + // GetGoValidators return validators methods that should be applied to Go code -func GetGoValidators() *[]Validator { - return &[]Validator{} +func GetGoValidators(filePath string) *[]Validator { + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = filePath + unitTestValidator := Validator{ + Validator: CheckIsUnitTestGo, + Args: validatorArgs, + Name: UnitTestValidatorName, + } + validators := []Validator{unitTestValidator} + return &validators +} + +func CheckIsUnitTestGo(args ...interface{}) (bool, error) { + filePath := args[0].(string) + code, err := ioutil.ReadFile(filePath) + if err != nil { + logger.Errorf("Validation: Error during open file: %s, err: %s\n", filePath, err.Error()) + return false, err + } + // check whether Go code is unit test code + return strings.Contains(string(code), goUnitTestPattern), nil } diff --git a/playground/backend/internal/validators/go_validators_test.go b/playground/backend/internal/validators/go_validators_test.go new file mode 100644 index 000000000000..2b22165ed86f --- /dev/null +++ b/playground/backend/internal/validators/go_validators_test.go @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "testing" +) + +const ( + goUnitTestFilePath = "unitTestCode.go" + goCodePath = "code.go" + goUnitTestCode = "func TestDedup(t *testing.T) {\n\ttests := []struct {\n\t\tdups []interface{}\n\t\texp []interface{}\n}}" + goCode = "func main() {\n\t// beam.Init() is an initialization hook that must be called on startup.\n\tbeam.Init()\n\n\t// Create the Pipeline object and root scope.\n\tp := beam.NewPipeline()\n\ts := p.Root()\n}" +) + +func TestCheckIsUnitTestGo(t *testing.T) { + testValidatorArgs := make([]interface{}, 1) + testValidatorArgs[0] = goUnitTestFilePath + + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = goCodePath + + type args struct { + args []interface{} + } + tests := []struct { + name string + args args + want bool + wantErr bool + }{ + { + // Test if code is unit test code + name: "if unit test", + args: args{ + testValidatorArgs, + }, + want: true, + wantErr: false, + }, + { + // Test if code is not unit test code + name: "if not unit test", + args: args{ + validatorArgs, + }, + want: false, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := CheckIsUnitTestGo(tt.args.args...) + if (err != nil) != tt.wantErr { + t.Errorf("CheckIsUnitTestGo() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("CheckIsUnitTestGo() got = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/validators/java_validators.go b/playground/backend/internal/validators/java_validators.go index 52f86fbc41bc..61b89646dde8 100644 --- a/playground/backend/internal/validators/java_validators.go +++ b/playground/backend/internal/validators/java_validators.go @@ -39,7 +39,7 @@ func GetJavaValidators(filePath string) *[]Validator { Name: "Valid path", } unitTestValidator := Validator{ - Validator: CheckIsUnitTests, + Validator: CheckIsUnitTestJava, Args: validatorArgs, Name: UnitTestValidatorName, } @@ -47,7 +47,7 @@ func GetJavaValidators(filePath string) *[]Validator { return &validators } -func CheckIsUnitTests(args ...interface{}) (bool, error) { +func CheckIsUnitTestJava(args ...interface{}) (bool, error) { filePath := args[0].(string) code, err := ioutil.ReadFile(filePath) if err != nil { diff --git a/playground/backend/internal/validators/java_validators_test.go b/playground/backend/internal/validators/java_validators_test.go index 70ad4dcec780..e3970ccd50d9 100644 --- a/playground/backend/internal/validators/java_validators_test.go +++ b/playground/backend/internal/validators/java_validators_test.go @@ -16,52 +16,22 @@ package validators import ( - "fmt" - "os" "testing" ) -const unitTestFilePath = "unitTestCode.java" -const filePath = "code.java" -const unitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" -const code = "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}" - -func TestMain(m *testing.M) { - setup() - defer teardown() - m.Run() -} - -func setup() { - writeFile(unitTestFilePath, unitTestCode) - writeFile(filePath, code) -} - -func teardown() { - removeFile(unitTestFilePath) - removeFile(filePath) -} - -func removeFile(path string) { - err := os.Remove(path) - if err != nil { - panic(fmt.Errorf("error during test teardown: %s", err.Error())) - } -} - -func writeFile(path string, code string) { - err := os.WriteFile(path, []byte(code), 0600) - if err != nil { - panic(fmt.Errorf("error during test setup: %s", err.Error())) - } -} +const ( + javaUnitTestFilePath = "unitTestCode.java" + javaCodePath = "code.java" + javaUnitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" + javaCode = "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}" +) func TestCheckIsUnitTests(t *testing.T) { testValidatorArgs := make([]interface{}, 1) - testValidatorArgs[0] = unitTestFilePath + testValidatorArgs[0] = javaUnitTestFilePath validatorArgs := make([]interface{}, 1) - validatorArgs[0] = filePath + validatorArgs[0] = javaCodePath type args struct { args []interface{} @@ -93,13 +63,13 @@ func TestCheckIsUnitTests(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := CheckIsUnitTests(tt.args.args...) + got, err := CheckIsUnitTestJava(tt.args.args...) if (err != nil) != tt.wantErr { - t.Errorf("CheckIsUnitTests() error = %v, wantErr %v", err, tt.wantErr) + t.Errorf("CheckIsUnitTestJava() error = %v, wantErr %v", err, tt.wantErr) return } if got != tt.want { - t.Errorf("CheckIsUnitTests() got = %v, want %v", got, tt.want) + t.Errorf("CheckIsUnitTestJava() got = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/validators/validator_test.go b/playground/backend/internal/validators/validator_test.go new file mode 100644 index 000000000000..04f5c5cc1945 --- /dev/null +++ b/playground/backend/internal/validators/validator_test.go @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "fmt" + "os" + "testing" +) + +func TestMain(m *testing.M) { + setup() + defer teardown() + m.Run() +} + +func setup() { + writeFile(javaUnitTestFilePath, javaUnitTestCode) + writeFile(javaCodePath, javaCode) + writeFile(goUnitTestFilePath, goUnitTestCode) + writeFile(goCodePath, goCode) +} + +func teardown() { + removeFile(javaUnitTestFilePath) + removeFile(javaCodePath) + removeFile(goUnitTestFilePath) + removeFile(goCodePath) +} + +func removeFile(path string) { + err := os.Remove(path) + if err != nil { + panic(fmt.Errorf("error during test teardown: %s", err.Error())) + } +} + +func writeFile(path string, code string) { + err := os.WriteFile(path, []byte(code), 0600) + if err != nil { + panic(fmt.Errorf("error during test setup: %s", err.Error())) + } +} From 333e54408ff16b8b4bfe5ac22efeffdfa23f4690 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Thu, 25 Nov 2021 22:45:36 +0300 Subject: [PATCH 04/12] Add go unit tests support --- playground/backend/configs/SDK_GO.json | 6 +- playground/backend/containers/go/Dockerfile | 2 +- playground/backend/containers/java/Dockerfile | 2 +- .../code_processing/code_processing.go | 2 +- .../environment/environment_service_test.go | 2 +- .../internal/executors/executor_builder.go | 8 ++ .../internal/executors/executor_test.go | 10 +-- playground/backend/internal/fs_tool/fs.go | 6 ++ .../setup_tools/builder/setup_builder.go | 20 +++-- .../setup_tools/builder/setup_builder_test.go | 13 ++-- .../internal/utils/validators_utils.go | 2 +- .../internal/validators/go_validators.go | 31 +++++++- .../internal/validators/go_validators_test.go | 76 +++++++++++++++++++ .../internal/validators/java_validators.go | 4 +- .../validators/java_validators_test.go | 52 +++---------- .../internal/validators/validator_test.go | 56 ++++++++++++++ 16 files changed, 221 insertions(+), 71 deletions(-) create mode 100644 playground/backend/internal/validators/go_validators_test.go create mode 100644 playground/backend/internal/validators/validator_test.go diff --git a/playground/backend/configs/SDK_GO.json b/playground/backend/configs/SDK_GO.json index 68a5a27f93ba..0c4775be5e3b 100644 --- a/playground/backend/configs/SDK_GO.json +++ b/playground/backend/configs/SDK_GO.json @@ -1,11 +1,15 @@ { "compile_cmd": "go", "run_cmd": "", + "test_cmd": "go", "compile_args": [ "build", "-o", "bin" ], "run_args": [ + ], + "test_args": [ + "test" ] -} \ No newline at end of file +} diff --git a/playground/backend/containers/go/Dockerfile b/playground/backend/containers/go/Dockerfile index 493b8b20f8e8..bb130f6da776 100644 --- a/playground/backend/containers/go/Dockerfile +++ b/playground/backend/containers/go/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ############################################################################### #Dokerfile to set up the Beam Go SDK -ARG BASE_IMAGE golang:1.16-buster +ARG BASE_IMAGE=golang:1.16-buster FROM $BASE_IMAGE # Setup Go Environment diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index 38303052e40b..cc95e19cd2f5 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG BASE_IMAGE +ARG BASE_IMAGE=apache/beam_java8_sdk:latest FROM golang:1.17-buster AS build # Setup Go Environment diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 281ec44bc46d..3673603c13fc 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(ctxWithTimeout, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv) + executorBuilder, err := builder.SetupExecutorBuilder(lc, sdkEnv) if err != nil { _ = processSetupError(err, pipelineId, cacheService, ctxWithTimeout) return diff --git a/playground/backend/internal/environment/environment_service_test.go b/playground/backend/internal/environment/environment_service_test.go index 5d5ab1aeacb1..88c032bc906b 100644 --- a/playground/backend/internal/environment/environment_service_test.go +++ b/playground/backend/internal/environment/environment_service_test.go @@ -63,7 +63,7 @@ func setup() error { } func teardown() { - err := os.RemoveAll(configFolderName) + err := os.Remove(configFolderName) if err != nil { panic(fmt.Errorf("error during test setup: %s", err.Error())) } diff --git a/playground/backend/internal/executors/executor_builder.go b/playground/backend/internal/executors/executor_builder.go index 44fabff48ee3..aee96ee3de18 100644 --- a/playground/backend/internal/executors/executor_builder.go +++ b/playground/backend/internal/executors/executor_builder.go @@ -165,6 +165,14 @@ func (b *UnitTestExecutorBuilder) WithArgs(testArgs []string) *UnitTestExecutorB return b } +// WithWorkingDir adds dir path to executor +func (b *UnitTestExecutorBuilder) WithWorkingDir(dir string) *UnitTestExecutorBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.testArgs.workingDir = dir + }) + return b +} + //WithGraphOutput adds the need of graph output to executor func (b *UnitTestExecutorBuilder) WithGraphOutput() *UnitTestExecutorBuilder { b.actions = append(b.actions, func(e *Executor) { diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index b1ac6c94e020..5210667cb230 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -51,7 +51,10 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath } builder := NewExecutorBuilder(). WithExecutableFileName("HelloWorld"). - WithWorkingDir(workingDir). + WithWorkingDir(workingDir).WithValidator(). + WithSdkValidators(validatorsFuncs). + WithPreparator(). + WithSdkPreparators(preparatorsFuncs). WithCompiler(). WithCommand(envs.ExecutorConfig.CompileCmd). WithArgs(envs.ExecutorConfig.CompileArgs). @@ -62,10 +65,7 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath WithTestRunner(). WithCommand(envs.ExecutorConfig.TestCmd). WithArgs(envs.ExecutorConfig.TestArgs). - WithValidator(). - WithSdkValidators(validatorsFuncs). - WithPreparator(). - WithSdkPreparators(preparatorsFuncs). + WithWorkingDir(workingDir). ExecutorBuilder return &builder } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 07c78ddfa2ce..c1cb75a97598 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -166,3 +166,9 @@ func (l *LifeCycle) GetAbsoluteLogFilePath() string { 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/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index 1184cd9557ed..119c9736a481 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -19,6 +19,7 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/environment" "beam.apache.org/playground/backend/internal/executors" + "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/utils" "fmt" "path/filepath" @@ -31,21 +32,21 @@ const ( ) // SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +func SetupExecutorBuilder(lc *fs_tool.LifeCycle, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk - val, err := utils.GetValidators(sdk, srcFilePath) + val, err := utils.GetValidators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, srcFilePath) + prep, err := utils.GetPreparators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(execFilePath). - WithWorkingDir(baseFolderPath). + WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). + WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -53,13 +54,14 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE WithCompiler(). WithCommand(executorConfig.CompileCmd). WithArgs(executorConfig.CompileArgs). - WithFileName(srcFilePath). + WithFileName(lc.GetAbsoluteSourceFilePath()). WithRunner(). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder switch sdk { @@ -67,7 +69,7 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE args := make([]string, 0) for _, arg := range executorConfig.RunArgs { if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(baseFolderPath, javaLogConfigFileName) + logConfigFilePath := filepath.Join(lc.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) } args = append(args, arg) @@ -77,7 +79,9 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE builder = builder. WithExecutableFileName(""). WithRunner(). - WithCommand(execFilePath).ExecutorBuilder + WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder + case pb.Sdk_SDK_PYTHON: + builder = *builder.WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()) default: return nil, fmt.Errorf("incorrect sdk: %s", sdkEnv.ApacheBeamSdk) } 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 46a64ccf6f43..c9a8feec7380 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -71,13 +71,12 @@ func TestSetupExecutor(t *testing.T) { WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder type args struct { - srcFilePath string - baseFolderPath string - execFilePath string - sdkEnv *environment.BeamEnvs + lc *fs_tool.LifeCycle + sdkEnv *environment.BeamEnvs } tests := []struct { name string @@ -89,7 +88,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, + args: args{lc, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, want: nil, wantErr: true, }, @@ -97,14 +96,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv}, + args: args{lc, sdkEnv}, want: &wantExecutor, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := SetupExecutorBuilder(tt.args.srcFilePath, tt.args.baseFolderPath, tt.args.execFilePath, tt.args.sdkEnv) + got, err := SetupExecutorBuilder(tt.args.lc, 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/utils/validators_utils.go b/playground/backend/internal/utils/validators_utils.go index 8cbe367c9b4b..539ac9496391 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/playground/backend/internal/utils/validators_utils.go @@ -28,7 +28,7 @@ func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) case pb.Sdk_SDK_JAVA: val = validators.GetJavaValidators(filepath) case pb.Sdk_SDK_GO: - val = validators.GetGoValidators() + val = validators.GetGoValidators(filepath) case pb.Sdk_SDK_PYTHON: val = validators.GetPythonValidators() default: diff --git a/playground/backend/internal/validators/go_validators.go b/playground/backend/internal/validators/go_validators.go index 4258971bbae8..1694f0705afc 100644 --- a/playground/backend/internal/validators/go_validators.go +++ b/playground/backend/internal/validators/go_validators.go @@ -15,7 +15,34 @@ package validators +import ( + "beam.apache.org/playground/backend/internal/logger" + "io/ioutil" + "strings" +) + +const goUnitTestPattern = "*testing.T" + // GetGoValidators return validators methods that should be applied to Go code -func GetGoValidators() *[]Validator { - return &[]Validator{} +func GetGoValidators(filePath string) *[]Validator { + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = filePath + unitTestValidator := Validator{ + Validator: CheckIsUnitTestGo, + Args: validatorArgs, + Name: UnitTestValidatorName, + } + validators := []Validator{unitTestValidator} + return &validators +} + +func CheckIsUnitTestGo(args ...interface{}) (bool, error) { + filePath := args[0].(string) + code, err := ioutil.ReadFile(filePath) + if err != nil { + logger.Errorf("Validation: Error during open file: %s, err: %s\n", filePath, err.Error()) + return false, err + } + // check whether Go code is unit test code + return strings.Contains(string(code), goUnitTestPattern), nil } diff --git a/playground/backend/internal/validators/go_validators_test.go b/playground/backend/internal/validators/go_validators_test.go new file mode 100644 index 000000000000..2b22165ed86f --- /dev/null +++ b/playground/backend/internal/validators/go_validators_test.go @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "testing" +) + +const ( + goUnitTestFilePath = "unitTestCode.go" + goCodePath = "code.go" + goUnitTestCode = "func TestDedup(t *testing.T) {\n\ttests := []struct {\n\t\tdups []interface{}\n\t\texp []interface{}\n}}" + goCode = "func main() {\n\t// beam.Init() is an initialization hook that must be called on startup.\n\tbeam.Init()\n\n\t// Create the Pipeline object and root scope.\n\tp := beam.NewPipeline()\n\ts := p.Root()\n}" +) + +func TestCheckIsUnitTestGo(t *testing.T) { + testValidatorArgs := make([]interface{}, 1) + testValidatorArgs[0] = goUnitTestFilePath + + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = goCodePath + + type args struct { + args []interface{} + } + tests := []struct { + name string + args args + want bool + wantErr bool + }{ + { + // Test if code is unit test code + name: "if unit test", + args: args{ + testValidatorArgs, + }, + want: true, + wantErr: false, + }, + { + // Test if code is not unit test code + name: "if not unit test", + args: args{ + validatorArgs, + }, + want: false, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := CheckIsUnitTestGo(tt.args.args...) + if (err != nil) != tt.wantErr { + t.Errorf("CheckIsUnitTestGo() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("CheckIsUnitTestGo() got = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/validators/java_validators.go b/playground/backend/internal/validators/java_validators.go index 52f86fbc41bc..61b89646dde8 100644 --- a/playground/backend/internal/validators/java_validators.go +++ b/playground/backend/internal/validators/java_validators.go @@ -39,7 +39,7 @@ func GetJavaValidators(filePath string) *[]Validator { Name: "Valid path", } unitTestValidator := Validator{ - Validator: CheckIsUnitTests, + Validator: CheckIsUnitTestJava, Args: validatorArgs, Name: UnitTestValidatorName, } @@ -47,7 +47,7 @@ func GetJavaValidators(filePath string) *[]Validator { return &validators } -func CheckIsUnitTests(args ...interface{}) (bool, error) { +func CheckIsUnitTestJava(args ...interface{}) (bool, error) { filePath := args[0].(string) code, err := ioutil.ReadFile(filePath) if err != nil { diff --git a/playground/backend/internal/validators/java_validators_test.go b/playground/backend/internal/validators/java_validators_test.go index 70ad4dcec780..e3970ccd50d9 100644 --- a/playground/backend/internal/validators/java_validators_test.go +++ b/playground/backend/internal/validators/java_validators_test.go @@ -16,52 +16,22 @@ package validators import ( - "fmt" - "os" "testing" ) -const unitTestFilePath = "unitTestCode.java" -const filePath = "code.java" -const unitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" -const code = "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}" - -func TestMain(m *testing.M) { - setup() - defer teardown() - m.Run() -} - -func setup() { - writeFile(unitTestFilePath, unitTestCode) - writeFile(filePath, code) -} - -func teardown() { - removeFile(unitTestFilePath) - removeFile(filePath) -} - -func removeFile(path string) { - err := os.Remove(path) - if err != nil { - panic(fmt.Errorf("error during test teardown: %s", err.Error())) - } -} - -func writeFile(path string, code string) { - err := os.WriteFile(path, []byte(code), 0600) - if err != nil { - panic(fmt.Errorf("error during test setup: %s", err.Error())) - } -} +const ( + javaUnitTestFilePath = "unitTestCode.java" + javaCodePath = "code.java" + javaUnitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" + javaCode = "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}" +) func TestCheckIsUnitTests(t *testing.T) { testValidatorArgs := make([]interface{}, 1) - testValidatorArgs[0] = unitTestFilePath + testValidatorArgs[0] = javaUnitTestFilePath validatorArgs := make([]interface{}, 1) - validatorArgs[0] = filePath + validatorArgs[0] = javaCodePath type args struct { args []interface{} @@ -93,13 +63,13 @@ func TestCheckIsUnitTests(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := CheckIsUnitTests(tt.args.args...) + got, err := CheckIsUnitTestJava(tt.args.args...) if (err != nil) != tt.wantErr { - t.Errorf("CheckIsUnitTests() error = %v, wantErr %v", err, tt.wantErr) + t.Errorf("CheckIsUnitTestJava() error = %v, wantErr %v", err, tt.wantErr) return } if got != tt.want { - t.Errorf("CheckIsUnitTests() got = %v, want %v", got, tt.want) + t.Errorf("CheckIsUnitTestJava() got = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/validators/validator_test.go b/playground/backend/internal/validators/validator_test.go new file mode 100644 index 000000000000..04f5c5cc1945 --- /dev/null +++ b/playground/backend/internal/validators/validator_test.go @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "fmt" + "os" + "testing" +) + +func TestMain(m *testing.M) { + setup() + defer teardown() + m.Run() +} + +func setup() { + writeFile(javaUnitTestFilePath, javaUnitTestCode) + writeFile(javaCodePath, javaCode) + writeFile(goUnitTestFilePath, goUnitTestCode) + writeFile(goCodePath, goCode) +} + +func teardown() { + removeFile(javaUnitTestFilePath) + removeFile(javaCodePath) + removeFile(goUnitTestFilePath) + removeFile(goCodePath) +} + +func removeFile(path string) { + err := os.Remove(path) + if err != nil { + panic(fmt.Errorf("error during test teardown: %s", err.Error())) + } +} + +func writeFile(path string, code string) { + err := os.WriteFile(path, []byte(code), 0600) + if err != nil { + panic(fmt.Errorf("error during test setup: %s", err.Error())) + } +} From 53c2dc408d8b69deb354f52e414027d4c19dae7d Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Fri, 3 Dec 2021 19:30:27 +0300 Subject: [PATCH 05/12] Merge conflicts --- .../backend/internal/environment/environment_service_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/playground/backend/internal/environment/environment_service_test.go b/playground/backend/internal/environment/environment_service_test.go index 88c032bc906b..5d5ab1aeacb1 100644 --- a/playground/backend/internal/environment/environment_service_test.go +++ b/playground/backend/internal/environment/environment_service_test.go @@ -63,7 +63,7 @@ func setup() error { } func teardown() { - err := os.Remove(configFolderName) + err := os.RemoveAll(configFolderName) if err != nil { panic(fmt.Errorf("error during test setup: %s", err.Error())) } From b4bfccf81e0dc8edcad3ea0c9fbad2036c78e454 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Mon, 6 Dec 2021 15:10:05 +0300 Subject: [PATCH 06/12] support Python unit tests --- playground/backend/configs/SDK_PYTHON.json | 4 ++- .../backend/containers/python/Dockerfile | 6 ++-- .../internal/streaming/run_output_writer.go | 1 + .../internal/utils/validators_utils.go | 2 +- .../internal/validators/python_validators.go | 34 ++++++++++++++++--- 5 files changed, 38 insertions(+), 9 deletions(-) diff --git a/playground/backend/configs/SDK_PYTHON.json b/playground/backend/configs/SDK_PYTHON.json index bb6ae89adff5..d1f89d366a52 100644 --- a/playground/backend/configs/SDK_PYTHON.json +++ b/playground/backend/configs/SDK_PYTHON.json @@ -1,6 +1,8 @@ { "compile_cmd": "", "run_cmd": "python3", + "test_cmd": "pytest", "compile_args": [], - "run_args": [] + "run_args": [], + "test_args": [] } diff --git a/playground/backend/containers/python/Dockerfile b/playground/backend/containers/python/Dockerfile index d02dff63e1d2..3ae1bd80feb2 100644 --- a/playground/backend/containers/python/Dockerfile +++ b/playground/backend/containers/python/Dockerfile @@ -16,8 +16,7 @@ # limitations under the License. ############################################################################### ARG BASE_IMAGE=apache/beam_python3.7_sdk:latest -ARG GO_BASE_IMAGE=golang:1.17-buster -FROM $GO_BASE_IMAGE AS build +FROM golang:1.17-buster AS build # Setup Go Environment ENV GOPATH /go @@ -47,5 +46,6 @@ ENV BEAM_SDK="SDK_PYTHON" COPY --from=build /go/bin/server_python_backend /opt/playground/backend/ COPY --from=build /go/src/playground/backend/configs /opt/playground/backend/configs/ + RUN printenv -ENTRYPOINT ["/opt/playground/backend/server_python_backend"] +ENTRYPOINT ["/opt/playground/backend/server_python_backend"] \ No newline at end of file diff --git a/playground/backend/internal/streaming/run_output_writer.go b/playground/backend/internal/streaming/run_output_writer.go index 94bd0a119f8a..ae1f19161449 100644 --- a/playground/backend/internal/streaming/run_output_writer.go +++ b/playground/backend/internal/streaming/run_output_writer.go @@ -56,5 +56,6 @@ func (row *RunOutputWriter) Write(p []byte) (int, error) { if err != nil { return 0, err } + fmt.Printf("Added: %s\n", str) return len(p), nil } diff --git a/playground/backend/internal/utils/validators_utils.go b/playground/backend/internal/utils/validators_utils.go index 539ac9496391..05e402c31335 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/playground/backend/internal/utils/validators_utils.go @@ -30,7 +30,7 @@ func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) case pb.Sdk_SDK_GO: val = validators.GetGoValidators(filepath) case pb.Sdk_SDK_PYTHON: - val = validators.GetPythonValidators() + val = validators.GetPyValidators(filepath) default: return nil, fmt.Errorf("incorrect sdk: %s", sdk) } diff --git a/playground/backend/internal/validators/python_validators.go b/playground/backend/internal/validators/python_validators.go index 4d997337df78..c0628ce937d0 100644 --- a/playground/backend/internal/validators/python_validators.go +++ b/playground/backend/internal/validators/python_validators.go @@ -15,8 +15,34 @@ package validators -// GetPythonValidators return validators methods that should be applied to Go code -func GetPythonValidators() *[]Validator { - //TODO: Will be added in task [BEAM-13292] - return &[]Validator{} +import ( + "beam.apache.org/playground/backend/internal/logger" + "io/ioutil" + "strings" +) + +const pyUnitTestPattern = "import unittest" + +// GetPyValidators return validators methods that should be applied to Python code +func GetPyValidators(filePath string) *[]Validator { + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = filePath + unitTestValidator := Validator{ + Validator: CheckIsUnitTestPy, + Args: validatorArgs, + Name: UnitTestValidatorName, + } + validators := []Validator{unitTestValidator} + return &validators +} + +func CheckIsUnitTestPy(args ...interface{}) (bool, error) { + filePath := args[0].(string) + code, err := ioutil.ReadFile(filePath) + if err != nil { + logger.Errorf("Validation: Error during open file: %s, err: %s\n", filePath, err.Error()) + return false, err + } + // check whether Python code is unit test code + return strings.Contains(string(code), pyUnitTestPattern), nil } From 03462ef8e62c7d5ac51dbeafb0acb955db1eadce Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Tue, 7 Dec 2021 22:19:02 +0300 Subject: [PATCH 07/12] bugfix --- .../backend/internal/setup_tools/builder/setup_builder.go | 3 +-- playground/backend/internal/streaming/run_output_writer.go | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index cae091deec1b..a745a28ba639 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -81,8 +81,7 @@ func SetupExecutorBuilder(lc *fs_tool.LifeCycle, sdkEnv *environment.BeamEnvs) ( WithRunner(). WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder case pb.Sdk_SDK_PYTHON: - builder = *builder.WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). - WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder + builder = *builder.WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()) case pb.Sdk_SDK_SCIO: return nil, fmt.Errorf("SCIO is not supported yet") default: diff --git a/playground/backend/internal/streaming/run_output_writer.go b/playground/backend/internal/streaming/run_output_writer.go index ae1f19161449..94bd0a119f8a 100644 --- a/playground/backend/internal/streaming/run_output_writer.go +++ b/playground/backend/internal/streaming/run_output_writer.go @@ -56,6 +56,5 @@ func (row *RunOutputWriter) Write(p []byte) (int, error) { if err != nil { return 0, err } - fmt.Printf("Added: %s\n", str) return len(p), nil } From 6c87b92b383577bbfe98257ab645e25e29812703 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Thu, 25 Nov 2021 22:45:36 +0300 Subject: [PATCH 08/12] Add go unit tests support --- playground/backend/configs/SDK_GO.json | 6 +- playground/backend/containers/go/Dockerfile | 6 +- playground/backend/containers/java/Dockerfile | 2 +- .../code_processing/code_processing.go | 51 ++++++++----- .../backend/internal/executors/executor.go | 5 +- .../internal/executors/executor_builder.go | 8 ++ .../internal/executors/executor_test.go | 10 +-- playground/backend/internal/fs_tool/fs.go | 6 ++ .../internal/preparators/go_preparators.go | 28 ++++++- .../setup_tools/builder/setup_builder.go | 18 +++-- .../setup_tools/builder/setup_builder_test.go | 13 ++-- .../internal/utils/validators_utils.go | 2 +- .../internal/validators/go_validators.go | 31 +++++++- .../internal/validators/go_validators_test.go | 76 +++++++++++++++++++ .../internal/validators/java_validators.go | 4 +- .../validators/java_validators_test.go | 52 +++---------- .../internal/validators/validator_test.go | 56 ++++++++++++++ 17 files changed, 277 insertions(+), 97 deletions(-) create mode 100644 playground/backend/internal/validators/go_validators_test.go create mode 100644 playground/backend/internal/validators/validator_test.go diff --git a/playground/backend/configs/SDK_GO.json b/playground/backend/configs/SDK_GO.json index 68a5a27f93ba..0c4775be5e3b 100644 --- a/playground/backend/configs/SDK_GO.json +++ b/playground/backend/configs/SDK_GO.json @@ -1,11 +1,15 @@ { "compile_cmd": "go", "run_cmd": "", + "test_cmd": "go", "compile_args": [ "build", "-o", "bin" ], "run_args": [ + ], + "test_args": [ + "test" ] -} \ No newline at end of file +} diff --git a/playground/backend/containers/go/Dockerfile b/playground/backend/containers/go/Dockerfile index 493b8b20f8e8..45f2cea98d94 100644 --- a/playground/backend/containers/go/Dockerfile +++ b/playground/backend/containers/go/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ############################################################################### #Dokerfile to set up the Beam Go SDK -ARG BASE_IMAGE golang:1.16-buster +ARG BASE_IMAGE=golang:1.16-buster FROM $BASE_IMAGE # Setup Go Environment @@ -56,6 +56,4 @@ ENV BEAM_SDK="SDK_GO" ## Copy build result COPY src/configs /opt/playground/backend/configs/ -ENTRYPOINT ["/opt/playground/backend/server_go_backend"] - - +ENTRYPOINT ["/opt/playground/backend/server_go_backend"] \ No newline at end of file diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index cfd2f016e109..06571a77d068 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### -ARG BASE_IMAGE +ARG BASE_IMAGE=apache/beam_java8_sdk:latest FROM golang:1.17-buster AS build # Setup Go Environment diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 917eae66dd31..d2ff84b487b6 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(ctxWithTimeout, pipelineId, cancelChannel, cacheService) - executorBuilder, err := builder.SetupExecutorBuilder(lc.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv) + executorBuilder, err := builder.SetupExecutorBuilder(lc, sdkEnv) if err != nil { _ = processSetupError(err, pipelineId, cacheService, ctxWithTimeout) return @@ -88,6 +88,12 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl _ = processError(ctxWithTimeout, errorChannel, pipelineId, cacheService, "Validate", pb.Status_STATUS_VALIDATION_ERROR) return } + // Check if unit test + isUnitTest := false + valResult, ok := validationResults.Load(validators.UnitTestValidatorName) + if ok && valResult.(bool) { + isUnitTest = true + } if err := processSuccess(ctxWithTimeout, pipelineId, cacheService, "Validate", pb.Status_STATUS_PREPARING); err != nil { return } @@ -95,7 +101,7 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl // Prepare logger.Infof("%s: Prepare() ...\n", pipelineId) prepareFunc := executor.Prepare() - go prepareFunc(successChannel, errorChannel) + go prepareFunc(successChannel, errorChannel, isUnitTest) ok, err = processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel) if err != nil { @@ -111,23 +117,29 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl switch sdkEnv.ApacheBeamSdk { case pb.Sdk_SDK_JAVA, pb.Sdk_SDK_GO: - // Compile - logger.Infof("%s: Compile() ...\n", pipelineId) - compileCmd := executor.Compile(ctxWithTimeout) - var compileError bytes.Buffer - var compileOutput bytes.Buffer - runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) - - ok, err = processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel) - if err != nil { - return - } - if !ok { - _ = processCompileError(ctxWithTimeout, errorChannel, compileError.Bytes(), pipelineId, cacheService) - return - } - if err := processCompileSuccess(ctxWithTimeout, compileOutput.Bytes(), pipelineId, cacheService); err != nil { - return + if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && !isUnitTest { + // Compile + logger.Infof("%s: Compile() ...\n", pipelineId) + compileCmd := executor.Compile(ctxWithTimeout) + var compileError bytes.Buffer + var compileOutput bytes.Buffer + runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) + + ok, err = processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel) + if err != nil { + return + } + if !ok { + _ = processCompileError(ctxWithTimeout, errorChannel, compileError.Bytes(), pipelineId, cacheService) + return + } + if err := processCompileSuccess(ctxWithTimeout, compileOutput.Bytes(), pipelineId, cacheService); err != nil { + return + } + } else { + if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { + return + } } case pb.Sdk_SDK_PYTHON: if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { @@ -249,6 +261,7 @@ func runCmdWithOutput(cmd *exec.Cmd, stdOutput io.Writer, stdError *bytes.Buffer cmd.Stdout = stdOutput cmd.Stderr = stdError go func(cmd *exec.Cmd, successChannel chan bool, errChannel chan error) { + fmt.Println("!!! cmd: " + cmd.String()) err := cmd.Run() if err != nil { errChannel <- err diff --git a/playground/backend/internal/executors/executor.go b/playground/backend/internal/executors/executor.go index 26e7747d9849..1291695c9edf 100644 --- a/playground/backend/internal/executors/executor.go +++ b/playground/backend/internal/executors/executor.go @@ -75,9 +75,10 @@ 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) { - return func(doneCh chan bool, errCh chan error) { +func (ex *Executor) Prepare() func(chan bool, chan error, bool) { + return func(doneCh chan bool, errCh chan error, isUnitTest bool) { for _, preparator := range ex.preparators { + preparator.Args = append(preparator.Args, isUnitTest) err := preparator.Prepare(preparator.Args...) if err != nil { errCh <- err diff --git a/playground/backend/internal/executors/executor_builder.go b/playground/backend/internal/executors/executor_builder.go index 44fabff48ee3..aee96ee3de18 100644 --- a/playground/backend/internal/executors/executor_builder.go +++ b/playground/backend/internal/executors/executor_builder.go @@ -165,6 +165,14 @@ func (b *UnitTestExecutorBuilder) WithArgs(testArgs []string) *UnitTestExecutorB return b } +// WithWorkingDir adds dir path to executor +func (b *UnitTestExecutorBuilder) WithWorkingDir(dir string) *UnitTestExecutorBuilder { + b.actions = append(b.actions, func(e *Executor) { + e.testArgs.workingDir = dir + }) + return b +} + //WithGraphOutput adds the need of graph output to executor func (b *UnitTestExecutorBuilder) WithGraphOutput() *UnitTestExecutorBuilder { b.actions = append(b.actions, func(e *Executor) { diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index b1ac6c94e020..5210667cb230 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -51,7 +51,10 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath } builder := NewExecutorBuilder(). WithExecutableFileName("HelloWorld"). - WithWorkingDir(workingDir). + WithWorkingDir(workingDir).WithValidator(). + WithSdkValidators(validatorsFuncs). + WithPreparator(). + WithSdkPreparators(preparatorsFuncs). WithCompiler(). WithCommand(envs.ExecutorConfig.CompileCmd). WithArgs(envs.ExecutorConfig.CompileArgs). @@ -62,10 +65,7 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath WithTestRunner(). WithCommand(envs.ExecutorConfig.TestCmd). WithArgs(envs.ExecutorConfig.TestArgs). - WithValidator(). - WithSdkValidators(validatorsFuncs). - WithPreparator(). - WithSdkPreparators(preparatorsFuncs). + WithWorkingDir(workingDir). ExecutorBuilder return &builder } diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 07c78ddfa2ce..c1cb75a97598 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -166,3 +166,9 @@ func (l *LifeCycle) GetAbsoluteLogFilePath() string { 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/preparators/go_preparators.go b/playground/backend/internal/preparators/go_preparators.go index 2cedc2f7e8ba..9738e12f0c5b 100644 --- a/playground/backend/internal/preparators/go_preparators.go +++ b/playground/backend/internal/preparators/go_preparators.go @@ -17,13 +17,17 @@ package preparators import ( "errors" + "fmt" "os/exec" "path/filepath" + "strings" ) const ( - nameBinGo = "go" - fmtArgs = "fmt" + goName = "go" + fmtArgs = "fmt" + mvCmd = "mv" + sep = "." ) // GetGoPreparators returns reparation methods that should be applied to Go code @@ -31,13 +35,14 @@ func GetGoPreparators(filePath string) *[]Preparator { preparatorArgs := make([]interface{}, 1) preparatorArgs[0] = filePath formatCodePreparator := Preparator{Prepare: formatCode, Args: preparatorArgs} - return &[]Preparator{formatCodePreparator} + changeNamePreparator := Preparator{Prepare: changeFileName, Args: preparatorArgs} + return &[]Preparator{formatCodePreparator, changeNamePreparator} } // formatCode formats go code func formatCode(args ...interface{}) error { filePath := args[0].(string) - cmd := exec.Command(nameBinGo, fmtArgs, filepath.Base(filePath)) + cmd := exec.Command(goName, fmtArgs, filepath.Base(filePath)) cmd.Dir = filepath.Dir(filePath) stdout, err := cmd.CombinedOutput() if err != nil { @@ -45,3 +50,18 @@ func formatCode(args ...interface{}) error { } return nil } + +func changeFileName(args ...interface{}) error { + filePath := args[0].(string) + isUnitTest := args[1].(bool) + if isUnitTest { + testFileName := fmt.Sprintf("%s_test.%s", strings.Split(filePath, sep)[0], goName) + cmd := exec.Command(mvCmd, filePath, testFileName) + fmt.Println(cmd.String()) + stdout, err := cmd.CombinedOutput() + if err != nil { + return errors.New(string(stdout)) + } + } + return nil +} diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index fb65aa312a43..e0fa12486838 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -19,6 +19,7 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/environment" "beam.apache.org/playground/backend/internal/executors" + "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/utils" "fmt" "path/filepath" @@ -31,21 +32,21 @@ const ( ) // SetupExecutorBuilder return executor with set args for validator, preparator, compiler and runner -func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { +func SetupExecutorBuilder(lc *fs_tool.LifeCycle, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk - val, err := utils.GetValidators(sdk, srcFilePath) + val, err := utils.GetValidators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } - prep, err := utils.GetPreparators(sdk, srcFilePath) + prep, err := utils.GetPreparators(sdk, lc.GetAbsoluteSourceFilePath()) if err != nil { return nil, err } executorConfig := sdkEnv.ExecutorConfig builder := executors.NewExecutorBuilder(). - WithExecutableFileName(execFilePath). - WithWorkingDir(baseFolderPath). + WithExecutableFileName(lc.GetAbsoluteExecutableFilePath()). + WithWorkingDir(lc.GetAbsoluteBaseFolderPath()). WithValidator(). WithSdkValidators(val). WithPreparator(). @@ -53,13 +54,14 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE WithCompiler(). WithCommand(executorConfig.CompileCmd). WithArgs(executorConfig.CompileArgs). - WithFileName(srcFilePath). + WithFileName(lc.GetAbsoluteSourceFilePath()). WithRunner(). WithCommand(executorConfig.RunCmd). WithArgs(executorConfig.RunArgs). WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder switch sdk { @@ -67,7 +69,7 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE args := make([]string, 0) for _, arg := range executorConfig.RunArgs { if strings.Contains(arg, javaLogConfigFilePlaceholder) { - logConfigFilePath := filepath.Join(baseFolderPath, javaLogConfigFileName) + logConfigFilePath := filepath.Join(lc.GetAbsoluteBaseFolderPath(), javaLogConfigFileName) arg = strings.Replace(arg, javaLogConfigFilePlaceholder, logConfigFilePath, 1) } args = append(args, arg) @@ -77,7 +79,7 @@ func SetupExecutorBuilder(srcFilePath, baseFolderPath, execFilePath string, sdkE builder = builder. WithExecutableFileName(""). WithRunner(). - WithCommand(execFilePath).ExecutorBuilder + WithCommand(lc.GetAbsoluteExecutableFilePath()).ExecutorBuilder case pb.Sdk_SDK_PYTHON: // Nothing is needed for Python case pb.Sdk_SDK_SCIO: 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 46a64ccf6f43..c9a8feec7380 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -71,13 +71,12 @@ func TestSetupExecutor(t *testing.T) { WithTestRunner(). WithCommand(executorConfig.TestCmd). WithArgs(executorConfig.TestArgs). + WithWorkingDir(lc.GetAbsoluteSourceFolderPath()). ExecutorBuilder type args struct { - srcFilePath string - baseFolderPath string - execFilePath string - sdkEnv *environment.BeamEnvs + lc *fs_tool.LifeCycle + sdkEnv *environment.BeamEnvs } tests := []struct { name string @@ -89,7 +88,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, + args: args{lc, environment.NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, executorConfig, "")}, want: nil, wantErr: true, }, @@ -97,14 +96,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.GetAbsoluteSourceFilePath(), lc.GetAbsoluteBaseFolderPath(), lc.GetAbsoluteExecutableFilePath(), sdkEnv}, + args: args{lc, sdkEnv}, want: &wantExecutor, wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := SetupExecutorBuilder(tt.args.srcFilePath, tt.args.baseFolderPath, tt.args.execFilePath, tt.args.sdkEnv) + got, err := SetupExecutorBuilder(tt.args.lc, 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/utils/validators_utils.go b/playground/backend/internal/utils/validators_utils.go index 8cbe367c9b4b..539ac9496391 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/playground/backend/internal/utils/validators_utils.go @@ -28,7 +28,7 @@ func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) case pb.Sdk_SDK_JAVA: val = validators.GetJavaValidators(filepath) case pb.Sdk_SDK_GO: - val = validators.GetGoValidators() + val = validators.GetGoValidators(filepath) case pb.Sdk_SDK_PYTHON: val = validators.GetPythonValidators() default: diff --git a/playground/backend/internal/validators/go_validators.go b/playground/backend/internal/validators/go_validators.go index 4258971bbae8..1694f0705afc 100644 --- a/playground/backend/internal/validators/go_validators.go +++ b/playground/backend/internal/validators/go_validators.go @@ -15,7 +15,34 @@ package validators +import ( + "beam.apache.org/playground/backend/internal/logger" + "io/ioutil" + "strings" +) + +const goUnitTestPattern = "*testing.T" + // GetGoValidators return validators methods that should be applied to Go code -func GetGoValidators() *[]Validator { - return &[]Validator{} +func GetGoValidators(filePath string) *[]Validator { + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = filePath + unitTestValidator := Validator{ + Validator: CheckIsUnitTestGo, + Args: validatorArgs, + Name: UnitTestValidatorName, + } + validators := []Validator{unitTestValidator} + return &validators +} + +func CheckIsUnitTestGo(args ...interface{}) (bool, error) { + filePath := args[0].(string) + code, err := ioutil.ReadFile(filePath) + if err != nil { + logger.Errorf("Validation: Error during open file: %s, err: %s\n", filePath, err.Error()) + return false, err + } + // check whether Go code is unit test code + return strings.Contains(string(code), goUnitTestPattern), nil } diff --git a/playground/backend/internal/validators/go_validators_test.go b/playground/backend/internal/validators/go_validators_test.go new file mode 100644 index 000000000000..2b22165ed86f --- /dev/null +++ b/playground/backend/internal/validators/go_validators_test.go @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "testing" +) + +const ( + goUnitTestFilePath = "unitTestCode.go" + goCodePath = "code.go" + goUnitTestCode = "func TestDedup(t *testing.T) {\n\ttests := []struct {\n\t\tdups []interface{}\n\t\texp []interface{}\n}}" + goCode = "func main() {\n\t// beam.Init() is an initialization hook that must be called on startup.\n\tbeam.Init()\n\n\t// Create the Pipeline object and root scope.\n\tp := beam.NewPipeline()\n\ts := p.Root()\n}" +) + +func TestCheckIsUnitTestGo(t *testing.T) { + testValidatorArgs := make([]interface{}, 1) + testValidatorArgs[0] = goUnitTestFilePath + + validatorArgs := make([]interface{}, 1) + validatorArgs[0] = goCodePath + + type args struct { + args []interface{} + } + tests := []struct { + name string + args args + want bool + wantErr bool + }{ + { + // Test if code is unit test code + name: "if unit test", + args: args{ + testValidatorArgs, + }, + want: true, + wantErr: false, + }, + { + // Test if code is not unit test code + name: "if not unit test", + args: args{ + validatorArgs, + }, + want: false, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := CheckIsUnitTestGo(tt.args.args...) + if (err != nil) != tt.wantErr { + t.Errorf("CheckIsUnitTestGo() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("CheckIsUnitTestGo() got = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/validators/java_validators.go b/playground/backend/internal/validators/java_validators.go index 52f86fbc41bc..61b89646dde8 100644 --- a/playground/backend/internal/validators/java_validators.go +++ b/playground/backend/internal/validators/java_validators.go @@ -39,7 +39,7 @@ func GetJavaValidators(filePath string) *[]Validator { Name: "Valid path", } unitTestValidator := Validator{ - Validator: CheckIsUnitTests, + Validator: CheckIsUnitTestJava, Args: validatorArgs, Name: UnitTestValidatorName, } @@ -47,7 +47,7 @@ func GetJavaValidators(filePath string) *[]Validator { return &validators } -func CheckIsUnitTests(args ...interface{}) (bool, error) { +func CheckIsUnitTestJava(args ...interface{}) (bool, error) { filePath := args[0].(string) code, err := ioutil.ReadFile(filePath) if err != nil { diff --git a/playground/backend/internal/validators/java_validators_test.go b/playground/backend/internal/validators/java_validators_test.go index 70ad4dcec780..e3970ccd50d9 100644 --- a/playground/backend/internal/validators/java_validators_test.go +++ b/playground/backend/internal/validators/java_validators_test.go @@ -16,52 +16,22 @@ package validators import ( - "fmt" - "os" "testing" ) -const unitTestFilePath = "unitTestCode.java" -const filePath = "code.java" -const unitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" -const code = "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}" - -func TestMain(m *testing.M) { - setup() - defer teardown() - m.Run() -} - -func setup() { - writeFile(unitTestFilePath, unitTestCode) - writeFile(filePath, code) -} - -func teardown() { - removeFile(unitTestFilePath) - removeFile(filePath) -} - -func removeFile(path string) { - err := os.Remove(path) - if err != nil { - panic(fmt.Errorf("error during test teardown: %s", err.Error())) - } -} - -func writeFile(path string, code string) { - err := os.WriteFile(path, []byte(code), 0600) - if err != nil { - panic(fmt.Errorf("error during test setup: %s", err.Error())) - } -} +const ( + javaUnitTestFilePath = "unitTestCode.java" + javaCodePath = "code.java" + javaUnitTestCode = "@RunWith(JUnit4.class)\npublic class DeduplicateTest {\n\n @Rule public TestPipeline p = TestPipeline.create();\n\n @Test\n @Category({NeedsRunner.class, UsesTestStream.class})\n public void testInDifferentWindows() {}}" + javaCode = "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}" +) func TestCheckIsUnitTests(t *testing.T) { testValidatorArgs := make([]interface{}, 1) - testValidatorArgs[0] = unitTestFilePath + testValidatorArgs[0] = javaUnitTestFilePath validatorArgs := make([]interface{}, 1) - validatorArgs[0] = filePath + validatorArgs[0] = javaCodePath type args struct { args []interface{} @@ -93,13 +63,13 @@ func TestCheckIsUnitTests(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := CheckIsUnitTests(tt.args.args...) + got, err := CheckIsUnitTestJava(tt.args.args...) if (err != nil) != tt.wantErr { - t.Errorf("CheckIsUnitTests() error = %v, wantErr %v", err, tt.wantErr) + t.Errorf("CheckIsUnitTestJava() error = %v, wantErr %v", err, tt.wantErr) return } if got != tt.want { - t.Errorf("CheckIsUnitTests() got = %v, want %v", got, tt.want) + t.Errorf("CheckIsUnitTestJava() got = %v, want %v", got, tt.want) } }) } diff --git a/playground/backend/internal/validators/validator_test.go b/playground/backend/internal/validators/validator_test.go new file mode 100644 index 000000000000..04f5c5cc1945 --- /dev/null +++ b/playground/backend/internal/validators/validator_test.go @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +import ( + "fmt" + "os" + "testing" +) + +func TestMain(m *testing.M) { + setup() + defer teardown() + m.Run() +} + +func setup() { + writeFile(javaUnitTestFilePath, javaUnitTestCode) + writeFile(javaCodePath, javaCode) + writeFile(goUnitTestFilePath, goUnitTestCode) + writeFile(goCodePath, goCode) +} + +func teardown() { + removeFile(javaUnitTestFilePath) + removeFile(javaCodePath) + removeFile(goUnitTestFilePath) + removeFile(goCodePath) +} + +func removeFile(path string) { + err := os.Remove(path) + if err != nil { + panic(fmt.Errorf("error during test teardown: %s", err.Error())) + } +} + +func writeFile(path string, code string) { + err := os.WriteFile(path, []byte(code), 0600) + if err != nil { + panic(fmt.Errorf("error during test setup: %s", err.Error())) + } +} From d1b5af7ab3ee72935eddfe1ec23dde046b8a7396 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Wed, 8 Dec 2021 15:56:08 +0300 Subject: [PATCH 09/12] unit tests fixes --- .../internal/code_processing/code_processing.go | 12 ++++++------ .../internal/preparators/go_preparators_test.go | 3 +-- .../setup_tools/builder/setup_builder_test.go | 5 +++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index a429ec53734c..afad2bdfbb47 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -117,7 +117,11 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl switch sdkEnv.ApacheBeamSdk { case pb.Sdk_SDK_JAVA, pb.Sdk_SDK_GO: - if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && !isUnitTest { + if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && isUnitTest { + if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { + return + } + } else { // Compile logger.Infof("%s: Compile() ...\n", pipelineId) compileCmd := executor.Compile(ctxWithTimeout) @@ -136,11 +140,8 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl if err := processCompileSuccess(ctxWithTimeout, compileOutput.Bytes(), pipelineId, cacheService); err != nil { return } - } else { - if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { - return - } } + case pb.Sdk_SDK_PYTHON: if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { return @@ -261,7 +262,6 @@ func runCmdWithOutput(cmd *exec.Cmd, stdOutput io.Writer, stdError *bytes.Buffer cmd.Stdout = stdOutput cmd.Stderr = stdError go func(cmd *exec.Cmd, successChannel chan bool, errChannel chan error) { - fmt.Println("!!! cmd: " + cmd.String()) err := cmd.Run() if err != nil { errChannel <- err diff --git a/playground/backend/internal/preparators/go_preparators_test.go b/playground/backend/internal/preparators/go_preparators_test.go index e72a7c158c0e..0505a7528b49 100644 --- a/playground/backend/internal/preparators/go_preparators_test.go +++ b/playground/backend/internal/preparators/go_preparators_test.go @@ -89,7 +89,6 @@ func getPreparedArgs(args ...interface{}) []interface{} { } func TestGetGoPreparators(t *testing.T) { - expectedPreparator := Preparator{Prepare: formatCode, Args: nil} type args struct { filePath string } @@ -102,7 +101,7 @@ func TestGetGoPreparators(t *testing.T) { // getting the expected preparator name: "get expected preparator", args: args{filePath: ""}, - want: &[]Preparator{expectedPreparator}, + want: &[]Preparator{{Prepare: formatCode, Args: nil}, {Prepare: changeFileName, Args: nil}}, }, } for _, tt := range tests { 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 fd48ff5705b5..1dcbf4a5ccc7 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -78,8 +78,9 @@ func TestSetupExecutor(t *testing.T) { ExecutorBuilder type args struct { - lc *fs_tool.LifeCycle - pipelineOptions stringsdkEnv *environment.BeamEnvs + lc *fs_tool.LifeCycle + pipelineOptions string + sdkEnv *environment.BeamEnvs } tests := []struct { name string From 82b8c0ca2a39810c49d30c6578aa712169fd5714 Mon Sep 17 00:00:00 2001 From: "daria.malkova" Date: Thu, 9 Dec 2021 11:14:38 +0300 Subject: [PATCH 10/12] Update playground/backend/internal/executors/executor_test.go Co-authored-by: Aydar Zainutdinov --- playground/backend/internal/executors/executor_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index b6a9b5a5c7b8..c52c4bd435d4 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -51,7 +51,8 @@ func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath } builder := NewExecutorBuilder(). WithExecutableFileName("HelloWorld"). - WithWorkingDir(workingDir).WithValidator(). + WithWorkingDir(workingDir). + WithValidator(). WithSdkValidators(validatorsFuncs). WithPreparator(). WithSdkPreparators(preparatorsFuncs). From c5c6b563fc4d623e51b6abb60b5de6c75a91f296 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Thu, 9 Dec 2021 12:38:33 +0300 Subject: [PATCH 11/12] code processing small refactoring --- .../code_processing/code_processing.go | 48 ++++++++----------- 1 file changed, 20 insertions(+), 28 deletions(-) diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index afad2bdfbb47..fb7cf55b0ee7 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -115,37 +115,29 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl return } - switch sdkEnv.ApacheBeamSdk { - case pb.Sdk_SDK_JAVA, pb.Sdk_SDK_GO: - if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && isUnitTest { - if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { - return - } - } else { - // Compile - logger.Infof("%s: Compile() ...\n", pipelineId) - compileCmd := executor.Compile(ctxWithTimeout) - var compileError bytes.Buffer - var compileOutput bytes.Buffer - runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) - - ok, err = processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel) - if err != nil { - return - } - if !ok { - _ = processCompileError(ctxWithTimeout, errorChannel, compileError.Bytes(), pipelineId, cacheService) - return - } - if err := processCompileSuccess(ctxWithTimeout, compileOutput.Bytes(), pipelineId, cacheService); err != nil { - return - } - } - - case pb.Sdk_SDK_PYTHON: + if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_PYTHON || (sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO && isUnitTest) { if err := processCompileSuccess(ctxWithTimeout, []byte(""), pipelineId, cacheService); err != nil { return } + } else { // in case of Java, Go (not unit test), Scala - need compile step + // Compile + logger.Infof("%s: Compile() ...\n", pipelineId) + compileCmd := executor.Compile(ctxWithTimeout) + var compileError bytes.Buffer + var compileOutput bytes.Buffer + runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) + + ok, err = processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel) + if err != nil { + return + } + if !ok { + _ = processCompileError(ctxWithTimeout, errorChannel, compileError.Bytes(), pipelineId, cacheService) + return + } + if err := processCompileSuccess(ctxWithTimeout, compileOutput.Bytes(), pipelineId, cacheService); err != nil { + return + } } // Run From 6ca5b2eae03e49f7fd40485afd5687bf9926d46c Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Fri, 10 Dec 2021 18:42:10 +0300 Subject: [PATCH 12/12] Add processing of an error --- playground/backend/internal/code_processing/code_processing.go | 3 +++ playground/backend/internal/streaming/run_output_writer.go | 3 +++ 2 files changed, 6 insertions(+) diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index fb7cf55b0ee7..e5bbb513ef6f 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -158,6 +158,9 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl return } if !ok { + if runOutput.Error != nil { + runError.Write([]byte(runOutput.Error.Error())) + } _ = processRunError(ctxWithTimeout, errorChannel, runError.Bytes(), pipelineId, cacheService, stopReadLogsChannel, finishReadLogsChannel) return } diff --git a/playground/backend/internal/streaming/run_output_writer.go b/playground/backend/internal/streaming/run_output_writer.go index 94bd0a119f8a..36b129371733 100644 --- a/playground/backend/internal/streaming/run_output_writer.go +++ b/playground/backend/internal/streaming/run_output_writer.go @@ -27,6 +27,7 @@ type RunOutputWriter struct { Ctx context.Context CacheService cache.Cache PipelineId uuid.UUID + Error error } // Write writes len(p) bytes from p to cache with cache.RunOutput subKey. @@ -45,6 +46,7 @@ func (row *RunOutputWriter) Write(p []byte) (int, error) { prevOutput, err := row.CacheService.GetValue(row.Ctx, row.PipelineId, cache.RunOutput) if err != nil { + row.Error = err return 0, err } @@ -54,6 +56,7 @@ func (row *RunOutputWriter) Write(p []byte) (int, error) { // set new cache value err = row.CacheService.SetValue(row.Ctx, row.PipelineId, cache.RunOutput, str) if err != nil { + row.Error = err return 0, err } return len(p), nil