Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
b73f5f7
[Playground][BEAM-12941][Bugfix] Fix workflows for playground applica…
snkalinin Nov 16, 2021
d449275
Revert "[Playground][BEAM-12941][Bugfix] Fix workflows for playground…
ilya-kozyrev Nov 16, 2021
69d0191
Merge branch 'apache:master' into master
ilya-kozyrev Nov 16, 2021
1bf7f6f
Merge branch 'apache:master' into master
Nov 16, 2021
72bad55
Merge branch 'apache:master' into master
ilya-kozyrev Nov 16, 2021
1d6783e
Merge branch 'apache:master' into master
Nov 16, 2021
3a81f54
Merge branch 'apache:master' into master
ilya-kozyrev Nov 16, 2021
d9cce13
Merge branch 'apache:master' into master
Nov 17, 2021
c24b695
Merge branch 'apache:master' into master
Nov 19, 2021
6b6f0b9
draft
Nov 19, 2021
10e7762
Merge branch 'apache:master' into master
Nov 19, 2021
24ef6bf
Merge branch 'apache:master' into master
pavel-avilov Nov 21, 2021
df5e053
Rename LF executable -> source
Nov 22, 2021
6b7603d
Reanme LF compile -> executables
Nov 22, 2021
f0b508b
Merge remote-tracking branch 'origin/BEAM-13297-refactoring-ls' into …
Nov 22, 2021
2957ff2
working python sdk
Nov 22, 2021
f240b10
Update playground/backend/internal/fs_tool/python_fs_test.go
Nov 22, 2021
05878a7
Update playground/backend/internal/fs_tool/python_fs_test.go
Nov 22, 2021
ab58327
Merge branch 'apache:master' into master
Nov 23, 2021
d8639b9
Merge branch 'master' into BEAM-13258-playground-python-sdk
Nov 23, 2021
02412f8
refactoring + adding dockerfile
Nov 23, 2021
302cb1f
Merge branch 'apache:master' into master
ElessarST Nov 24, 2021
816d46c
add preparators array (will be filled later)
Nov 23, 2021
f44869a
Merge branch 'master' into BEAM-13307-playground-python-sdk
Nov 24, 2021
2b7569c
fix indentation
Nov 24, 2021
42a47e9
[BEAM-13307][Playground]
Nov 24, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion playground/backend/cmd/server/controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
}
Expand Down
6 changes: 6 additions & 0 deletions playground/backend/configs/SDK_PYTHON.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
{
"compile_cmd": "",
"run_cmd": "python3",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah this is interesting - now that I think about this, we should mention in the README that to update runtime environments, we need to change these config files (we cna leave it for later)

"compile_args": [],
"run_args": []
}
51 changes: 51 additions & 0 deletions playground/backend/containers/python/Dockerfile
Original file line number Diff line number Diff line change
@@ -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
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this line?

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"]
22 changes: 22 additions & 0 deletions playground/backend/containers/python/app.yaml
Original file line number Diff line number Diff line change
@@ -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
69 changes: 69 additions & 0 deletions playground/backend/containers/python/build.gradle
Original file line number Diff line number Diff line change
@@ -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" ])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's also document the places where the Beam container version is chosen in the README (also for later)

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
19 changes: 19 additions & 0 deletions playground/backend/containers/python/settings.gradle
Original file line number Diff line number Diff line change
@@ -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'
23 changes: 14 additions & 9 deletions playground/backend/internal/code_processing/code_processing.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
Expand Down
2 changes: 1 addition & 1 deletion playground/backend/internal/executors/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 2 additions & 0 deletions playground/backend/internal/fs_tool/fs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
18 changes: 18 additions & 0 deletions playground/backend/internal/fs_tool/lc_constructor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
}
}
29 changes: 29 additions & 0 deletions playground/backend/internal/fs_tool/python_fs.go
Original file line number Diff line number Diff line change
@@ -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)
}
79 changes: 79 additions & 0 deletions playground/backend/internal/fs_tool/python_fs_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
})
}
}
23 changes: 23 additions & 0 deletions playground/backend/internal/preparators/python_preparators.go
Original file line number Diff line number Diff line change
@@ -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{}
}
Loading