You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2023/01/04 14:02:52 UTC

[GitHub] [beam] TSultanov opened a new pull request, #24874: [Playground] Robust way for looking up Java examples entry point

TSultanov opened a new pull request, #24874:
URL: https://github.com/apache/beam/pull/24874

   To run a Java application we need to know name of a `.class` file with the `main()` method. This change introduces logic to reliably locate such file using `javap` tool.
   
   We cannot rely on `IsMain` metadata argument, available for multi-file examples, as we have no way of knowing names of the generated `.class` files without parsing the source code to a significant degree, as Java compiler produces one such file for each class definition in the source `.java` file with names corresponding to the name of the class defined in the source code.
   We also cannot rely on the convention that the Java source file name should be the same as the name of a public class defined in that `.java` file because [we trim all](https://github.com/apache/beam/blob/0b8f0b4db7a0de4977e30bcfeb50b5c14c7c1572/playground/backend/internal/preparers/java_preparers.go#L61) `public` keywords from class definitions.
   
   resolves #24860
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [x] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #24874: [Playground] Robust way for looking up Java examples entry point

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #24874:
URL: https://github.com/apache/beam/pull/24874#issuecomment-1380371922

   Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment `assign set of reviewers`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damccorm merged pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "damccorm (via GitHub)" <gi...@apache.org>.
damccorm merged PR #24874:
URL: https://github.com/apache/beam/pull/24874


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damondouglas commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "damondouglas (via GitHub)" <gi...@apache.org>.
damondouglas commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1093449668


##########
playground/backend/internal/code_processing/code_processing.go:
##########
@@ -172,7 +172,11 @@ func compileStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.L
 			return nil
 		}
 	} else { // in case of Java, Go (not unit test), Scala - need compile step
-		executorBuilder := builder.Compiler(paths, sdkEnv)
+		executorBuilder, err := builder.Compiler(paths, sdkEnv)

Review Comment:
   The fact that we see logging of an error and not returning it makes this method difficult to test.
   The `compileStep` method signature should not have a `pipelineLifeCycleCtx context.Context` at the end. Typically context.Context is the first method argument.  I realize this is not part of the PR but it is making updates to this code more challenging.  Perhaps consider refactoring the method to do something like https://go.dev/play/p/f4fn1Scx1vE.



##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +58,102 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
+		filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+		content, err := os.ReadFile(filePath)
 		if err != nil {
 			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
 			break
 		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		ext := filepath.Ext(entry.Name())
+		filename := strings.TrimSuffix(entry.Name(), ext)
+		sdk := utils.ToSDKFromExt(ext)
 
 		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
+			logger.Error("invalid file extension")
 			break
 		}
 
-		if utils.IsFileMain(string(content), sdk) {
-			return strings.Split(entry.Name(), ".")[0], nil
+		switch ext {
+		case javaCompiledFileExtension:
+			isMain, err := isMainClass(executableFileFolderPath, filename)
+			if err != nil {
+				return "", err
+			}
+			if isMain {
+				logger.Infof("executableName(): main file is %s", filename)
+				return filename, nil
+			}
+		default:
+			if utils.IsFileMain(string(content), sdk) {
+				return filename, nil
+			}
 		}
+
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+func testExecutableName(executableFileFolderPath string) (string, error) {

Review Comment:
   I was confused by the name of this method and its purpose.  What about `findExecutableName`?  Also, what if finding this file takes a long time?  Should there be a context.WithTimeout?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1148962463


##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +59,112 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
-		if err != nil {
-			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
-			break
-		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+			content, err := os.ReadFile(filePath)
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
+			sdk := utils.ToSDKFromExt(ext)
+
+			if sdk == pb.Sdk_SDK_UNSPECIFIED {
+				logger.Error("invalid file extension")
+				break
+			}
 
-		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
-			break
+			switch ext {
+			case javaCompiledFileExtension:
+				isMain, err := isMainClass(ctx, executableFileFolderPath, filename)
+				if err != nil {
+					return "", err
+				}
+				if isMain {
+					logger.Infof("executableName(): main file is %s", filename)
+					return filename, nil
+				}
+			default:
+				if utils.IsFileMain(string(content), sdk) {
+					return filename, nil
+				}
+			}
 		}
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+// findTestExecutableName returns name of .class file which has JUnit tests
+func findTestExecutableName(ctx context.Context, executableFileFolderPath string) (string, error) {
+	dirEntries, err := os.ReadDir(executableFileFolderPath)
+	if err != nil {
+		return "", err
+	}
+	if len(dirEntries) < 1 {
+		return "", errors.New("number of executable files should be at least one")
+	}
+
+	if len(dirEntries) == 1 {
+		return utils.TrimExtension(dirEntries[0].Name()), nil
+	}
+
+	for _, entry := range dirEntries {
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			if err != nil {

Review Comment:
   Removed the condition



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1094375294


##########
playground/backend/internal/code_processing/code_processing.go:
##########
@@ -172,7 +172,11 @@ func compileStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.L
 			return nil
 		}
 	} else { // in case of Java, Go (not unit test), Scala - need compile step
-		executorBuilder := builder.Compiler(paths, sdkEnv)
+		executorBuilder, err := builder.Compiler(paths, sdkEnv)

Review Comment:
   There is a problem with this method that it takes two contexts, this is something which will require careful consideration on why this was done this way and how it can be rectified.
   Also refactoring the call chain leading to this function so it will be able to return the error also will require some thought and it goes beyond the scope of this PR. 
   Filed a task to resolve this separately: https://github.com/apache/beam/issues/25277



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damondouglas commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by GitBox <gi...@apache.org>.
damondouglas commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1082831517


##########
playground/backend/internal/setup_tools/builder/setup_builder.go:
##########
@@ -185,7 +196,7 @@ func replaceLogPlaceholder(paths *fs_tool.LifeCyclePaths, executorConfig *enviro
 }
 
 // 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]
+func GetFilesFromFolder(folderAbsolutePath string, extension string) []string {
+	files, _ := filepath.Glob(fmt.Sprintf("%s/*%s", folderAbsolutePath, extension))

Review Comment:
   What if `filepath.Glob` returns an error?



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -144,27 +171,78 @@ func Test_executableName(t *testing.T) {
 			name: "Multiple files where one of them is main",
 			prepare: func() {
 				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
-				secondaryFilePath := filepath.Join(compiled, "temp.scala")
-				err := os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
+				primaryFilePath := filepath.Join(compiled, "main.scala")
+				err := os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
 				if err != nil {
 					panic(err)
 				}
-				primaryFilePath := filepath.Join(compiled, "main.scala")
-				err = os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
+				secondaryFilePath := filepath.Join(compiled, "temp.scala")
+				err = os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
 				if err != nil {
 					panic(err)
 				}
 			},
+			cleanup: cleanupFunc,
 			args: args{
 				executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"),
 			},
 			want:    "main",
 			wantErr: false,
 		},
+		{
+			// Test case with calling sourceFileName method with multiple files where one of them is a .class file
+			// with main() method
+			// As a result, want to receive a name that should be executed
+			name: "Multiple Java class files where one of them contains main",
+			prepare: func() {
+				testdataPath := "java_testdata"
+				dirEntries, err := os.ReadDir(testdataPath)
+				if err != nil {
+					panic(err)

Review Comment:
   Return error, and fail test as a result of the error.



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -144,27 +171,78 @@ func Test_executableName(t *testing.T) {
 			name: "Multiple files where one of them is main",
 			prepare: func() {
 				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
-				secondaryFilePath := filepath.Join(compiled, "temp.scala")
-				err := os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
+				primaryFilePath := filepath.Join(compiled, "main.scala")
+				err := os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
 				if err != nil {
 					panic(err)
 				}
-				primaryFilePath := filepath.Join(compiled, "main.scala")
-				err = os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
+				secondaryFilePath := filepath.Join(compiled, "temp.scala")
+				err = os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
 				if err != nil {
 					panic(err)
 				}
 			},
+			cleanup: cleanupFunc,
 			args: args{
 				executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"),
 			},
 			want:    "main",
 			wantErr: false,
 		},
+		{
+			// Test case with calling sourceFileName method with multiple files where one of them is a .class file
+			// with main() method
+			// As a result, want to receive a name that should be executed
+			name: "Multiple Java class files where one of them contains main",
+			prepare: func() {
+				testdataPath := "java_testdata"
+				dirEntries, err := os.ReadDir(testdataPath)
+				if err != nil {
+					panic(err)
+				}
+
+				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+
+				for _, entry := range dirEntries {
+					src := filepath.Join(testdataPath, entry.Name())
+					dst := filepath.Join(compiled, entry.Name())
+					err = os.Link(src, dst)
+					if err != nil {
+						panic(err)

Review Comment:
   Instead return error and fail test as a result of the error.



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)
+		}
+
+		for _, entry := range dirEntries {
+			err := os.Remove(filepath.Join(compiled, entry.Name()))
+			if err != nil {
+				panic(err)
+			}
+		}
+	}
 
 	type args struct {
 		executableFolder string
 	}
 	tests := []struct {
 		name    string
 		prepare func()
+		cleanup func()
 		args    args
 		want    string
 		wantErr bool

Review Comment:
   I didn't see any tests where `wantErr=true`



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)

Review Comment:
   Return error, and fail test as a result of the error.



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)
+		}
+
+		for _, entry := range dirEntries {
+			err := os.Remove(filepath.Join(compiled, entry.Name()))
+			if err != nil {
+				panic(err)

Review Comment:
   Return error, and fail test as a result of the error.



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)

Review Comment:
   Return error, and fail test as a result of the error.



##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -82,87 +82,97 @@ func (b *ExecutorBuilder) WithTestRunner() *UnitTestExecutorBuilder {
 	return &UnitTestExecutorBuilder{*b}
 }
 
-//WithCommand adds compile command to executor
+// WithCommand adds compile command to executor
 func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandName = compileCmd
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithArgs adds compile args to executor
 func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandArgs = compileArgs
 	})
 	return b
 }
 
-//WithFileName adds file name to executor
+// WithFileName adds file name to executor
 func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	return b.WithFileNames([]string{fileName})
+}
+
+// WithFileNames adds file names to executor
+func (b *CompileBuilder) WithFileNames(fileNames []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.compileArgs.fileName = fileName
+		e.compileArgs.fileNames = fileNames
 	})
 	return b
 }
 
-//WithExecutableFileName adds file name to executor
+// WithExecutableFileName adds file name to executor
 func (b *RunBuilder) WithExecutableFileName(name string) *RunBuilder {
+	return b.WithExecutableFileNames([]string{name})
+}
+
+// WithExecutableFileNames adds file name to executor
+func (b *RunBuilder) WithExecutableFileNames(names []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.runArgs.fileName = name
+		e.runArgs.fileNames = names
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithCommand adds run command to executor
 func (b *RunBuilder) WithCommand(runCmd string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandName = runCmd
 	})
 	return b
 }
 
-//WithArgs adds run args to executor
+// WithArgs adds run args to executor
 func (b *RunBuilder) WithArgs(runArgs []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandArgs = runArgs
 	})
 	return b
 }
 
-//WithGraphOutput adds the need of graph output to executor
+// WithGraphOutput adds the need of graph output to executor
 func (b *RunBuilder) WithGraphOutput() *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		//todo

Review Comment:
   What is the work of the `//todo`?  Could we instead refer to a GitHub issue?



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)

Review Comment:
   Return error, and fail test as a result of the error.



##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -82,87 +82,97 @@ func (b *ExecutorBuilder) WithTestRunner() *UnitTestExecutorBuilder {
 	return &UnitTestExecutorBuilder{*b}
 }
 
-//WithCommand adds compile command to executor
+// WithCommand adds compile command to executor
 func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandName = compileCmd
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithArgs adds compile args to executor
 func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandArgs = compileArgs
 	})
 	return b
 }
 
-//WithFileName adds file name to executor
+// WithFileName adds file name to executor
 func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	return b.WithFileNames([]string{fileName})
+}
+
+// WithFileNames adds file names to executor
+func (b *CompileBuilder) WithFileNames(fileNames []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.compileArgs.fileName = fileName
+		e.compileArgs.fileNames = fileNames
 	})
 	return b
 }

Review Comment:
   What about just one func that handles both cases?:
   ```
   func (e *CompileBuilder) WithFileNames(fileNames ...string) *CompileBuilder {
   	e.compileArgs.fileNames = fileNames
           return e
   }
   ```
   
   https://go.dev/play/p/IQ59C-t8wNA



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)
+		}
+
+		for _, entry := range dirEntries {
+			err := os.Remove(filepath.Join(compiled, entry.Name()))
+			if err != nil {
+				panic(err)

Review Comment:
   Return error, fail test with error.



##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -82,87 +82,97 @@ func (b *ExecutorBuilder) WithTestRunner() *UnitTestExecutorBuilder {
 	return &UnitTestExecutorBuilder{*b}
 }
 
-//WithCommand adds compile command to executor
+// WithCommand adds compile command to executor
 func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandName = compileCmd
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithArgs adds compile args to executor
 func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandArgs = compileArgs
 	})
 	return b
 }
 
-//WithFileName adds file name to executor
+// WithFileName adds file name to executor
 func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	return b.WithFileNames([]string{fileName})
+}
+
+// WithFileNames adds file names to executor
+func (b *CompileBuilder) WithFileNames(fileNames []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.compileArgs.fileName = fileName
+		e.compileArgs.fileNames = fileNames
 	})
 	return b
 }
 
-//WithExecutableFileName adds file name to executor
+// WithExecutableFileName adds file name to executor
 func (b *RunBuilder) WithExecutableFileName(name string) *RunBuilder {
+	return b.WithExecutableFileNames([]string{name})
+}
+
+// WithExecutableFileNames adds file name to executor
+func (b *RunBuilder) WithExecutableFileNames(names []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.runArgs.fileName = name
+		e.runArgs.fileNames = names
 	})
 	return b
 }
 

Review Comment:
   Same as above comment `WithExecutableFileNames(names ...string) *RunBuilder`



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)

Review Comment:
   Return error, fail test with error.



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)

Review Comment:
   Return error, fail test with error.



##########
playground/backend/internal/fs_tool/java_testdata/Foo.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+class Bar {
+    public static void bar() {
+        System.out.println("Hello");
+    }
+}
+
+public class Foo {
+    public static void main(java.lang.String[] args) {
+        Bar.bar();
+    }

Review Comment:
   Do we have a case that doesn't have main?  Or it has main but no `String[] args`?  What about a Java source that has syntax or possible compile errors?  Additionally what about a source that has runtime errors?



##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -178,47 +188,47 @@ func (b *UnitTestExecutorBuilder) WithWorkingDir(dir string) *UnitTestExecutorBu
 	return b
 }
 
-//WithGraphOutput adds the need of graph output to executor
+// WithGraphOutput adds the need of graph output to executor
 func (b *UnitTestExecutorBuilder) WithGraphOutput() *UnitTestExecutorBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		//todo

Review Comment:
   What is the work remaining of this `//todo`?  Should it refer instead to a GitHub issue?



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)

Review Comment:
   Return error, fail test with error.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1084005503


##########
playground/backend/internal/fs_tool/java_testdata/Foo.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+class Bar {
+    public static void bar() {
+        System.out.println("Hello");
+    }
+}
+
+public class Foo {
+    public static void main(java.lang.String[] args) {
+        Bar.bar();
+    }

Review Comment:
   Added cases for several permutations of `main()` method.
   
   Compilation and runtime errors are handled by code here: https://github.com/apache/beam/blob/master/playground/backend/internal/code_processing/code_processing_test.go#L191



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1094359032


##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +58,102 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
+		filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+		content, err := os.ReadFile(filePath)
 		if err != nil {
 			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
 			break
 		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		ext := filepath.Ext(entry.Name())
+		filename := strings.TrimSuffix(entry.Name(), ext)
+		sdk := utils.ToSDKFromExt(ext)
 
 		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
+			logger.Error("invalid file extension")
 			break
 		}
 
-		if utils.IsFileMain(string(content), sdk) {
-			return strings.Split(entry.Name(), ".")[0], nil
+		switch ext {
+		case javaCompiledFileExtension:
+			isMain, err := isMainClass(executableFileFolderPath, filename)
+			if err != nil {
+				return "", err
+			}
+			if isMain {
+				logger.Infof("executableName(): main file is %s", filename)
+				return filename, nil
+			}
+		default:
+			if utils.IsFileMain(string(content), sdk) {
+				return filename, nil
+			}
 		}
+
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+func testExecutableName(executableFileFolderPath string) (string, error) {

Review Comment:
   Improved naming and passed context into the functions to support timeouts



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1148954284


##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -142,36 +184,159 @@ func Test_executableName(t *testing.T) {
 			// Test case with calling sourceFileName method with multiple files where one of them is main
 			// As a result, want to receive a name that should be executed
 			name: "Multiple files where one of them is main",
-			prepare: func() {
+			prepare: func() error {
 				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
-				secondaryFilePath := filepath.Join(compiled, "temp.scala")
-				err := os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
+				primaryFilePath := filepath.Join(compiled, "main.scala")
+				err := os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
-				primaryFilePath := filepath.Join(compiled, "main.scala")
-				err = os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
+				secondaryFilePath := filepath.Join(compiled, "temp.scala")
+				err = os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
+				return nil
 			},
+			cleanup: cleanupFunc,
 			args: args{
 				executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"),
 			},
 			want:    "main",
 			wantErr: false,
 		},
+		{
+			// Test case with calling sourceFileName method with multiple files where one of them is a .class file
+			// with main() method
+			// As a result, want to receive a name that should be executed
+			name: "Multiple Java class files where one of them contains main",
+			prepare: func() error {
+				testdataPath := "java_testdata"
+				sourceFile := filepath.Join(testdataPath, "HasMainTest1.java")
+
+				err := compileJavaFiles(sourceFile)

Review Comment:
   @damccorm the `.java` file we are passing here contains multiple classes, so Java compiler will produce several `.class` files.
   Yes, this is confusing, and this is the reason why we need to have this convoluted method of finding what .class file is executable as there is no direct correspondence between input and output file names in Java compiler.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1085254897


##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -82,87 +82,97 @@ func (b *ExecutorBuilder) WithTestRunner() *UnitTestExecutorBuilder {
 	return &UnitTestExecutorBuilder{*b}
 }
 
-//WithCommand adds compile command to executor
+// WithCommand adds compile command to executor
 func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandName = compileCmd
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithArgs adds compile args to executor
 func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandArgs = compileArgs
 	})
 	return b
 }
 
-//WithFileName adds file name to executor
+// WithFileName adds file name to executor
 func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	return b.WithFileNames([]string{fileName})
+}
+
+// WithFileNames adds file names to executor
+func (b *CompileBuilder) WithFileNames(fileNames []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.compileArgs.fileName = fileName
+		e.compileArgs.fileNames = fileNames
 	})
 	return b
 }
 
-//WithExecutableFileName adds file name to executor
+// WithExecutableFileName adds file name to executor
 func (b *RunBuilder) WithExecutableFileName(name string) *RunBuilder {
+	return b.WithExecutableFileNames([]string{name})
+}
+
+// WithExecutableFileNames adds file name to executor
+func (b *RunBuilder) WithExecutableFileNames(names []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.runArgs.fileName = name
+		e.runArgs.fileNames = names
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithCommand adds run command to executor
 func (b *RunBuilder) WithCommand(runCmd string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandName = runCmd
 	})
 	return b
 }
 
-//WithArgs adds run args to executor
+// WithArgs adds run args to executor
 func (b *RunBuilder) WithArgs(runArgs []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandArgs = runArgs
 	})
 	return b
 }
 
-//WithGraphOutput adds the need of graph output to executor
+// WithGraphOutput adds the need of graph output to executor
 func (b *RunBuilder) WithGraphOutput() *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		//todo

Review Comment:
   This functions are not used anywhere and we weren't able to find any tasks referring to them. Removed both of them.



##########
playground/backend/internal/executors/executor_builder.go:
##########
@@ -82,87 +82,97 @@ func (b *ExecutorBuilder) WithTestRunner() *UnitTestExecutorBuilder {
 	return &UnitTestExecutorBuilder{*b}
 }
 
-//WithCommand adds compile command to executor
+// WithCommand adds compile command to executor
 func (b *CompileBuilder) WithCommand(compileCmd string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandName = compileCmd
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithArgs adds compile args to executor
 func (b *CompileBuilder) WithArgs(compileArgs []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.compileArgs.commandArgs = compileArgs
 	})
 	return b
 }
 
-//WithFileName adds file name to executor
+// WithFileName adds file name to executor
 func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	return b.WithFileNames([]string{fileName})
+}
+
+// WithFileNames adds file names to executor
+func (b *CompileBuilder) WithFileNames(fileNames []string) *CompileBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.compileArgs.fileName = fileName
+		e.compileArgs.fileNames = fileNames
 	})
 	return b
 }
 
-//WithExecutableFileName adds file name to executor
+// WithExecutableFileName adds file name to executor
 func (b *RunBuilder) WithExecutableFileName(name string) *RunBuilder {
+	return b.WithExecutableFileNames([]string{name})
+}
+
+// WithExecutableFileNames adds file name to executor
+func (b *RunBuilder) WithExecutableFileNames(names []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
-		e.runArgs.fileName = name
+		e.runArgs.fileNames = names
 	})
 	return b
 }
 
-//WithWorkingDir adds dir path to executor
+// 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
+// WithCommand adds run command to executor
 func (b *RunBuilder) WithCommand(runCmd string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandName = runCmd
 	})
 	return b
 }
 
-//WithArgs adds run args to executor
+// WithArgs adds run args to executor
 func (b *RunBuilder) WithArgs(runArgs []string) *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		e.runArgs.commandArgs = runArgs
 	})
 	return b
 }
 
-//WithGraphOutput adds the need of graph output to executor
+// WithGraphOutput adds the need of graph output to executor
 func (b *RunBuilder) WithGraphOutput() *RunBuilder {
 	b.actions = append(b.actions, func(e *Executor) {
 		//todo

Review Comment:
   These functions are not used anywhere and we weren't able to find any tasks referring to them. Removed both of them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1083933022


##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -85,15 +85,39 @@ func Test_executableName(t *testing.T) {
 	workDir := "workingDir"
 	preparedPipelinesFolder := filepath.Join(workDir, pipelinesFolder)
 	lc := newJavaLifeCycle(pipelineId, preparedPipelinesFolder)
-	lc.CreateFolders()
-	defer os.RemoveAll(workDir)
+	err := lc.CreateFolders()
+	if err != nil {
+		panic(err)
+	}
+	defer func() {
+		err := os.RemoveAll(workDir)
+		if err != nil {
+			panic(err)
+		}
+	}()
+
+	cleanupFunc := func() {
+		compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
+		dirEntries, err := os.ReadDir(compiled)
+		if err != nil {
+			panic(err)
+		}
+
+		for _, entry := range dirEntries {
+			err := os.Remove(filepath.Join(compiled, entry.Name()))
+			if err != nil {
+				panic(err)
+			}
+		}
+	}
 
 	type args struct {
 		executableFolder string
 	}
 	tests := []struct {
 		name    string
 		prepare func()
+		cleanup func()
 		args    args
 		want    string
 		wantErr bool

Review Comment:
   There are couple:
   https://github.com/apache/beam/blob/0b3b2b6e3595d69bb9f8b4cf38975c8b97f1847b/playground/backend/internal/fs_tool/java_fs_test.go#L135
   https://github.com/apache/beam/blob/0b3b2b6e3595d69bb9f8b4cf38975c8b97f1847b/playground/backend/internal/fs_tool/java_fs_test.go#L166



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TSultanov commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "TSultanov (via GitHub)" <gi...@apache.org>.
TSultanov commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1084005503


##########
playground/backend/internal/fs_tool/java_testdata/Foo.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+class Bar {
+    public static void bar() {
+        System.out.println("Hello");
+    }
+}
+
+public class Foo {
+    public static void main(java.lang.String[] args) {
+        Bar.bar();
+    }

Review Comment:
   Added cases for several permutations of `main()` method. Made inability to find class with `main()` method into an explicit error.
   
   Compilation and runtime errors are handled by code here: https://github.com/apache/beam/blob/master/playground/backend/internal/code_processing/code_processing_test.go#L191



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damccorm commented on a diff in pull request #24874: [Playground] Implement Java & Go multifile examples execution

Posted by "damccorm (via GitHub)" <gi...@apache.org>.
damccorm commented on code in PR #24874:
URL: https://github.com/apache/beam/pull/24874#discussion_r1147997198


##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -30,19 +33,23 @@ import (
 )
 
 const (
-	JavaSourceFileExtension   = ".java"
-	javaCompiledFileExtension = ".class"
+	JavaSourceFileExtension            = ".java"
+	javaCompiledFileExtension          = ".class"
+	javaEntryPointFullName             = "public static void main(java.lang.String[])"
+	javaDecompilerCommand              = "javap"
+	juintRunWithTestAnnotationConstant = "Lorg/junit/runner/RunWith;"
 )
 
 // newJavaLifeCycle creates LifeCycle with java SDK environment.
 func newJavaLifeCycle(pipelineId uuid.UUID, pipelinesFolder string) *LifeCycle {
 	javaLifeCycle := newCompilingLifeCycle(pipelineId, pipelinesFolder, JavaSourceFileExtension, javaCompiledFileExtension)
-	javaLifeCycle.Paths.ExecutableName = executableName
+	javaLifeCycle.Paths.FindExecutableName = findExecutableName
+	javaLifeCycle.Paths.FindTestExecutableName = findTestExecutableName
 	return javaLifeCycle
 }
 
-// executableName returns name that should be executed (HelloWorld for HelloWorld.class for java SDK)
-func executableName(executableFileFolderPath string) (string, error) {
+// findExecutableName returns name of .class file which has main() method

Review Comment:
   ```suggestion
   // findExecutableName returns name of the .class file which has main() method
   ```



##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +59,112 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
-		if err != nil {
-			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
-			break
-		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+			content, err := os.ReadFile(filePath)
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
+			sdk := utils.ToSDKFromExt(ext)
+
+			if sdk == pb.Sdk_SDK_UNSPECIFIED {
+				logger.Error("invalid file extension")
+				break
+			}
 
-		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
-			break
+			switch ext {
+			case javaCompiledFileExtension:
+				isMain, err := isMainClass(ctx, executableFileFolderPath, filename)
+				if err != nil {
+					return "", err
+				}
+				if isMain {
+					logger.Infof("executableName(): main file is %s", filename)
+					return filename, nil
+				}
+			default:
+				if utils.IsFileMain(string(content), sdk) {
+					return filename, nil
+				}
+			}
 		}
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+// findTestExecutableName returns name of .class file which has JUnit tests
+func findTestExecutableName(ctx context.Context, executableFileFolderPath string) (string, error) {
+	dirEntries, err := os.ReadDir(executableFileFolderPath)
+	if err != nil {
+		return "", err
+	}
+	if len(dirEntries) < 1 {
+		return "", errors.New("number of executable files should be at least one")
+	}
+
+	if len(dirEntries) == 1 {
+		return utils.TrimExtension(dirEntries[0].Name()), nil
+	}
+
+	for _, entry := range dirEntries {
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			if err != nil {

Review Comment:
   When would this condition trigger?



##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +59,112 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
-		if err != nil {
-			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
-			break
-		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+			content, err := os.ReadFile(filePath)
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
+			sdk := utils.ToSDKFromExt(ext)
+
+			if sdk == pb.Sdk_SDK_UNSPECIFIED {
+				logger.Error("invalid file extension")
+				break
+			}
 
-		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
-			break
+			switch ext {
+			case javaCompiledFileExtension:
+				isMain, err := isMainClass(ctx, executableFileFolderPath, filename)
+				if err != nil {
+					return "", err

Review Comment:
   Should we just log and break here like we do elsewhere in this function?



##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +59,112 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
-		if err != nil {
-			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
-			break
-		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+			content, err := os.ReadFile(filePath)
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
+			sdk := utils.ToSDKFromExt(ext)
+
+			if sdk == pb.Sdk_SDK_UNSPECIFIED {
+				logger.Error("invalid file extension")
+				break
+			}
 
-		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
-			break
+			switch ext {
+			case javaCompiledFileExtension:
+				isMain, err := isMainClass(ctx, executableFileFolderPath, filename)
+				if err != nil {
+					return "", err
+				}
+				if isMain {
+					logger.Infof("executableName(): main file is %s", filename)
+					return filename, nil
+				}
+			default:
+				if utils.IsFileMain(string(content), sdk) {
+					return filename, nil
+				}
+			}
 		}
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+// findTestExecutableName returns name of .class file which has JUnit tests
+func findTestExecutableName(ctx context.Context, executableFileFolderPath string) (string, error) {
+	dirEntries, err := os.ReadDir(executableFileFolderPath)
+	if err != nil {
+		return "", err
+	}
+	if len(dirEntries) < 1 {
+		return "", errors.New("number of executable files should be at least one")
+	}
+
+	if len(dirEntries) == 1 {
+		return utils.TrimExtension(dirEntries[0].Name()), nil
+	}
+
+	for _, entry := range dirEntries {
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			if err != nil {

Review Comment:
   I think never



##########
playground/backend/internal/fs_tool/java_fs.go:
##########
@@ -52,27 +59,112 @@ func executableName(executableFileFolderPath string) (string, error) {
 	}
 
 	if len(dirEntries) == 1 {
-		return strings.Split(dirEntries[0].Name(), ".")[0], nil
+		return utils.TrimExtension(dirEntries[0].Name()), nil
 	}
 
 	for _, entry := range dirEntries {
-		content, err := ioutil.ReadFile(fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name()))
-		if err != nil {
-			logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
-			break
-		}
-		ext := strings.Split(entry.Name(), ".")[1]
-		sdk := utils.ToSDKFromExt("." + ext)
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			filePath := fmt.Sprintf("%s/%s", executableFileFolderPath, entry.Name())
+			content, err := os.ReadFile(filePath)
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
+			sdk := utils.ToSDKFromExt(ext)
+
+			if sdk == pb.Sdk_SDK_UNSPECIFIED {
+				logger.Error("invalid file extension")
+				break
+			}
 
-		if sdk == pb.Sdk_SDK_UNSPECIFIED {
-			logger.Error("invalid a file extension")
-			break
+			switch ext {
+			case javaCompiledFileExtension:
+				isMain, err := isMainClass(ctx, executableFileFolderPath, filename)
+				if err != nil {
+					return "", err
+				}
+				if isMain {
+					logger.Infof("executableName(): main file is %s", filename)
+					return filename, nil
+				}
+			default:
+				if utils.IsFileMain(string(content), sdk) {
+					return filename, nil
+				}
+			}
 		}
+	}
+
+	return "", errors.New("cannot find file with main() method")
+}
+
+// findTestExecutableName returns name of .class file which has JUnit tests
+func findTestExecutableName(ctx context.Context, executableFileFolderPath string) (string, error) {
+	dirEntries, err := os.ReadDir(executableFileFolderPath)
+	if err != nil {
+		return "", err
+	}
+	if len(dirEntries) < 1 {
+		return "", errors.New("number of executable files should be at least one")
+	}
+
+	if len(dirEntries) == 1 {
+		return utils.TrimExtension(dirEntries[0].Name()), nil
+	}
+
+	for _, entry := range dirEntries {
+		select {
+		case <-ctx.Done():
+			return "", ctx.Err()
+		default:
+			if err != nil {
+				logger.Error(fmt.Sprintf("error during file reading: %s", err.Error()))
+				break
+			}
+			ext := filepath.Ext(entry.Name())
+			filename := strings.TrimSuffix(entry.Name(), ext)
 
-		if utils.IsFileMain(string(content), sdk) {
-			return strings.Split(entry.Name(), ".")[0], nil
+			if ext == javaCompiledFileExtension {
+				isMain, err := isTestClass(ctx, executableFileFolderPath, filename)

Review Comment:
   Instead of `isMain`, should this be `isTest`?



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -142,36 +184,159 @@ func Test_executableName(t *testing.T) {
 			// Test case with calling sourceFileName method with multiple files where one of them is main
 			// As a result, want to receive a name that should be executed
 			name: "Multiple files where one of them is main",
-			prepare: func() {
+			prepare: func() error {
 				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
-				secondaryFilePath := filepath.Join(compiled, "temp.scala")
-				err := os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
+				primaryFilePath := filepath.Join(compiled, "main.scala")
+				err := os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
-				primaryFilePath := filepath.Join(compiled, "main.scala")
-				err = os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
+				secondaryFilePath := filepath.Join(compiled, "temp.scala")
+				err = os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
+				return nil
 			},
+			cleanup: cleanupFunc,
 			args: args{
 				executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"),
 			},
 			want:    "main",
 			wantErr: false,
 		},
+		{
+			// Test case with calling sourceFileName method with multiple files where one of them is a .class file
+			// with main() method
+			// As a result, want to receive a name that should be executed
+			name: "Multiple Java class files where one of them contains main",
+			prepare: func() error {
+				testdataPath := "java_testdata"
+				sourceFile := filepath.Join(testdataPath, "HasMainTest1.java")
+
+				err := compileJavaFiles(sourceFile)

Review Comment:
   The test description says "Multiple Java class files" - should we also be passing in other files here? Same question applies elsewhere



##########
playground/backend/internal/fs_tool/java_fs_test.go:
##########
@@ -142,36 +184,159 @@ func Test_executableName(t *testing.T) {
 			// Test case with calling sourceFileName method with multiple files where one of them is main
 			// As a result, want to receive a name that should be executed
 			name: "Multiple files where one of them is main",
-			prepare: func() {
+			prepare: func() error {
 				compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName)
-				secondaryFilePath := filepath.Join(compiled, "temp.scala")
-				err := os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
+				primaryFilePath := filepath.Join(compiled, "main.scala")
+				err := os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
-				primaryFilePath := filepath.Join(compiled, "main.scala")
-				err = os.WriteFile(primaryFilePath, []byte("object MinimalWordCount {def main(cmdlineArgs: Array[String]): Unit = {}}"), 0600)
+				secondaryFilePath := filepath.Join(compiled, "temp.scala")
+				err = os.WriteFile(secondaryFilePath, []byte("TEMP_DATA"), 0600)
 				if err != nil {
-					panic(err)
+					return err
 				}
+				return nil
 			},
+			cleanup: cleanupFunc,
 			args: args{
 				executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"),
 			},
 			want:    "main",
 			wantErr: false,
 		},
+		{
+			// Test case with calling sourceFileName method with multiple files where one of them is a .class file
+			// with main() method
+			// As a result, want to receive a name that should be executed
+			name: "Multiple Java class files where one of them contains main",
+			prepare: func() error {
+				testdataPath := "java_testdata"
+				sourceFile := filepath.Join(testdataPath, "HasMainTest1.java")
+
+				err := compileJavaFiles(sourceFile)

Review Comment:
   (e.g. maybe HasNoMain.java)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org