Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
5c65610
Initialize SBT cache during SCIO playground container build
TSultanov Jan 9, 2023
617c182
Fix issue with reading of GRPC_TIMEOUT environment variables in CI/CD…
TSultanov Jan 9, 2023
b42ad5b
Fix cleanup of execution environment for Scala examples
TSultanov Jan 9, 2023
f59bbe7
Fix panic in preparers when an empty file is passed
TSultanov Jan 10, 2023
2d782c1
Use better name for SCIO project directory
TSultanov Jan 10, 2023
3960bf0
Run "sbt compile" during container build to fetch all Scala dependencies
TSultanov Jan 11, 2023
e8689bd
Disable forking JVM in SBT to significantly reduce memory usage
TSultanov Jan 12, 2023
08473d2
Impose memory limits on local deployments of SCIO runner container to…
TSultanov Jan 12, 2023
eecba19
Fine-tune Java GC to improve performance and memory usage of SCIO exa…
TSultanov Jan 12, 2023
d645408
Remove large blobs of text from common_test.go
TSultanov Jan 27, 2023
fd908d2
Add `sbt` to the list of development dependencies
TSultanov Feb 8, 2023
f02d56a
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Feb 10, 2023
c8900df
Clarify running of backend tests in Playground
TSultanov Feb 16, 2023
e127647
Clarify local running of backend
TSultanov Feb 17, 2023
5e8a2de
Improve consistency in code blocks in backend Readme
TSultanov Feb 17, 2023
f8f6457
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Feb 20, 2023
bfd402e
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Feb 21, 2023
4f68a76
Fixing trailing whitespace
TSultanov Feb 21, 2023
6d85ebd
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Mar 7, 2023
dc15658
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Mar 7, 2023
7ba045f
Merge remote-tracking branch 'upstream/master' into beam-24943-scio-r…
TSultanov Mar 15, 2023
36e42cf
Update playground/backend/README.md
TSultanov Mar 22, 2023
f9dbb51
Update playground/backend/internal/utils/preparers_utils_test.go
TSultanov Mar 22, 2023
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
35 changes: 34 additions & 1 deletion playground/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -35,11 +35,44 @@ The following requirements are needed for development, testing, and deploying.
- [Docker Compose](https://docs.docker.com/compose/install/)
- [gcloud CLI](https://cloud.google.com/sdk/docs/install)
- [gcloud Beta Commands](https://cloud.google.com/sdk/gcloud/reference/components/install)
- [Cloud Datastore Emulator](https://cloud.google.com/sdk/gcloud/reference/components/install)
- [Cloud Datastore Emulator](https://cloud.google.com/datastore/docs/tools/datastore-emulator)
- [sbt](https://www.scala-sbt.org/)

### Google Cloud Shell Prerequisites Installation
Google Cloud Shell already has most of the prerequisites installed. Only few tools need to be installed separately

#### Flutter
```shell
git config --global --add safe.directory /google/flutter
flutter doctor
```

#### Protobuf
```shell
go install google.golang.org/protobuf/cmd/protoc-gen-go@v1.28
go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@v1.2
dart pub global activate protoc_plugin
npm install -g @bufbuild/buf
```
#### sbt
```shell
echo "deb https://repo.scala-sbt.org/scalasbt/debian all main" | sudo tee /etc/apt/sources.list.d/sbt.list
echo "deb https://repo.scala-sbt.org/scalasbt/debian /" | sudo tee /etc/apt/sources.list.d/sbt_old.list
curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0x2EE0EA64E40A89B84B2DF73499E82A75642AC823" | sudo -H gpg --no-default-keyring --keyring gnupg-ring:/etc/apt/trusted.gpg.d/scalasbt-release.gpg --import
sudo chmod 644 /etc/apt/trusted.gpg.d/scalasbt-release.gpg
sudo apt-get update
sudo apt-get install sbt
```
### Additional tools
Google Cloud shell machines do not have `netcat` and `lsof` preinstalled. Install them using:
```shell
sudo apt install netcat lsof
```

# Available Gradle Tasks

## Perform overall pre-commit checks
> **Google Cloud Shell note:** run `unset GOOGLE_CLOUD_PROJECT` before running tests so they would use locally running datastore emulator.

```
cd beam
Expand Down
61 changes: 51 additions & 10 deletions playground/backend/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -27,43 +27,84 @@ no setup.

## Getting Started

See [playground/README.md](../README.md) for details on requirements and setup.
See [playground/README.md](../README.md) for details on installing development dependencies.

This section describes what is needed to run the backend application.

- Go commands to run/test the backend locally
- Set up environment variables to run the backend locally
- Running the backend via Docker

### Go commands to run/test application locally
## Go commands to run/test application locally

### Prerequisite

> **Google Cloud Shell note:** `start_datastore_emulator.sh` script makes use of `nc` and `lsof` commands which are not installed on Google Cloud Shell machines. You can install them using `sudo apt install netcat lsof`.

> **Google Cloud Shell note:** run `unset GOOGLE_CLOUD_PROJECT` before running tests so they would use locally running datastore emulator.

Start datastore emulator
```shell
bash start_datastore_emulator.sh
```

After you have finished running tests
```shell
bash stop_datastore_emulator.sh
```

### Run/build
Go to the backend directory:

```shell
$ cd backend
cd backend
```

The following command is used to build and serve the backend locally:
To run backend server on development machine without using docker you'll need first to prepare a working directory anywhere outside of Beam source tree:
```shell
mkdir ~/path/to/workdir
```
and then copy `datasets/` and `configs/` and `logging.properties` from [`playground/backend/`](/playground/backend/) directory:
```shell
cp -r {logging.properties,datasets/,configs/} ~/path/to/workdir
```

In case if you want to start backend for Go SDK you additionally will also need to create a prepared mod dir and export an additional environment variable:
```shell
$ go run ./cmd/server/server.go
export PREPARED_MOD_DIR=~/path/to/workdir/prepared_folder
SDK_TAG=2.44.0 bash ./containers/go/setup_sdk.sh $PREPARED_MOD_DIR
```

The following command will build and serve the backend locally:

```shell
SERVER_PORT=<port> \
BEAM_SDK=<beam_sdk_type> \
APP_WORK_DIR=<path_to_workdir> \
DATASTORE_EMULATOR_HOST=127.0.0.1:8888 \
DATASTORE_PROJECT_ID=test \
SDK_CONFIG=../sdks-emulator.yaml \
go run ./cmd/server
```

where `<port>` should be the value of port on which you want to have the backend server available; `<beam_sdk_type>` is a value of desired Beam SDK, possible values are `SDK_UNSPECIFIED`, `SDK_JAVA`, `SDK_PYTHON`, `SDK_GO`, `SDK_SCIO`; `<path_to_workdir>` should be set to path to your work dir, e.g. `~/path/to/workdir`.

Run the following command to generate a release build file:

```shell
$ go build ./cmd/server/server.go
go build ./cmd/server/server.go
```

### Test
Playground tests may be run using this command:

```shell
$ go test ... -v
go test ./... -v
```

The full list of commands can be found [here](https://pkg.go.dev/cmd/go).

### Set up environment variables to run the backend locally
## Set up environment variables to run the backend locally

These environment variables should be set to run the backend locally:

Expand Down Expand Up @@ -96,7 +137,7 @@ default value and there is no need to set them up to launch locally:
- `PROPERTY_PATH` - is the application properties path (default value = `.`)
- `CACHE_REQUEST_TIMEOUT` - is the timeout to request data from cache (default value = `5 sec`)

### Application properties
## Application properties

These properties are stored in `backend/properties.yaml` file:

Expand All @@ -106,7 +147,7 @@ These properties are stored in `backend/properties.yaml` file:
- `removing_unused_snippets_cron` - is the cron expression for the scheduled task to remove unused snippets.
- `removing_unused_snippets_days` - is the number of days after which a snippet becomes unused.

### Running the server app via Docker
## Running the server app via Docker

To run the server using Docker images there are `Docker` files in the `containers` folder for Java, Python and Go
languages. Each of them processes the corresponding SDK, so the backend with Go SDK will work with Go
Expand Down
17 changes: 15 additions & 2 deletions playground/backend/containers/scio/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ COPY --from=build /go/src/playground/backend/configs /opt/playground/backend/con
COPY --from=build /go/src/playground/backend/logging.properties /opt/playground/backend/
COPY --from=build /go/src/playground/backend/new_scio_project.sh /opt/playground/backend/
COPY --from=build /go/src/playground/backend/internal/fs_tool/ExampleData.scala /opt/playground/backend/
RUN chmod +x /opt/playground/backend/new_scio_project.sh

# Install sbt
RUN echo "deb https://repo.scala-sbt.org/scalasbt/debian all main" | tee /etc/apt/sources.list.d/sbt.list &&\
Expand All @@ -64,8 +65,6 @@ RUN mkdir /opt/mitmproxy &&\
mkdir /usr/local/share/ca-certificates/extra
COPY allow_list_proxy.py /opt/mitmproxy/
COPY allow_list.py /opt/mitmproxy/
ENV HTTP_PROXY="http://127.0.0.1:8081"
ENV HTTPS_PROXY="http://127.0.0.1:8081"

COPY src/properties.yaml /opt/playground/backend/properties.yaml
COPY entrypoint.sh /
Expand All @@ -90,4 +89,18 @@ RUN chown -R appuser:appgroup /opt/playground/backend/executable_files/ \
# Switch to appuser
USER appuser

# Let sbt download files from Maven
RUN mkdir -p /tmp/sbt-initialize
WORKDIR /tmp/sbt-initialize
RUN /opt/playground/backend/new_scio_project.sh
WORKDIR /tmp/sbt-initialize/scio
RUN sbt "+compile"
WORKDIR /
RUN rm -r /tmp/sbt-initialize

# Enable mitmproxy
ENV HTTP_PROXY="http://127.0.0.1:8081"
ENV HTTPS_PROXY="http://127.0.0.1:8081"
ENV SBT_OPTS="-Xmx512M -XX:+UseG1GC -XX:+UseStringDeduplication"
Comment on lines +101 to +104
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you explain again why we need mitmproxy? I read the website briefly and it looks like a networking tool. However, aren't we achieving our networking needs using kubernetes Service and other related kinds? Can our images and the Dockerfiles that describe them serve just to ship the binary?

Copy link
Contributor Author

@TSultanov TSultanov Jan 20, 2023

Choose a reason for hiding this comment

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

It's being used to filter out which Google Cloud Storage buckets are accessible by the examples.

However, after investigating a little bit it seems that this is not the best place to have it. We discussed this with @MakarkinSAkvelon and think that the best solution would be to have a single proxy node which would act as a reverse proxy for letting external connections to the runners and also as a single point though which runners would be able to only access the allowlisted hosts.

This approach with a single proxy node will simplify origin policy for frontend and we will be able to get rid of the need to generate config.g.dart completely.

This issue, however, I think, is beyond the scope of this PR and we can address it separately.


ENTRYPOINT ["/entrypoint.sh"]
Original file line number Diff line number Diff line change
Expand Up @@ -424,21 +424,21 @@ func readGraphFile(pipelineLifeCycleCtx, backgroundCtx context.Context, cacheSer
case <-ticker.C:
if _, err := os.Stat(graphFilePath); err == nil {
ticker.Stop()
graph, err := utils.ReadFile(pipelineId, graphFilePath)
graph, err := os.ReadFile(graphFilePath)
if err != nil {
logger.Errorf("%s: Error during saving graph to the file: %s", pipelineId, err.Error())
}
_ = utils.SetToCache(backgroundCtx, cacheService, pipelineId, cache.Graph, graph)
_ = utils.SetToCache(backgroundCtx, cacheService, pipelineId, cache.Graph, string(graph))
}
// in case of timeout or cancel
case <-pipelineLifeCycleCtx.Done():
ticker.Stop()
if _, err := os.Stat(graphFilePath); err == nil {
graph, err := utils.ReadFile(pipelineId, graphFilePath)
graph, err := os.ReadFile(graphFilePath)
if err != nil {
logger.Errorf("%s: Error during saving graph to the file: %s", pipelineId, err.Error())
}
_ = utils.SetToCache(backgroundCtx, cacheService, pipelineId, cache.Graph, graph)
_ = utils.SetToCache(backgroundCtx, cacheService, pipelineId, cache.Graph, string(graph))
}
return
}
Expand Down
36 changes: 1 addition & 35 deletions playground/backend/internal/fs_tool/fs.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,9 @@ package fs_tool
import (
"beam.apache.org/playground/backend/internal/logger"
"fmt"
"io"
"github.com/google/uuid"
"io/fs"
"os"
"path/filepath"

"github.com/google/uuid"

pb "beam.apache.org/playground/backend/internal/api/v1"
"beam.apache.org/playground/backend/internal/db/entity"
Expand Down Expand Up @@ -114,37 +111,6 @@ func (lc *LifeCycle) CreateSourceCodeFiles(sources []entity.FileEntity) error {
return nil
}

// CopyFile copies a file with fileName from sourceDir to destinationDir.
func (lc *LifeCycle) CopyFile(fileName, sourceDir, destinationDir string) error {
absSourcePath := filepath.Join(sourceDir, fileName)
absDestinationPath := filepath.Join(destinationDir, fileName)
sourceFileStat, err := os.Stat(absSourcePath)
if err != nil {
return err
}

if !sourceFileStat.Mode().IsRegular() {
return fmt.Errorf("%s is not a regular file", fileName)
}

sourceFile, err := os.Open(absSourcePath)
if err != nil {
return err
}
defer sourceFile.Close()

destinationFile, err := os.Create(absDestinationPath)
if err != nil {
return err
}
defer destinationFile.Close()
_, err = io.Copy(destinationFile, sourceFile)
if err != nil {
return err
}
return nil
}

func (lc *LifeCycle) GetPreparerParameters() map[string]string {
if lc.emulatorMockCluster == nil {
return map[string]string{}
Expand Down
6 changes: 1 addition & 5 deletions playground/backend/internal/fs_tool/fs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,11 +135,7 @@ func TestLifeCycle_CopyFile(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
l := &LifeCycle{
folderGlobs: tt.fields.folderGlobs,
Paths: tt.fields.Paths,
}
if err := l.CopyFile(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir); (err != nil) != tt.wantErr {
if err := utils.CopyFilePreservingName(tt.args.fileName, tt.args.sourceDir, tt.args.destinationDir); (err != nil) != tt.wantErr {
t.Errorf("CopyFile() error = %v, wantErr %v", err, tt.wantErr)
}
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (
"beam.apache.org/playground/backend/internal/db/entity"
"beam.apache.org/playground/backend/internal/fs_tool"
"beam.apache.org/playground/backend/internal/logger"
"beam.apache.org/playground/backend/internal/utils"
utils "beam.apache.org/playground/backend/internal/utils"
)

const (
Expand All @@ -41,13 +41,11 @@ const (
javaLogFilePlaceholder = "{logFilePath}"
goModFileName = "go.mod"
goSumFileName = "go.sum"
scioProjectName = "y"
bashCmd = "bash"
scioProjectName = "scio"
scioProjectPath = scioProjectName + "/src/main/scala/" + scioProjectName
logFileName = "logs.log"
defaultExampleInSbt = "WordCount.scala"
shCmd = "sh"
rmCmd = "rm"
cpCmd = "cp"
scioProject = "new_scio_project.sh"
scioCommonConstants = "ExampleData.scala"
)
Expand Down Expand Up @@ -128,11 +126,11 @@ func Setup(sdk pb.Sdk, sources []entity.FileEntity, pipelineId uuid.UUID, workin
// prepareGoFiles prepares file for Go environment.
// Copy go.mod and go.sum file from /path/to/preparedModDir to /path/to/workingDir/pipelinesFolder/{pipelineId}
func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uuid.UUID) error {
if err := lc.CopyFile(goModFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil {
if err := utils.CopyFilePreservingName(goModFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil {
logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goModFileName, err.Error())
return err
}
if err := lc.CopyFile(goSumFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil {
if err := utils.CopyFilePreservingName(goSumFileName, preparedModDir, lc.Paths.AbsoluteBaseFolderPath); err != nil {
logger.Errorf("%s: error during copying %s file: %s\n", pipelineId, goSumFileName, err.Error())
return err
}
Expand All @@ -144,7 +142,7 @@ func prepareGoFiles(lc *fs_tool.LifeCycle, preparedModDir string, pipelineId uui
//
// and update this file according to pipeline.
func prepareJavaFiles(lc *fs_tool.LifeCycle, workingDir string, pipelineId uuid.UUID) error {
err := lc.CopyFile(javaLogConfigFileName, workingDir, lc.Paths.AbsoluteBaseFolderPath)
err := utils.CopyFilePreservingName(javaLogConfigFileName, workingDir, lc.Paths.AbsoluteBaseFolderPath)
if err != nil {
logger.Errorf("%s: error during copying logging.properties file: %s\n", pipelineId, err.Error())
return err
Expand Down Expand Up @@ -194,7 +192,7 @@ func updateJavaLogConfigFile(paths fs_tool.LifeCyclePaths) error {
}

func prepareSbtFiles(lc *fs_tool.LifeCycle, pipelineFolder string, workingDir string) (*fs_tool.LifeCycle, error) {
cmd := exec.Command(shCmd, filepath.Join(workingDir, scioProject))
cmd := exec.Command(bashCmd, filepath.Join(workingDir, scioProject))
cmd.Dir = pipelineFolder
_, err := cmd.Output()
if err != nil {
Expand All @@ -210,30 +208,29 @@ func prepareSbtFiles(lc *fs_tool.LifeCycle, pipelineFolder string, workingDir st
projectFolder, _ := filepath.Abs(filepath.Join(pipelineFolder, scioProjectName))
executableName := lc.Paths.ExecutableName

_, err = exec.Command(rmCmd, filepath.Join(absFileFolderPath, defaultExampleInSbt)).Output()
err = os.Remove(filepath.Join(absFileFolderPath, defaultExampleInSbt))
if err != nil {
return lc, err
}

_, err = exec.Command(cpCmd, filepath.Join(workingDir, scioCommonConstants), absFileFolderPath).Output()
err = utils.CopyFilePreservingName(scioCommonConstants, workingDir, absFileFolderPath)
if err != nil {
return lc, err
}

lc = &fs_tool.LifeCycle{
Paths: fs_tool.LifeCyclePaths{
SourceFileName: fileName,
AbsoluteSourceFileFolderPath: absFileFolderPath,
AbsoluteSourceFilePath: absFilePath,
ExecutableFileName: fileName,
AbsoluteExecutableFileFolderPath: absFileFolderPath,
AbsoluteExecutableFilePath: absFilePath,
AbsoluteBaseFolderPath: absFileFolderPath,
AbsoluteLogFilePath: absLogFilePath,
AbsoluteGraphFilePath: absGraphFilePath,
ProjectDir: projectFolder,
},
}
lc.Paths.ExecutableName = executableName
lc.Paths = fs_tool.LifeCyclePaths{
SourceFileName: fileName,
AbsoluteSourceFileFolderPath: absFileFolderPath,
AbsoluteSourceFilePath: absFilePath,
ExecutableFileName: fileName,
AbsoluteExecutableFileFolderPath: absFileFolderPath,
AbsoluteExecutableFilePath: absFilePath,
AbsoluteBaseFolderPath: absFileFolderPath,
AbsoluteLogFilePath: absLogFilePath,
AbsoluteGraphFilePath: absGraphFilePath,
ProjectDir: projectFolder,
ExecutableName: executableName,
}

return lc, nil
}
Loading