Skip to content

Commit

Permalink
[Playground] Resolve issue with SCIO examples failing on start due to…
Browse files Browse the repository at this point in the history
… a timeout (#24946)

* Initialize SBT cache during SCIO playground container build

Run sbt tool during container build to let it download Scala dependencies form Maven during build time instead of having to wait for downloading all dependencies during first run of examples in container

* Fix issue with reading of GRPC_TIMEOUT environment variables in CI/CD scripts

* Fix cleanup of execution environment for Scala examples

* Fix panic in preparers when an empty file is passed

* Use better name for SCIO project directory

* Run "sbt compile" during container build to fetch all Scala dependencies

* Disable forking JVM in SBT to significantly reduce memory usage

* Impose memory limits on local deployments of SCIO runner container to better imitate real deployments

* Fine-tune Java GC to improve performance and memory usage of SCIO examples

* Remove large blobs of text from common_test.go

* Add `sbt` to the list of development dependencies

* Clarify running of backend tests in Playground

* Clarify local running of backend

* Improve consistency in code blocks in backend Readme

* Fixing trailing whitespace

* Update playground/backend/README.md

Co-authored-by: Danny McCormick <dannymccormick@google.com>

* Update playground/backend/internal/utils/preparers_utils_test.go

Co-authored-by: Danny McCormick <dannymccormick@google.com>

---------

Co-authored-by: Danny McCormick <dannymccormick@google.com>
  • Loading branch information
TSultanov and damccorm authored Mar 23, 2023
1 parent 545cadc commit 92abdf6
Show file tree
Hide file tree
Showing 16 changed files with 384 additions and 184 deletions.
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"

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

0 comments on commit 92abdf6

Please sign in to comment.