You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by pa...@apache.org on 2021/10/26 07:22:58 UTC

[beam] branch master updated: Merge pull request #15744 from [BEAM-13072][Playground] Executor builder

This is an automated email from the ASF dual-hosted git repository.

pabloem pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 49ae709  Merge pull request #15744 from [BEAM-13072][Playground] Executor builder
49ae709 is described below

commit 49ae709e95c6f5fcd88dd4aba86398c111180279
Author: daria.malkova <da...@akvelon.com>
AuthorDate: Tue Oct 26 10:21:54 2021 +0300

    Merge pull request #15744 from [BEAM-13072][Playground] Executor builder
    
    * Add environment_service.go and structures for beam sdk, network envs and application envs
    
    * Add executor config structure to beam environment structure
    Fix tests, add new
    
    * add not yet supported
    
    * add java configuration file
    
    * Add executor builder
    
    * resolve conflicts with master
    
    * adapt server to new envs
    
    * Bugfix for a moved file
    
    * test bugfix
    
    Co-authored-by: Ilya Kozyrev <il...@akvelon.com>
---
 playground/backend/internal/executors/executor.go  |  97 ++++------
 .../backend/internal/executors/executor_builder.go | 151 +++++++++++++++
 .../backend/internal/executors/executor_test.go    | 207 +++++++++++++++++++++
 .../backend/internal/executors/java_helper.go      |  69 -------
 .../backend/internal/executors/java_helper_test.go |  76 --------
 .../backend/internal/fs_tool/path_checker.go       |   5 +-
 .../go_helper.go => validators/java_validators.go} |  28 ++-
 .../go_helper.go => validators/validator.go}       |  16 +-
 8 files changed, 417 insertions(+), 232 deletions(-)

diff --git a/playground/backend/internal/executors/executor.go b/playground/backend/internal/executors/executor.go
index b85641b..f34895b 100644
--- a/playground/backend/internal/executors/executor.go
+++ b/playground/backend/internal/executors/executor.go
@@ -17,81 +17,52 @@
 package executors
 
 import (
-	pb "beam.apache.org/playground/backend/internal/api/v1"
-	"beam.apache.org/playground/backend/internal/fs_tool"
-	"fmt"
+	"beam.apache.org/playground/backend/internal/validators"
 	"os/exec"
 )
 
-type validatorWithArgs struct {
-	validator func(filePath string, args ...interface{}) error
-	args      []interface{}
+//CmdConfiguration for base cmd code execution
+type CmdConfiguration struct {
+	fileName    string
+	workingDir  string
+	commandName string
+	commandArgs []string
 }
 
-// Executor interface for all executors (Java/Python/Go/SCIO)
+// Executor struct for all executors (Java/Python/Go/SCIO)
 type Executor struct {
-	relativeFilePath string
-	absoulteFilePath string
-	dirPath          string
-	executableDir    string
-	validators       []validatorWithArgs
-	compileName      string
-	compileArgs      []string
-	runName          string
-	runArgs          []string
+	compileArgs CmdConfiguration
+	runArgs     CmdConfiguration
+	validators  []validators.Validator
 }
 
-// Validate checks that the file exists and that extension of the file matches the SDK.
-// Return result of validation (true/false) and error if it occurs
-func (ex *Executor) Validate() error {
-	for _, validator := range ex.validators {
-		err := validator.validator(ex.absoulteFilePath, validator.args...)
-		if err != nil {
-			return err
+// Validate return the function that apply all validators of executor
+func (ex *Executor) Validate() func() error {
+	return func() error {
+		for _, validator := range ex.validators {
+			err := validator.Validator(validator.Args...)
+			if err != nil {
+				return err
+			}
 		}
+		return nil
 	}
-	return nil
 }
 
-// Compile compiles the code and creates executable file.
-// Return error if it occurs
-func (ex *Executor) Compile() error {
-	args := append(ex.compileArgs, ex.relativeFilePath)
-	cmd := exec.Command(ex.compileName, args...)
-	cmd.Dir = ex.dirPath
-	s := cmd.String()
-	fmt.Println(s)
-	out, err := cmd.CombinedOutput()
-	if err != nil {
-		return &CompileError{string(out)}
-	}
-	return nil
-}
-
-// Run runs the executable file.
-// Return logs and error if it occurs
-func (ex *Executor) Run(name string) (string, error) {
-	args := append(ex.runArgs, name)
-	cmd := exec.Command(ex.runName, args...)
-	cmd.Dir = ex.dirPath
-	out, err := cmd.Output()
-	return string(out), err
-}
-
-// NewExecutor executes the compilation, running and validation of code
-func NewExecutor(apacheBeamSdk pb.Sdk, fs *fs_tool.LifeCycle) (*Executor, error) {
-	switch apacheBeamSdk {
-	case pb.Sdk_SDK_JAVA:
-		return NewJavaExecutor(fs, GetJavaValidators()), nil
-	default:
-		return nil, fmt.Errorf("%s isn't supported now", apacheBeamSdk)
-	}
-}
-
-type CompileError struct {
-	error string
+// Compile prepares the Cmd for code compilation
+// Returns Cmd instance
+func (ex *Executor) Compile() *exec.Cmd {
+	args := append(ex.compileArgs.commandArgs, ex.compileArgs.fileName)
+	cmd := exec.Command(ex.compileArgs.commandName, args...)
+	cmd.Dir = ex.compileArgs.workingDir
+	return cmd
 }
 
-func (e *CompileError) Error() string {
-	return fmt.Sprintf("Compilation error: %v", e.error)
+// Run prepares the Cmd for execution of the code
+// Returns Cmd instance
+func (ex *Executor) Run() *exec.Cmd {
+	args := append(ex.runArgs.commandArgs, ex.runArgs.fileName)
+	cmd := exec.Command(ex.runArgs.commandName, args...)
+	cmd.Dir = ex.runArgs.workingDir
+	return cmd
 }
diff --git a/playground/backend/internal/executors/executor_builder.go b/playground/backend/internal/executors/executor_builder.go
new file mode 100644
index 0000000..efe68e2
--- /dev/null
+++ b/playground/backend/internal/executors/executor_builder.go
@@ -0,0 +1,151 @@
+// 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 executors
+
+import (
+	"beam.apache.org/playground/backend/internal/validators"
+)
+
+type handler func(executor *Executor)
+
+//ExecutorBuilder struct
+type ExecutorBuilder struct {
+	actions []handler
+}
+
+//CompileBuilder facet of ExecutorBuilder
+type CompileBuilder struct {
+	ExecutorBuilder
+}
+
+//RunBuilder facet of ExecutorBuilder
+type RunBuilder struct {
+	ExecutorBuilder
+}
+
+//ValidatorBuilder facet of ExecutorBuilder
+type ValidatorBuilder struct {
+	ExecutorBuilder
+}
+
+//NewExecutorBuilder constructor for Executor
+func NewExecutorBuilder() *ExecutorBuilder {
+	return &ExecutorBuilder{}
+}
+
+// WithCompiler - Lives chains to type *ExecutorBuilder and returns a *CompileBuilder
+func (b *ExecutorBuilder) WithCompiler() *CompileBuilder {
+	return &CompileBuilder{*b}
+}
+
+// WithRunner - Lives chains to type *ExecutorBuilder and returns a *CompileBuilder
+func (b *ExecutorBuilder) WithRunner() *RunBuilder {
+	return &RunBuilder{*b}
+}
+
+// WithValidator - Lives chains to type *ExecutorBuilder and returns a *CompileBuilder
+func (b *ExecutorBuilder) WithValidator() *ValidatorBuilder {
+	return &ValidatorBuilder{*b}
+}
+
+//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
+}
+
+//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
+func (b *CompileBuilder) WithFileName(fileName string) *CompileBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.compileArgs.fileName = fileName
+	})
+	return b
+}
+
+//WithWorkingDir adds dir path to executor
+func (b *CompileBuilder) WithWorkingDir(dir string) *CompileBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.compileArgs.workingDir = dir
+	})
+	return b
+}
+
+//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
+func (b *RunBuilder) WithArgs(runArgs []string) *RunBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.runArgs.commandArgs = runArgs
+	})
+	return b
+}
+
+//WithClassName adds file name to executor
+func (b *RunBuilder) WithClassName(name string) *RunBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.runArgs.fileName = name
+	})
+	return b
+}
+
+//WithGraphOutput adds the need of graph output to executor
+func (b *RunBuilder) WithGraphOutput() *RunBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		//todo
+	})
+	return b
+}
+
+//WithWorkingDir adds dir path to executor
+func (b *RunBuilder) WithWorkingDir(dir string) *RunBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.runArgs.workingDir = dir
+	})
+	return b
+}
+
+//WithSdkValidators sets validators to executor
+func (b *ValidatorBuilder) WithSdkValidators(validators *[]validators.Validator) *ValidatorBuilder {
+	b.actions = append(b.actions, func(e *Executor) {
+		e.validators = *validators
+	})
+	return b
+}
+
+//Build builds the executor object
+func (b *ExecutorBuilder) Build() Executor {
+	executor := Executor{}
+	for _, a := range b.actions {
+		a(&executor)
+	}
+	return executor
+}
diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go
new file mode 100644
index 0000000..5614334
--- /dev/null
+++ b/playground/backend/internal/executors/executor_test.go
@@ -0,0 +1,207 @@
+// 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 executors
+
+import (
+	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/environment"
+	"beam.apache.org/playground/backend/internal/validators"
+	"os"
+	"os/exec"
+	"reflect"
+	"testing"
+)
+
+var (
+	executorConfig = environment.NewExecutorConfig("javac", "java", []string{"-d", "bin", "-classpath"}, []string{"-cp", "bin:"})
+	env            = environment.NewEnvironment(environment.NetworkEnvs{}, *environment.NewBeamEnvs(pb.Sdk_SDK_JAVA, executorConfig), environment.ApplicationEnvs{})
+)
+
+// BaseExecutorBuilder fills up an executor with base parameters
+func BaseExecutorBuilder(envs environment.BeamEnvs, workingDir string, filePath string, validatorsFuncs *[]validators.Validator) *ExecutorBuilder {
+	if validatorsFuncs == nil {
+		v := make([]validators.Validator, 0)
+		validatorsFuncs = &v
+	}
+	builder := NewExecutorBuilder().
+		WithCompiler().
+		WithCommand(envs.ExecutorConfig.CompileCmd).
+		WithArgs(envs.ExecutorConfig.CompileArgs).
+		WithFileName(filePath).
+		WithWorkingDir(workingDir).
+		WithRunner().
+		WithCommand(envs.ExecutorConfig.RunCmd).
+		WithArgs(envs.ExecutorConfig.RunArgs).
+		WithClassName("HelloWorld").
+		WithWorkingDir(workingDir).
+		WithValidator().
+		WithSdkValidators(validatorsFuncs).ExecutorBuilder
+	return &builder
+}
+
+func TestExecutor_Compile(t *testing.T) {
+	type fields struct {
+		compileArgs CmdConfiguration
+		runArgs     CmdConfiguration
+		validators  []validators.Validator
+	}
+	tests := []struct {
+		name   string
+		fields fields
+		want   *exec.Cmd
+	}{
+		{
+			name: "TestCompile",
+			fields: fields{
+				compileArgs: CmdConfiguration{
+					fileName:    "filePath",
+					workingDir:  "./",
+					commandName: "javac",
+					commandArgs: []string{"-d", "bin", "-classpath", "/opt/apache/beam/jars/beam-sdks-java-harness.jar"},
+				},
+			},
+			want: &exec.Cmd{
+				Path:         "/usr/bin/javac",
+				Args:         []string{"javac", "-d", "bin", "-classpath", "/opt/apache/beam/jars/beam-sdks-java-harness.jar", "filePath"},
+				Env:          nil,
+				Dir:          "",
+				Stdin:        nil,
+				Stdout:       nil,
+				Stderr:       nil,
+				ExtraFiles:   nil,
+				SysProcAttr:  nil,
+				Process:      nil,
+				ProcessState: nil,
+			},
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			ex := &Executor{
+				compileArgs: tt.fields.compileArgs,
+				runArgs:     tt.fields.runArgs,
+				validators:  tt.fields.validators,
+			}
+			if got := ex.Compile(); !reflect.DeepEqual(got.String(), tt.want.String()) {
+				t.Errorf("WithCompiler() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func TestExecutor_Run(t *testing.T) {
+	type fields struct {
+		compileArgs CmdConfiguration
+		runArgs     CmdConfiguration
+		validators  []validators.Validator
+	}
+	tests := []struct {
+		name   string
+		fields fields
+		want   *exec.Cmd
+	}{
+		{
+			name: "TestRun",
+			fields: fields{
+				runArgs: CmdConfiguration{
+					fileName:    "HelloWorld",
+					workingDir:  "./",
+					commandName: "java",
+					commandArgs: []string{"-cp", "bin:/opt/apache/beam/jars/beam-sdks-java-harness.jar:" +
+						"/opt/apache/beam/jars/beam-runners-direct.jar:/opt/apache/beam/jars/slf4j-jdk14.jar"},
+				},
+			},
+			want: &exec.Cmd{
+				Path: "/usr/bin/java",
+				Args: []string{"java", "-cp", "bin:/opt/apache/beam/jars/beam-sdks-java-harness.jar:" +
+					"/opt/apache/beam/jars/beam-runners-direct.jar:/opt/apache/beam/jars/slf4j-jdk14.jar", "HelloWorld"},
+				Env:          nil,
+				Dir:          "",
+				Stdin:        nil,
+				Stdout:       nil,
+				Stderr:       nil,
+				ExtraFiles:   nil,
+				SysProcAttr:  nil,
+				Process:      nil,
+				ProcessState: nil,
+			},
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			ex := &Executor{
+				compileArgs: tt.fields.compileArgs,
+				runArgs:     tt.fields.runArgs,
+				validators:  tt.fields.validators,
+			}
+			if got := ex.Run(); !reflect.DeepEqual(got.String(), tt.want.String()) {
+				t.Errorf("WithRunner() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func TestBaseExecutorBuilder(t *testing.T) {
+	validatorsFuncs := validators.GetJavaValidators("filePath")
+
+	type args struct {
+		envs            environment.BeamEnvs
+		workingDir      string
+		filePath        string
+		validatorsFuncs *[]validators.Validator
+	}
+	tests := []struct {
+		name string
+		args args
+		want Executor
+	}{
+		{
+			name: "NewCmdProvider",
+			args: args{
+				envs:            env.BeamSdkEnvs,
+				workingDir:      "./",
+				filePath:        "filePath",
+				validatorsFuncs: validatorsFuncs,
+			},
+			want: Executor{
+				compileArgs: CmdConfiguration{
+					fileName:    "filePath",
+					workingDir:  "./",
+					commandName: "javac",
+					commandArgs: []string{"-d", "bin", "-classpath"},
+				},
+				runArgs: CmdConfiguration{
+					fileName:    "HelloWorld",
+					workingDir:  "./",
+					commandName: "java",
+					commandArgs: []string{"-cp", "bin:"},
+				},
+				validators: *validatorsFuncs,
+			},
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			if got := BaseExecutorBuilder(tt.args.envs, tt.args.workingDir, tt.args.filePath, tt.args.validatorsFuncs).Build(); !reflect.DeepEqual(got, tt.want) {
+				t.Errorf("BaseExecutorBuilder() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+	err := os.RemoveAll("configs")
+	if err != nil {
+		return
+	}
+}
diff --git a/playground/backend/internal/executors/java_helper.go b/playground/backend/internal/executors/java_helper.go
deleted file mode 100644
index 3945c63..0000000
--- a/playground/backend/internal/executors/java_helper.go
+++ /dev/null
@@ -1,69 +0,0 @@
-// 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 executors
-package executors
-
-import (
-	"beam.apache.org/playground/backend/internal/fs_tool"
-	"os"
-	"path/filepath"
-	"strings"
-)
-
-const (
-	beamJarPath    = "/opt/apache/beam/jars/beam-sdks-java-harness.jar"
-	runnerJarPath  = "/opt/apache/beam/jars/beam-runners-direct.jar"
-	slf4jPath      = "/opt/apache/beam/jars/slf4j-jdk14.jar"
-	javaExtension  = ".java"
-	javaCompileCmd = "javac"
-	javaRunCmd     = "java"
-	binFolder      = "bin"
-)
-
-// NewJavaExecutor creates an executor with Go specifics
-func NewJavaExecutor(fs *fs_tool.LifeCycle, javaValidators *[]validatorWithArgs) *Executor {
-	compileArgs := []string{"-d", binFolder, "-classpath", beamJarPath}
-	fullClassPath := strings.Join([]string{binFolder, beamJarPath, runnerJarPath, slf4jPath}, ":")
-	runArgs := []string{"-cp", fullClassPath}
-	if javaValidators == nil {
-		v := make([]validatorWithArgs, 0)
-		javaValidators = &v
-	}
-	path, _ := os.Getwd()
-
-	exec := new(Executor)
-	exec.validators = *javaValidators
-	exec.relativeFilePath = fs.GetRelativeExecutableFilePath()
-	exec.absoulteFilePath = fs.GetAbsoluteExecutableFilePath()
-	exec.dirPath = filepath.Join(path, fs.Folder.BaseFolder)
-	exec.compileName = javaCompileCmd
-	exec.runName = javaRunCmd
-	exec.compileArgs = compileArgs
-	exec.runArgs = runArgs
-	return exec
-}
-
-// GetJavaValidators return validators methods that needed for Java file
-func GetJavaValidators() *[]validatorWithArgs {
-	validatorArgs := make([]interface{}, 1)
-	validatorArgs[0] = javaExtension
-	pathCheckerValidator := validatorWithArgs{
-		validator: fs_tool.CheckPathIsValid,
-		args:      validatorArgs,
-	}
-	validators := []validatorWithArgs{pathCheckerValidator}
-	return &validators
-}
diff --git a/playground/backend/internal/executors/java_helper_test.go b/playground/backend/internal/executors/java_helper_test.go
deleted file mode 100644
index 19b3934..0000000
--- a/playground/backend/internal/executors/java_helper_test.go
+++ /dev/null
@@ -1,76 +0,0 @@
-// 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 executors
-
-import (
-	pb "beam.apache.org/playground/backend/internal/api/v1"
-	"beam.apache.org/playground/backend/internal/fs_tool"
-	"github.com/google/uuid"
-	"testing"
-)
-
-var (
-	javaExecutor *Executor
-	pipelineId   = uuid.New()
-)
-
-const (
-	javaCode = "class HelloWorld {\n    public static void main(String[] args) {\n        System.out.println(\"Hello World!\");\n    }\n}"
-)
-
-func TestMain(m *testing.M) {
-	javaFS := setup()
-	defer teardown(javaFS)
-	m.Run()
-}
-
-func setup() *fs_tool.LifeCycle {
-	javaFS, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, pipelineId)
-	_ = javaFS.CreateFolders()
-	_, _ = javaFS.CreateExecutableFile(javaCode)
-	javaExecutor = NewJavaExecutor(javaFS, GetJavaValidators())
-	return javaFS
-}
-
-func teardown(javaFS *fs_tool.LifeCycle) {
-	err := javaFS.DeleteFolders()
-	if err != nil {
-		return
-	}
-}
-
-func TestValidateJavaFile(t *testing.T) {
-	err := javaExecutor.Validate()
-	if err != nil {
-		t.Fatalf(`TestValidateJavaFile error: %v `, err)
-	}
-}
-
-func TestCompileJavaFile(t *testing.T) {
-	err := javaExecutor.Compile()
-	if err != nil {
-		t.Fatalf("TestCompileJavaFile: Unexpexted error at compiliation: %s ", err.Error())
-	}
-}
-
-func TestRunJavaFile(t *testing.T) {
-	className := "HelloWorld"
-	expected := "Hello World!\n"
-	out, err := javaExecutor.Run(className)
-	if expected != out || err != nil {
-		t.Fatalf(`TestRunJavaFile: '%q, %v' doesn't match for '%#q', nil`, out, err, expected)
-	}
-}
diff --git a/playground/backend/internal/fs_tool/path_checker.go b/playground/backend/internal/fs_tool/path_checker.go
index d92ee78..2e63523 100644
--- a/playground/backend/internal/fs_tool/path_checker.go
+++ b/playground/backend/internal/fs_tool/path_checker.go
@@ -46,8 +46,9 @@ func isCorrectExtension(filePath string, correctExtension string) bool {
 }
 
 // CheckPathIsValid checks that the file exists and has a correct extension
-func CheckPathIsValid(filePath string, args ...interface{}) error {
-	correctExtension := args[0].(string)
+func CheckPathIsValid(args ...interface{}) error {
+	filePath := args[0].(string)
+	correctExtension := args[1].(string)
 	notExists := isNotExist(filePath)
 	if notExists {
 		return fs.ErrNotExist
diff --git a/playground/backend/internal/executors/go_helper.go b/playground/backend/internal/validators/java_validators.go
similarity index 59%
copy from playground/backend/internal/executors/go_helper.go
copy to playground/backend/internal/validators/java_validators.go
index 6772145..4eb032f 100644
--- a/playground/backend/internal/executors/go_helper.go
+++ b/playground/backend/internal/validators/java_validators.go
@@ -13,17 +13,25 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-// Package executors
-package executors
+package validators
 
-import "beam.apache.org/playground/backend/internal/fs_tool"
+import (
+	"beam.apache.org/playground/backend/internal/fs_tool"
+)
 
-// NewGoExecutor creates an executor with Go specifics
-func NewGoExecutor(fs *fs_tool.LifeCycle, javaValidators *[]validatorWithArgs) *Executor {
-	return nil
-}
+const (
+	javaExtension = ".java"
+)
 
-// GetGoValidators return validators methods that needed for Go file
-func GetGoValidators() *[]validatorWithArgs {
-	return nil
+// GetJavaValidators return validators methods that should be applied to Java code
+func GetJavaValidators(filePath string) *[]Validator {
+	validatorArgs := make([]interface{}, 2)
+	validatorArgs[0] = filePath
+	validatorArgs[1] = javaExtension
+	pathCheckerValidator := Validator{
+		Validator: fs_tool.CheckPathIsValid,
+		Args:      validatorArgs,
+	}
+	validators := []Validator{pathCheckerValidator}
+	return &validators
 }
diff --git a/playground/backend/internal/executors/go_helper.go b/playground/backend/internal/validators/validator.go
similarity index 67%
rename from playground/backend/internal/executors/go_helper.go
rename to playground/backend/internal/validators/validator.go
index 6772145..22b4fb2 100644
--- a/playground/backend/internal/executors/go_helper.go
+++ b/playground/backend/internal/validators/validator.go
@@ -13,17 +13,9 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-// Package executors
-package executors
+package validators
 
-import "beam.apache.org/playground/backend/internal/fs_tool"
-
-// NewGoExecutor creates an executor with Go specifics
-func NewGoExecutor(fs *fs_tool.LifeCycle, javaValidators *[]validatorWithArgs) *Executor {
-	return nil
-}
-
-// GetGoValidators return validators methods that needed for Go file
-func GetGoValidators() *[]validatorWithArgs {
-	return nil
+type Validator struct {
+	Validator func(args ...interface{}) error
+	Args      []interface{}
 }