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/18 19:00:25 UTC

[beam] branch master updated: [BEAM-12988] [Playground] Add LifeCycle; Add Java SDK environment of LifeCycle; Fix test description into README;

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 fb6d554  [BEAM-12988] [Playground] Add LifeCycle; Add Java SDK environment of LifeCycle; Fix test description into README;
     new bf91bd7  Merge pull request #15645 from [BEAM-12988] [Playground] Add FileSystem service
fb6d554 is described below

commit fb6d5546e7f29d8104789c490fbca2e735b0f5be
Author: AydarZaynutdinov <ay...@akvelon.com>
AuthorDate: Fri Oct 15 08:24:53 2021 +0300

    [BEAM-12988] [Playground]
    Add LifeCycle;
    Add Java SDK environment of LifeCycle;
    Fix test description into README;
---
 playground/backend/README.md                       |   2 +-
 playground/backend/go.mod                          |   1 +
 playground/backend/internal/fs_tool/fs.go          | 135 +++++++
 playground/backend/internal/fs_tool/fs_test.go     | 415 +++++++++++++++++++++
 playground/backend/internal/fs_tool/java_fs.go     |  48 +++
 .../backend/internal/fs_tool/java_fs_test.go       |  64 ++++
 6 files changed, 664 insertions(+), 1 deletion(-)

diff --git a/playground/backend/README.md b/playground/backend/README.md
index e845e3b..c4a13b2 100644
--- a/playground/backend/README.md
+++ b/playground/backend/README.md
@@ -74,7 +74,7 @@ go build ./cmd/server/server.go
 Playground tests may be run using this command:
 
 ```
-go test ./test/... -v
+go test ... -v
 ```
 
 The full list of commands can be found [here](https://pkg.go.dev/cmd/go).
diff --git a/playground/backend/go.mod b/playground/backend/go.mod
index 7bc8c78..c8d81a0 100644
--- a/playground/backend/go.mod
+++ b/playground/backend/go.mod
@@ -18,6 +18,7 @@ module beam.apache.org/playground/backend
 go 1.16
 
 require (
+	github.com/google/uuid v1.3.0
 	google.golang.org/grpc v1.41.0
 	google.golang.org/protobuf v1.27.1
 )
diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go
new file mode 100644
index 0000000..2e2e9e7
--- /dev/null
+++ b/playground/backend/internal/fs_tool/fs.go
@@ -0,0 +1,135 @@
+// 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 (
+	pb "beam.apache.org/playground/backend/internal/api"
+	"errors"
+	"fmt"
+	"github.com/google/uuid"
+	"io/fs"
+	"os"
+	"path/filepath"
+	"strings"
+)
+
+const (
+	parentBaseFileFolder = "internal"
+	fileMode             = 0600
+)
+
+// Folder contains names of folders with executable and compiled files (/src and /bin for java SDK)
+type Folder struct {
+	BaseFolder       string
+	ExecutableFolder string
+	CompiledFolder   string
+}
+
+// Extension contains executable and compiled files' extensions (.java and .class for java SDK)
+type Extension struct {
+	ExecutableExtension string
+	CompiledExtension   string
+}
+
+type LifeCycle struct {
+	folderGlobs []string
+	Folder      Folder
+	Extension   Extension
+	pipelineId  uuid.UUID
+}
+
+// NewLifeCycle returns a corresponding LifeCycle depending on the given SDK.
+func NewLifeCycle(sdk pb.Sdk, pipelineId uuid.UUID) (*LifeCycle, error) {
+	switch sdk {
+	case pb.Sdk_SDK_JAVA:
+		return newJavaLifeCycle(pipelineId), nil
+	default:
+		return nil, fmt.Errorf("%s isn't supported now", sdk)
+	}
+}
+
+// CreateFolders creates all folders which will be used for code execution.
+func (l *LifeCycle) CreateFolders() error {
+	err := os.Chdir("../..")
+	if err != nil {
+		return err
+	}
+	for _, folder := range l.folderGlobs {
+		err := os.MkdirAll(folder, fs.ModePerm)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// DeleteFolders deletes all previously provisioned folders.
+func (l *LifeCycle) DeleteFolders() error {
+	for _, folder := range l.folderGlobs {
+		err := os.RemoveAll(folder)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// CreateExecutableFile creates an executable file (i.e. file.java for the Java SDK).
+func (l *LifeCycle) CreateExecutableFile(code string) (string, error) {
+	if _, err := os.Stat(l.Folder.ExecutableFolder); os.IsNotExist(err) {
+		return "", err
+	}
+
+	fileName := getFileName(l.pipelineId, l.Extension.ExecutableExtension)
+	filePath := filepath.Join(l.Folder.ExecutableFolder, fileName)
+	err := os.WriteFile(filePath, []byte(code), fileMode)
+	if err != nil {
+		return "", err
+	}
+	return fileName, nil
+}
+
+// GetAbsoluteExecutableFilePath returns absolute filepath to executable file (.../internal/executable_files/src/file.java for java SDK).
+func (l *LifeCycle) GetAbsoluteExecutableFilePath() string {
+	fileName := getFileName(l.pipelineId, l.Extension.ExecutableExtension)
+	filePath := filepath.Join(l.Folder.ExecutableFolder, fileName)
+	absoluteFilePath, _ := filepath.Abs(filePath)
+	return absoluteFilePath
+}
+
+// GetRelativeExecutableFilePath returns relative filepath to executable file (src/file.java for java SDK).
+func (l *LifeCycle) GetRelativeExecutableFilePath() string {
+	fileName := getFileName(l.pipelineId, l.Extension.ExecutableExtension)
+	filePath := filepath.Join(l.Folder.ExecutableFolder, fileName)
+	return filePath[len(l.Folder.BaseFolder)+1:]
+}
+
+// GetExecutableName returns name that should be executed (HelloWorld for HelloWorld.class for java SDK)
+func (l *LifeCycle) GetExecutableName() (string, error) {
+	dirEntries, err := os.ReadDir(l.Folder.CompiledFolder)
+	if err != nil {
+		return "", err
+	}
+	if len(dirEntries) != 1 {
+		return "", errors.New("number of executable files should be equal to one")
+	}
+	return strings.Split(dirEntries[0].Name(), ".")[0], nil
+}
+
+// getFileName returns fileName by pipelineId and fileType (pipelineId.java for java SDK).
+func getFileName(pipelineId uuid.UUID, fileType string) string {
+	return fmt.Sprintf("%s.%s", pipelineId, fileType)
+}
diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go
new file mode 100644
index 0000000..7936ad2
--- /dev/null
+++ b/playground/backend/internal/fs_tool/fs_test.go
@@ -0,0 +1,415 @@
+// 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 (
+	pb "beam.apache.org/playground/backend/internal/api"
+	"fmt"
+	"github.com/google/uuid"
+	"io/fs"
+	"os"
+	"path/filepath"
+	"reflect"
+	"testing"
+)
+
+func TestLifeCycle_CreateExecutableFile(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+	binFileFolder := baseFileFolder + "/bin"
+
+	type fields struct {
+		folderGlobs []string
+		folder      Folder
+		extension   Extension
+		pipelineId  uuid.UUID
+	}
+	type args struct {
+		code string
+	}
+	tests := []struct {
+		name          string
+		createFolders []string
+		fields        fields
+		args          args
+		want          string
+		wantErr       bool
+	}{
+		{
+			name: "executable folder doesn't exist",
+			fields: fields{
+				folder: Folder{
+					ExecutableFolder: srcFileFolder,
+					CompiledFolder:   binFileFolder,
+				},
+				pipelineId: pipelineId,
+			},
+			args:    args{},
+			want:    "",
+			wantErr: true,
+		},
+		{
+			name:          "executable folder exists",
+			createFolders: []string{srcFileFolder},
+			fields: fields{
+				folder:     Folder{ExecutableFolder: srcFileFolder},
+				extension:  Extension{ExecutableExtension: javaExecutableFileExtension},
+				pipelineId: pipelineId,
+			},
+			args:    args{code: "TEST_CODE"},
+			want:    fmt.Sprintf("%s.%s", pipelineId, javaExecutableFileExtension),
+			wantErr: false,
+		},
+	}
+	for _, tt := range tests {
+		for _, folder := range tt.createFolders {
+			os.MkdirAll(folder, fs.ModePerm)
+		}
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.folder,
+				Extension:   tt.fields.extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			got, err := l.CreateExecutableFile(tt.args.code)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("CreateExecutableFile() error = %v, wantErr %v", err, tt.wantErr)
+				return
+			}
+			if got != tt.want {
+				t.Errorf("CreateExecutableFile() got = %v, want %v", got, tt.want)
+			}
+		})
+		os.RemoveAll(parentBaseFileFolder)
+	}
+}
+
+func TestLifeCycle_CreateFolders(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+
+	type fields struct {
+		folderGlobs []string
+		folder      Folder
+		extension   Extension
+		pipelineId  uuid.UUID
+	}
+	tests := []struct {
+		name    string
+		fields  fields
+		wantErr bool
+	}{
+		{
+			name:    "CreateFolders",
+			fields:  fields{folderGlobs: []string{baseFileFolder}},
+			wantErr: false,
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.folder,
+				Extension:   tt.fields.extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			if err := l.CreateFolders(); (err != nil) != tt.wantErr {
+				t.Errorf("CreateFolders() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			for _, folder := range tt.fields.folderGlobs {
+				if _, err := os.Stat(folder); os.IsNotExist(err) {
+					t.Errorf("CreateFolders() should create folder %s, but it dosn't", folder)
+				}
+			}
+		})
+		os.RemoveAll(baseFileFolder)
+	}
+}
+
+func TestLifeCycle_DeleteFolders(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+
+	type fields struct {
+		folderGlobs []string
+		folder      Folder
+		extension   Extension
+		pipelineId  uuid.UUID
+	}
+	tests := []struct {
+		name    string
+		fields  fields
+		wantErr bool
+	}{
+		{
+			name: "DeleteFolders",
+			fields: fields{
+				folderGlobs: []string{baseFileFolder},
+				pipelineId:  pipelineId,
+			},
+			wantErr: false,
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.folder,
+				Extension:   tt.fields.extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			if err := l.DeleteFolders(); (err != nil) != tt.wantErr {
+				t.Errorf("DeleteFolders() error = %v, wantErr %v", err, tt.wantErr)
+			}
+		})
+	}
+}
+
+func TestNewLifeCycle(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+	binFileFolder := baseFileFolder + "/bin"
+
+	type args struct {
+		sdk        pb.Sdk
+		pipelineId uuid.UUID
+	}
+	tests := []struct {
+		name    string
+		args    args
+		want    *LifeCycle
+		wantErr bool
+	}{
+		{
+			name: "Available SDK",
+			args: args{
+				sdk:        pb.Sdk_SDK_JAVA,
+				pipelineId: pipelineId,
+			},
+			want: &LifeCycle{
+				folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder},
+				Folder: Folder{
+					BaseFolder:       baseFileFolder,
+					ExecutableFolder: srcFileFolder,
+					CompiledFolder:   binFileFolder,
+				},
+				Extension: Extension{
+					ExecutableExtension: javaExecutableFileExtension,
+					CompiledExtension:   javaCompiledFileExtension,
+				},
+				pipelineId: pipelineId,
+			},
+			wantErr: false,
+		},
+		{
+			name: "Unavailable SDK",
+			args: args{
+				sdk:        pb.Sdk_SDK_UNSPECIFIED,
+				pipelineId: pipelineId,
+			},
+			want:    nil,
+			wantErr: true,
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			got, err := NewLifeCycle(tt.args.sdk, tt.args.pipelineId)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("NewLifeCycle() error = %v, wantErr %v", err, tt.wantErr)
+				return
+			}
+			if !reflect.DeepEqual(got, tt.want) {
+				t.Errorf("NewLifeCycle() got = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func Test_getFileName(t *testing.T) {
+	pipelineId := uuid.New()
+	type args struct {
+		pipelineId uuid.UUID
+		fileType   string
+	}
+	tests := []struct {
+		name string
+		args args
+		want string
+	}{
+		{
+			name: "getFileName",
+			args: args{
+				pipelineId: pipelineId,
+				fileType:   javaExecutableFileExtension,
+			},
+			want: fmt.Sprintf("%s.%s", pipelineId, javaExecutableFileExtension),
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			if got := getFileName(tt.args.pipelineId, tt.args.fileType); got != tt.want {
+				t.Errorf("getFileName() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func TestLifeCycle_GetRelativeExecutableFilePath(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+
+	type fields struct {
+		folderGlobs []string
+		Folder      Folder
+		Extension   Extension
+		pipelineId  uuid.UUID
+	}
+	tests := []struct {
+		name   string
+		fields fields
+		want   string
+	}{
+		{
+			name: "GetRelativeExecutableFilePath",
+			fields: fields{
+				Folder: Folder{
+					BaseFolder:       baseFileFolder,
+					ExecutableFolder: srcFileFolder,
+				},
+				Extension:  Extension{ExecutableExtension: javaExecutableFileExtension},
+				pipelineId: pipelineId,
+			},
+			want: fmt.Sprintf("%s/%s.%s", srcFileFolder[len(baseFileFolder)+1:], pipelineId, javaExecutableFileExtension),
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.Folder,
+				Extension:   tt.fields.Extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			if got := l.GetRelativeExecutableFilePath(); got != tt.want {
+				t.Errorf("GetExecutableFilePath() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func TestLifeCycle_GetAbsoluteExecutableFilePath(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+
+	filePath := fmt.Sprintf("%s/%s.%s", srcFileFolder, pipelineId, javaExecutableFileExtension)
+	absolutePath, _ := filepath.Abs(filePath)
+	type fields struct {
+		folderGlobs []string
+		Folder      Folder
+		Extension   Extension
+		pipelineId  uuid.UUID
+	}
+	tests := []struct {
+		name    string
+		fields  fields
+		want    string
+		wantErr bool
+	}{
+		{
+			name: "GetAbsoluteExecutableFilePath",
+			fields: fields{
+				Folder: Folder{
+					BaseFolder:       baseFileFolder,
+					ExecutableFolder: srcFileFolder,
+				},
+				Extension:  Extension{ExecutableExtension: javaExecutableFileExtension},
+				pipelineId: pipelineId,
+			},
+			want: absolutePath,
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.Folder,
+				Extension:   tt.fields.Extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			got := l.GetAbsoluteExecutableFilePath()
+			if got != tt.want {
+				t.Errorf("GetAbsoluteExecutableFilePath() got = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
+
+func TestLifeCycle_GetExecutableName(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	binFileFolder := baseFileFolder + "/bin"
+
+	type fields struct {
+		folderGlobs []string
+		Folder      Folder
+		Extension   Extension
+		pipelineId  uuid.UUID
+	}
+	tests := []struct {
+		name   string
+		fields fields
+		want   string
+	}{
+		{
+			name: "GetExecutableName",
+			fields: fields{
+				Folder: Folder{
+					BaseFolder:     baseFileFolder,
+					CompiledFolder: binFileFolder,
+				},
+				pipelineId:  pipelineId,
+				folderGlobs: []string{baseFileFolder, binFileFolder},
+			},
+			want: pipelineId.String(),
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			l := &LifeCycle{
+				folderGlobs: tt.fields.folderGlobs,
+				Folder:      tt.fields.Folder,
+				Extension:   tt.fields.Extension,
+				pipelineId:  tt.fields.pipelineId,
+			}
+			if err := l.CreateFolders(); err != nil {
+				t.Errorf("CreateFolders() error = %v", err)
+			}
+			_, err := os.Create(binFileFolder + "/" + pipelineId.String() + ".class")
+			if err != nil {
+				t.Errorf("Unable to write file: %v", err)
+			}
+			got, err := l.GetExecutableName()
+			if got != tt.want {
+				t.Errorf("GetExecutableName() got = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
diff --git a/playground/backend/internal/fs_tool/java_fs.go b/playground/backend/internal/fs_tool/java_fs.go
new file mode 100644
index 0000000..3e0b1d5
--- /dev/null
+++ b/playground/backend/internal/fs_tool/java_fs.go
@@ -0,0 +1,48 @@
+// 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"
+)
+
+const (
+	javaBaseFileFolder          = parentBaseFileFolder + "/executable_files"
+	javaExecutableFileExtension = "java"
+	javaCompiledFileExtension   = "class"
+)
+
+// newJavaLifeCycle creates LifeCycle with java SDK environment.
+func newJavaLifeCycle(pipelineId uuid.UUID) *LifeCycle {
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+	binFileFolder := baseFileFolder + "/bin"
+
+	return &LifeCycle{
+		folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder},
+		Folder: Folder{
+			BaseFolder:       baseFileFolder,
+			ExecutableFolder: srcFileFolder,
+			CompiledFolder:   binFileFolder,
+		},
+		Extension: Extension{
+			ExecutableExtension: javaExecutableFileExtension,
+			CompiledExtension:   javaCompiledFileExtension,
+		},
+		pipelineId: pipelineId,
+	}
+}
diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go
new file mode 100644
index 0000000..8933bb1
--- /dev/null
+++ b/playground/backend/internal/fs_tool/java_fs_test.go
@@ -0,0 +1,64 @@
+// 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_newJavaLifeCycle(t *testing.T) {
+	pipelineId := uuid.New()
+	baseFileFolder := fmt.Sprintf("%s_%s", javaBaseFileFolder, pipelineId)
+	srcFileFolder := baseFileFolder + "/src"
+	binFileFolder := baseFileFolder + "/bin"
+
+	type args struct {
+		pipelineId uuid.UUID
+	}
+	tests := []struct {
+		name string
+		args args
+		want *LifeCycle
+	}{
+		{
+			name: "newJavaLifeCycle",
+			args: args{pipelineId: pipelineId},
+			want: &LifeCycle{
+				folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder},
+				Folder: Folder{
+					BaseFolder:       baseFileFolder,
+					ExecutableFolder: srcFileFolder,
+					CompiledFolder:   binFileFolder,
+				},
+				Extension: Extension{
+					ExecutableExtension: javaExecutableFileExtension,
+					CompiledExtension:   javaCompiledFileExtension,
+				},
+				pipelineId: pipelineId,
+			},
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			if got := newJavaLifeCycle(tt.args.pipelineId); !reflect.DeepEqual(got, tt.want) {
+				t.Errorf("newJavaLifeCycle() = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}