diff --git a/playground/backend/cmd/server/controller.go b/playground/backend/cmd/server/controller.go index c42823462c69..d7ae2b345063 100644 --- a/playground/backend/cmd/server/controller.go +++ b/playground/backend/cmd/server/controller.go @@ -51,7 +51,7 @@ func (controller *playgroundController) RunCode(ctx context.Context, info *pb.Ru return nil, errors.InvalidArgumentError("Run code()", "incorrect sdk: "+info.Sdk.String()) } switch info.Sdk { - case pb.Sdk_SDK_UNSPECIFIED, pb.Sdk_SDK_PYTHON, pb.Sdk_SDK_SCIO: + case pb.Sdk_SDK_UNSPECIFIED, pb.Sdk_SDK_SCIO: logger.Errorf("RunCode(): unimplemented sdk: %s\n", info.Sdk) return nil, errors.InvalidArgumentError("Run code()", "unimplemented sdk: "+info.Sdk.String()) } diff --git a/playground/backend/configs/SDK_PYTHON.json b/playground/backend/configs/SDK_PYTHON.json new file mode 100644 index 000000000000..bb6ae89adff5 --- /dev/null +++ b/playground/backend/configs/SDK_PYTHON.json @@ -0,0 +1,6 @@ +{ + "compile_cmd": "", + "run_cmd": "python3", + "compile_args": [], + "run_args": [] +} diff --git a/playground/backend/containers/python/Dockerfile b/playground/backend/containers/python/Dockerfile new file mode 100644 index 000000000000..d02dff63e1d2 --- /dev/null +++ b/playground/backend/containers/python/Dockerfile @@ -0,0 +1,51 @@ +############################################################################### +# 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. +############################################################################### +ARG BASE_IMAGE=apache/beam_python3.7_sdk:latest +ARG GO_BASE_IMAGE=golang:1.17-buster +FROM $GO_BASE_IMAGE AS build + +# Setup Go Environment +ENV GOPATH /go +ENV PATH $GOPATH/bin:$PATH +RUN mkdir -p "$GOPATH/src" "$GOPATH/bin" && chmod -R 777 "$GOPATH" +RUN go install google.golang.org/protobuf/cmd/protoc-gen-go@v1.27.1 &&\ + go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@v1.1 + +# Prepare Application +COPY src /go/src/playground/backend +#COPY playground /go/src/playground/playground +WORKDIR /go/src/playground/backend +RUN ls +# Build Application +RUN go mod download &&\ + go mod tidy &&\ + cd cmd/server &&\ + go build -o /go/bin/server_python_backend + +FROM $BASE_IMAGE +ENV SERVER_IP=0.0.0.0 +ENV SERVER_PORT=8080 +ENV APP_WORK_DIR=/opt/playground/backend/ +ENV BEAM_SDK="SDK_PYTHON" + +# Copy build result +COPY --from=build /go/bin/server_python_backend /opt/playground/backend/ +COPY --from=build /go/src/playground/backend/configs /opt/playground/backend/configs/ + +RUN printenv +ENTRYPOINT ["/opt/playground/backend/server_python_backend"] diff --git a/playground/backend/containers/python/app.yaml b/playground/backend/containers/python/app.yaml new file mode 100644 index 000000000000..28fb02e84195 --- /dev/null +++ b/playground/backend/containers/python/app.yaml @@ -0,0 +1,22 @@ +/* + * 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. + */ +service: backend-python +runtime: custom +env: flex +manual_scaling: + instances: 1 diff --git a/playground/backend/containers/python/build.gradle b/playground/backend/containers/python/build.gradle new file mode 100644 index 000000000000..5e2e20b38ec7 --- /dev/null +++ b/playground/backend/containers/python/build.gradle @@ -0,0 +1,69 @@ +/* + * 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. + */ + + +apply plugin: 'org.apache.beam.module' +apply plugin: 'base' +applyDockerNature() + +def playgroundJobServerProject = "${project.path.replace('-container', '')}" + +description = project(playgroundJobServerProject).description + " :: Container" + +configurations { + dockerDependency +} + +dependencies { + dockerDependency project(path: playgroundJobServerProject, configuration: "shadow") +} + +task copyDockerfileDependencies(type: Copy) { + copy { + from '../../../backend/' + into 'build/src' + exclude 'containers' + } + copy { + from 'Dockerfile' + into 'build/' + } + copy { + from '../../../playground' + into 'build/playground' + } +} + +docker { + name containerImageName( + name: project.docker_image_default_repo_prefix + "playground-backend-python", + root: project.rootProject.hasProperty(["docker-repository-root"]) ? + project.rootProject["docker-repository-root"] : + project.docker_image_default_repo_root) + files "./build/" + tags containerImageTags() + buildArgs(['BASE_IMAGE': project.rootProject.hasProperty(["base-image"]) ? + project.rootProject["base-image"] : + "apache/beam_python3.7_sdk" ]) + buildArgs(['GO_BASE_IMAGE': project.rootProject.hasProperty(["go-base-image"]) ? + project.rootProject["go-base-image"] : + "golang:1.17-buster" ]) +} + +// Ensure that we build the required resources and copy and file dependencies from related projects +dockerPrepare.dependsOn copyDockerfileDependencies diff --git a/playground/backend/containers/python/settings.gradle b/playground/backend/containers/python/settings.gradle new file mode 100644 index 000000000000..63b23b446ded --- /dev/null +++ b/playground/backend/containers/python/settings.gradle @@ -0,0 +1,19 @@ +/* + * 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. + */ + +rootProject.name = 'apache-beam-playground-backend-python' diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 85827bdcf92d..8d81d35f97e6 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -78,15 +78,20 @@ func Process(ctx context.Context, cacheService cache.Cache, lc *fs_tool.LifeCycl return } - // compile - logger.Infof("%s: Compile() ...\n", pipelineId) - compileCmd := executor.Compile(ctxWithTimeout) - var compileError bytes.Buffer - var compileOutput bytes.Buffer - runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) - - if err := processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel, &compileOutput, &compileError, errorChannel, pb.Status_STATUS_COMPILE_ERROR, pb.Status_STATUS_EXECUTING); err != nil { - return + switch sdkEnv.ApacheBeamSdk { + case pb.Sdk_SDK_JAVA, pb.Sdk_SDK_GO: + // compile + logger.Infof("%s: Compile() ...\n", pipelineId) + compileCmd := executor.Compile(ctxWithTimeout) + var compileError bytes.Buffer + var compileOutput bytes.Buffer + runCmdWithOutput(compileCmd, &compileOutput, &compileError, successChannel, errorChannel) + + if err := processStep(ctxWithTimeout, pipelineId, cacheService, cancelChannel, successChannel, &compileOutput, &compileError, errorChannel, pb.Status_STATUS_COMPILE_ERROR, pb.Status_STATUS_EXECUTING); err != nil { + return + } + case pb.Sdk_SDK_PYTHON: + processSuccess(ctx, []byte(""), pipelineId, cacheService, pb.Status_STATUS_EXECUTING) } runBuilder, err := run_builder.Setup(pipelineId, lc, appEnv.WorkingDir(), sdkEnv, compileBuilder) diff --git a/playground/backend/internal/environment/environment_service.go b/playground/backend/internal/environment/environment_service.go index 8d3d4273ba5d..3622663073e0 100644 --- a/playground/backend/internal/environment/environment_service.go +++ b/playground/backend/internal/environment/environment_service.go @@ -187,7 +187,7 @@ func createExecutorConfig(apacheBeamSdk pb.Sdk, configPath string) (*ExecutorCon case pb.Sdk_SDK_GO: // Go sdk doesn't need any additional arguments from the config file case pb.Sdk_SDK_PYTHON: - return nil, errors.New("not yet supported") + // Python sdk doesn't need any additional arguments from the config file case pb.Sdk_SDK_SCIO: return nil, errors.New("not yet supported") } diff --git a/playground/backend/internal/executors/executor_test.go b/playground/backend/internal/executors/executor_test.go index fbcfb525e6d5..08e687398f9a 100644 --- a/playground/backend/internal/executors/executor_test.go +++ b/playground/backend/internal/executors/executor_test.go @@ -29,7 +29,7 @@ import ( 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{}) + env = environment.NewEnvironment(environment.NetworkEnvs{}, *environment.NewBeamEnvs(pb.Sdk_SDK_JAVA, executorConfig, ""), environment.ApplicationEnvs{}) ) // BaseExecutorBuilder fills up an executor with base parameters diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 4bb30ff03d6d..61011c9a2e61 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -62,6 +62,8 @@ func NewLifeCycle(sdk pb.Sdk, pipelineId uuid.UUID, workingDir string) (*LifeCyc return newJavaLifeCycle(pipelineId, workingDir), nil case pb.Sdk_SDK_GO: return newGoLifeCycle(pipelineId, workingDir), nil + case pb.Sdk_SDK_PYTHON: + return newPythonLifeCycle(pipelineId, workingDir), nil default: return nil, fmt.Errorf("%s isn't supported now", sdk) } diff --git a/playground/backend/internal/fs_tool/lc_constructor.go b/playground/backend/internal/fs_tool/lc_constructor.go index a979f825a68e..d2a6164a141b 100644 --- a/playground/backend/internal/fs_tool/lc_constructor.go +++ b/playground/backend/internal/fs_tool/lc_constructor.go @@ -45,3 +45,21 @@ func newCompilingLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFileEx pipelineId: pipelineId, } } + +// newInterpretedLifeCycle creates LifeCycle for interpreted SDK environment. +func newInterpretedLifeCycle(pipelineId uuid.UUID, workingDir string, sourceFileExtension string) *LifeCycle { + sourceFileFolder := filepath.Join(workingDir, baseFileFolder, pipelineId.String()) + return &LifeCycle{ + folderGlobs: []string{sourceFileFolder}, + Folder: Folder{ + BaseFolder: sourceFileFolder, + SourceFileFolder: sourceFileFolder, + ExecutableFileFolder: sourceFileFolder, + }, + Extension: Extension{ + ExecutableFileExtension: sourceFileExtension, + SourceFileExtension: sourceFileExtension, + }, + pipelineId: pipelineId, + } +} diff --git a/playground/backend/internal/fs_tool/python_fs.go b/playground/backend/internal/fs_tool/python_fs.go new file mode 100644 index 000000000000..4fc35241a582 --- /dev/null +++ b/playground/backend/internal/fs_tool/python_fs.go @@ -0,0 +1,29 @@ +// 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 fs_tool + +import ( + "github.com/google/uuid" +) + +const ( + pythonExecutableFileExtension = ".py" +) + +// newPythonLifeCycle creates LifeCycle with go SDK environment. +func newPythonLifeCycle(pipelineId uuid.UUID, workingDir string) *LifeCycle { + return newInterpretedLifeCycle(pipelineId, workingDir, pythonExecutableFileExtension) +} diff --git a/playground/backend/internal/fs_tool/python_fs_test.go b/playground/backend/internal/fs_tool/python_fs_test.go new file mode 100644 index 000000000000..c0e6ded2755c --- /dev/null +++ b/playground/backend/internal/fs_tool/python_fs_test.go @@ -0,0 +1,79 @@ +// 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 fs_tool + +import ( + "fmt" + "github.com/google/uuid" + "reflect" + "testing" +) + +func Test_newPythonLifeCycle(t *testing.T) { + pipelineId := uuid.New() + workingDir := "workingDir" + baseFileFolder := fmt.Sprintf("%s/%s/%s", workingDir, baseFileFolder, pipelineId) + + type args struct { + pipelineId uuid.UUID + workingDir string + } + tests := []struct { + name string + args args + want *LifeCycle + }{ + { + // Test case with calling newPythonLifeCycle method with correct pipelineId and workingDir. + // As a result, want to receive an expected python life cycle. + name: "newPythonLifeCycle", + args: args{ + pipelineId: pipelineId, + workingDir: workingDir, + }, + want: &LifeCycle{ + folderGlobs: []string{baseFileFolder}, + Folder: Folder{ + BaseFolder: baseFileFolder, + SourceFileFolder: baseFileFolder, + ExecutableFileFolder: baseFileFolder, + }, + Extension: Extension{ + SourceFileExtension: pythonExecutableFileExtension, + ExecutableFileExtension: pythonExecutableFileExtension, + }, + pipelineId: pipelineId, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := newPythonLifeCycle(tt.args.pipelineId, tt.args.workingDir) + if !reflect.DeepEqual(got.folderGlobs, tt.want.folderGlobs) { + t.Errorf("newPythonLifeCycle() folderGlobs = %v, want %v", got.folderGlobs, tt.want.folderGlobs) + } + if !reflect.DeepEqual(got.Folder, tt.want.Folder) { + t.Errorf("newPythonLifeCycle() Folder = %v, want %v", got.Folder, tt.want.Folder) + } + if !reflect.DeepEqual(got.Extension, tt.want.Extension) { + t.Errorf("newPythonLifeCycle() Extension = %v, want %v", got.Extension, tt.want.Extension) + } + if !reflect.DeepEqual(got.pipelineId, tt.want.pipelineId) { + t.Errorf("newPythonLifeCycle() pipelineId = %v, want %v", got.pipelineId, tt.want.pipelineId) + } + }) + } +} diff --git a/playground/backend/internal/preparators/python_preparators.go b/playground/backend/internal/preparators/python_preparators.go new file mode 100644 index 000000000000..d2ff96382bfb --- /dev/null +++ b/playground/backend/internal/preparators/python_preparators.go @@ -0,0 +1,23 @@ +// 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 preparators + +const () + +// GetPythonPreparators returns preparation methods that should be applied to Python code +func GetPythonPreparators(filePath string) *[]Preparator { + return &[]Preparator{} +} diff --git a/playground/backend/internal/setup_tools/run_builder/run_builder_setuper.go b/playground/backend/internal/setup_tools/run_builder/run_builder_setuper.go index e3cc0e7c4903..32199cb2724d 100644 --- a/playground/backend/internal/setup_tools/run_builder/run_builder_setuper.go +++ b/playground/backend/internal/setup_tools/run_builder/run_builder_setuper.go @@ -45,6 +45,8 @@ func Setup(pipelineId uuid.UUID, lc *fs_tool.LifeCycle, workingDir string, sdkEn WithExecutableName(className) case pb.Sdk_SDK_GO: runBuilder = runBuilder.WithCommand(lc.GetAbsoluteExecutableFilePath()) + case pb.Sdk_SDK_PYTHON: + runBuilder = runBuilder.WithExecutableName(lc.GetAbsoluteExecutableFilePath()) default: return nil, fmt.Errorf("incorrect sdk: %s", sdkEnv.ApacheBeamSdk) } diff --git a/playground/backend/internal/setup_tools/run_builder/run_builder_setuper_test.go b/playground/backend/internal/setup_tools/run_builder/run_builder_setuper_test.go index c07f7dc98dec..41c05dc37adb 100644 --- a/playground/backend/internal/setup_tools/run_builder/run_builder_setuper_test.go +++ b/playground/backend/internal/setup_tools/run_builder/run_builder_setuper_test.go @@ -53,7 +53,7 @@ func TestSetup(t *testing.T) { CompileArgs: []string{"MOCK_COMPILE_ARG"}, RunArgs: []string{"MOCK_RUN_ARG"}, } - sdkEnv := environment.NewBeamEnvs(playground.Sdk_SDK_JAVA, executorConfig) + sdkEnv := environment.NewBeamEnvs(playground.Sdk_SDK_JAVA, executorConfig, "") compileBuilder := &executors.CompileBuilder{} className, err := successLc.ExecutableName(successPipelineId, "") if err != nil { @@ -81,7 +81,7 @@ func TestSetup(t *testing.T) { pipelineId: successPipelineId, lc: successLc, workingDir: "", - sdkEnv: environment.NewBeamEnvs(playground.Sdk_SDK_UNSPECIFIED, executorConfig), + sdkEnv: environment.NewBeamEnvs(playground.Sdk_SDK_UNSPECIFIED, executorConfig, ""), compileBuilder: compileBuilder, }, want: nil, diff --git a/playground/backend/internal/utils/validators_utils.go b/playground/backend/internal/utils/validators_utils.go index c5e576177830..8cbe367c9b4b 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/playground/backend/internal/utils/validators_utils.go @@ -29,6 +29,8 @@ func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) val = validators.GetJavaValidators(filepath) case pb.Sdk_SDK_GO: val = validators.GetGoValidators() + case pb.Sdk_SDK_PYTHON: + val = validators.GetPythonValidators() default: return nil, fmt.Errorf("incorrect sdk: %s", sdk) } diff --git a/playground/backend/internal/validators/python_validators.go b/playground/backend/internal/validators/python_validators.go new file mode 100644 index 000000000000..4d997337df78 --- /dev/null +++ b/playground/backend/internal/validators/python_validators.go @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package validators + +// GetPythonValidators return validators methods that should be applied to Go code +func GetPythonValidators() *[]Validator { + //TODO: Will be added in task [BEAM-13292] + return &[]Validator{} +} diff --git a/settings.gradle.kts b/settings.gradle.kts index 9e3d5bae3d73..0b91a663520a 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -60,6 +60,7 @@ include(":playground") include(":playground:backend") include(":playground:frontend") include(":playground:backend:containers:java") +include(":playground:backend:containers:python") include(":runners:core-construction-java") include(":runners:core-java") include(":runners:direct-java")