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 2022/09/07 15:44:56 UTC

[beam] branch master updated: [Playground] [Backend] Datastore queries and mappers to get examples (#22955)

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 0d937d4cd72 [Playground] [Backend] Datastore queries and mappers to get examples (#22955)
0d937d4cd72 is described below

commit 0d937d4cd725965572d4720811fa2d6efaa8edf8
Author: Vladislav Chunikhin <10...@users.noreply.github.com>
AuthorDate: Wed Sep 7 18:44:48 2022 +0300

    [Playground] [Backend] Datastore queries and mappers to get examples (#22955)
    
    * [Playground] [Backend] added the datastore query to get an example catalog
    
    * [Playground] [Backend] updated complexity field
    
    * [Playground] [Backend] fixed integration tests for datastore
    
    * [Playground] [Backend] added testing data cleaner
    
    * [Playground] [Backend] edited getting datastore key
    
    * [Playground] [Infrastructure] added the datastore client
    
    * [Playground] [Infrastructure] added the type property to an example entity
    
    * [Playground] [Infrastructure] removed the cloud storage client
    
    * [Playground] [Infrastructure] changed validation examples for duplicates by name
    
    * [Playground] [Infrastructure] fixed duplicated example names
    
    * [Playground] [Infrastructure] added the google cloud project id as an env variable
    
    * [Playground] [Backend] updated docker files for runners
    
    * [Playground] [Backend] added SDK validation to save a code snippet
    
    * [Playground] [Backend] removed comments
    
    * [Playground] [Backend] resolved comments after review
    
    * [Playground] [Infrastructure] updated dependency versions
    
    * [Playground] [Backend] fixed the java runner for examples with graphs
    
    * [Playground] [Infrastructure] added loading catalogs to the cloud datastore
    
    * [Playground] [Backend] remove extra imports after merging
    
    * [Playground] [Backend] fixes after merging with other branches
    
    Co-authored-by: oborysevych <ol...@akvelon.com>
---
 playground/backend/cmd/remove_unused_snippets.go   |   4 +-
 playground/backend/cmd/server/controller_test.go   |  53 ++--
 playground/backend/cmd/server/server.go            |   8 +-
 .../mapper.go => constants/datastore_constants.go} |  33 ++-
 .../backend/internal/db/datastore/datastore_db.go  | 136 +++++++---
 .../internal/db/datastore/datastore_db_test.go     | 301 +++++++++++++--------
 playground/backend/internal/db/db.go               |   8 +-
 .../backend/internal/db/dto/precompiled_object.go  |  82 ++++++
 .../db_utils_test.go => db/entity/example.go}      |  51 ++--
 .../backend/internal/db/entity/snippet_test.go     |   9 +-
 .../backend/internal/db/mapper/datastore_mapper.go |  14 +-
 .../internal/db/mapper/datastore_mapper_test.go    |  14 +-
 playground/backend/internal/db/mapper/mapper.go    |   7 +
 .../db/mapper/precompiled_object_mapper.go         | 119 ++++++++
 .../db/mapper/precompiled_object_mapper_test.go    | 178 ++++++++++++
 .../db/schema/migration/migration_v001_test.go     |  21 +-
 .../internal/tests/test_cleaner/test_cleaner.go    |  73 +++++
 .../backend/internal/utils/datastore_utils.go      | 132 +++++++++
 .../backend/internal/utils/datastore_utils_test.go | 206 ++++++++++++++
 playground/backend/internal/utils/db_utils.go      |  52 ----
 playground/sdks-emulator.yaml                      |  28 ++
 21 files changed, 1230 insertions(+), 299 deletions(-)

diff --git a/playground/backend/cmd/remove_unused_snippets.go b/playground/backend/cmd/remove_unused_snippets.go
index ab875f3f400..f78a9a59dd5 100644
--- a/playground/backend/cmd/remove_unused_snippets.go
+++ b/playground/backend/cmd/remove_unused_snippets.go
@@ -23,6 +23,7 @@ import (
 	"time"
 
 	"beam.apache.org/playground/backend/internal/db/datastore"
+	"beam.apache.org/playground/backend/internal/db/mapper"
 )
 
 func main() {
@@ -42,7 +43,8 @@ func main() {
 	}
 
 	ctx := context.Background()
-	db, err := datastore.New(ctx, projectId)
+	pcMapper := mapper.NewPrecompiledObjectMapper()
+	db, err := datastore.New(ctx, pcMapper, projectId)
 	if err != nil {
 		fmt.Printf("Couldn't create the database client, err: %s \n", err.Error())
 		return
diff --git a/playground/backend/cmd/server/controller_test.go b/playground/backend/cmd/server/controller_test.go
index 84b2ac808c8..84bde110a5e 100644
--- a/playground/backend/cmd/server/controller_test.go
+++ b/playground/backend/cmd/server/controller_test.go
@@ -36,6 +36,7 @@ import (
 	pb "beam.apache.org/playground/backend/internal/api/v1"
 	"beam.apache.org/playground/backend/internal/cache"
 	"beam.apache.org/playground/backend/internal/cache/local"
+	"beam.apache.org/playground/backend/internal/constants"
 	"beam.apache.org/playground/backend/internal/db"
 	datastoreDb "beam.apache.org/playground/backend/internal/db/datastore"
 	"beam.apache.org/playground/backend/internal/db/entity"
@@ -43,18 +44,16 @@ import (
 	"beam.apache.org/playground/backend/internal/db/schema"
 	"beam.apache.org/playground/backend/internal/db/schema/migration"
 	"beam.apache.org/playground/backend/internal/environment"
+	"beam.apache.org/playground/backend/internal/tests/test_cleaner"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
 const (
-	bufSize                    = 1024 * 1024
-	javaConfig                 = "{\n  \"compile_cmd\": \"javac\",\n  \"run_cmd\": \"java\",\n  \"test_cmd\": \"java\",\n  \"compile_args\": [\n    \"-d\",\n    \"bin\",\n    \"-classpath\"\n  ],\n  \"run_args\": [\n    \"-cp\",\n    \"bin:\"\n  ],\n  \"test_args\": [\n    \"-cp\",\n    \"bin:\",\n    \"JUnit\"\n  ]\n}"
-	javaLogConfigFilename      = "logging.properties"
-	baseFileFolder             = "executable_files"
-	configFolder               = "configs"
-	datastoreEmulatorHostKey   = "DATASTORE_EMULATOR_HOST"
-	datastoreEmulatorHostValue = "127.0.0.1:8888"
-	datastoreEmulatorProjectId = "test"
+	bufSize               = 1024 * 1024
+	javaConfig            = "{\n  \"compile_cmd\": \"javac\",\n  \"run_cmd\": \"java\",\n  \"test_cmd\": \"java\",\n  \"compile_args\": [\n    \"-d\",\n    \"bin\",\n    \"-classpath\"\n  ],\n  \"run_args\": [\n    \"-cp\",\n    \"bin:\"\n  ],\n  \"test_args\": [\n    \"-cp\",\n    \"bin:\",\n    \"JUnit\"\n  ]\n}"
+	javaLogConfigFilename = "logging.properties"
+	baseFileFolder        = "executable_files"
+	configFolder          = "configs"
 )
 
 var lis *bufconn.Listener
@@ -97,13 +96,13 @@ func setup() *grpc.Server {
 	cacheService = local.New(ctx)
 
 	// setup database
-	datastoreEmulatorHost := os.Getenv(datastoreEmulatorHostKey)
+	datastoreEmulatorHost := os.Getenv(constants.EmulatorHostKey)
 	if datastoreEmulatorHost == "" {
-		if err = os.Setenv(datastoreEmulatorHostKey, datastoreEmulatorHostValue); err != nil {
+		if err = os.Setenv(constants.EmulatorHostKey, constants.EmulatorHostValue); err != nil {
 			panic(err)
 		}
 	}
-	dbClient, err = datastoreDb.New(ctx, datastoreEmulatorProjectId)
+	dbClient, err = datastoreDb.New(ctx, mapper.NewPrecompiledObjectMapper(), constants.EmulatorProjectId)
 	if err != nil {
 		panic(err)
 	}
@@ -119,12 +118,15 @@ func setup() *grpc.Server {
 	if err = os.Setenv("APP_WORK_DIR", path); err != nil {
 		panic(err)
 	}
-	if err = os.Setenv("SDK_CONFIG", "../../../sdks.yaml"); err != nil {
+	if err = os.Setenv("SDK_CONFIG", "../../../sdks-emulator.yaml"); err != nil {
 		panic(err)
 	}
 	if err = os.Setenv("PROPERTY_PATH", "../../."); err != nil {
 		panic(err)
 	}
+	if err = os.Setenv(constants.DatastoreNamespaceKey, "main"); err != nil {
+		panic(err)
+	}
 
 	networkEnv, err := environment.GetNetworkEnvsFromOsEnvs()
 	if err != nil {
@@ -156,7 +158,7 @@ func setup() *grpc.Server {
 	}
 	appEnv.SetSchemaVersion(actualSchemaVersion)
 
-	entityMapper := mapper.New(appEnv, props)
+	entityMapper := mapper.NewDatastoreMapper(ctx, appEnv, props)
 
 	pb.RegisterPlaygroundServiceServer(s, &playgroundController{
 		env:          environment.NewEnvironment(*networkEnv, *sdkEnv, *appEnv),
@@ -866,6 +868,8 @@ func TestPlaygroundController_SaveSnippet(t *testing.T) {
 				if len(got.Id) != 11 {
 					t.Errorf("PlaygroundController_SaveSnippet() unexpected generated ID")
 				}
+				test_cleaner.CleanFiles(ctx, t, got.Id, 1)
+				test_cleaner.CleanSnippet(ctx, t, got.Id)
 			}
 		})
 	}
@@ -882,10 +886,11 @@ func TestPlaygroundController_GetSnippet(t *testing.T) {
 		info *pb.GetSnippetRequest
 	}
 	tests := []struct {
-		name    string
-		args    args
-		prepare func()
-		wantErr bool
+		name      string
+		args      args
+		prepare   func()
+		wantErr   bool
+		cleanData func()
 	}{
 		// Test case with calling GetSnippet method with ID that is not in the database.
 		// As a result, want to receive an error.
@@ -895,8 +900,9 @@ func TestPlaygroundController_GetSnippet(t *testing.T) {
 				ctx:  ctx,
 				info: &pb.GetSnippetRequest{Id: "MOCK_ID_G"},
 			},
-			prepare: func() {},
-			wantErr: true,
+			prepare:   func() {},
+			wantErr:   true,
+			cleanData: func() {},
 		},
 		// Test case with calling GetSnippet method with a correct ID.
 		// As a result, want to receive a snippet entity.
@@ -911,10 +917,10 @@ func TestPlaygroundController_GetSnippet(t *testing.T) {
 					&entity.Snippet{
 						Snippet: &entity.SnippetEntity{
 							OwnerId:       "",
-							Sdk:           utils.GetNameKey(datastoreDb.SdkKind, pb.Sdk_SDK_JAVA.String(), datastoreDb.Namespace, nil),
+							Sdk:           utils.GetSdkKey(ctx, pb.Sdk_SDK_JAVA.String()),
 							PipeOpts:      "MOCK_OPTIONS",
 							Created:       nowDate,
-							Origin:        "PG_USER",
+							Origin:        constants.UserSnippetOrigin,
 							NumberOfFiles: 1,
 						},
 						Files: []*entity.FileEntity{{
@@ -925,6 +931,10 @@ func TestPlaygroundController_GetSnippet(t *testing.T) {
 				)
 			},
 			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanFiles(ctx, t, "MOCK_ID", 1)
+				test_cleaner.CleanSnippet(ctx, t, "MOCK_ID")
+			},
 		},
 	}
 
@@ -940,6 +950,7 @@ func TestPlaygroundController_GetSnippet(t *testing.T) {
 					t.Errorf("PlaygroundController_GetSnippet() unexpected response")
 				}
 			}
+			tt.cleanData()
 		})
 	}
 }
diff --git a/playground/backend/cmd/server/server.go b/playground/backend/cmd/server/server.go
index 1c1c5f040a2..aa89b4ed03a 100644
--- a/playground/backend/cmd/server/server.go
+++ b/playground/backend/cmd/server/server.go
@@ -22,8 +22,6 @@ import (
 	"github.com/improbable-eng/grpc-web/go/grpcweb"
 	"google.golang.org/grpc"
 
-	"beam.apache.org/playground/backend/internal/tasks"
-
 	pb "beam.apache.org/playground/backend/internal/api/v1"
 	"beam.apache.org/playground/backend/internal/cache"
 	"beam.apache.org/playground/backend/internal/cache/local"
@@ -36,6 +34,7 @@ import (
 	"beam.apache.org/playground/backend/internal/db/schema/migration"
 	"beam.apache.org/playground/backend/internal/environment"
 	"beam.apache.org/playground/backend/internal/logger"
+	"beam.apache.org/playground/backend/internal/tasks"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
@@ -75,7 +74,7 @@ func runServer() error {
 			return err
 		}
 
-		dbClient, err = datastore.New(ctx, envService.ApplicationEnvs.GoogleProjectId())
+		dbClient, err = datastore.New(ctx, mapper.NewPrecompiledObjectMapper(), envService.ApplicationEnvs.GoogleProjectId())
 		if err != nil {
 			return err
 		}
@@ -84,7 +83,7 @@ func runServer() error {
 			return err
 		}
 
-		entityMapper = mapper.New(&envService.ApplicationEnvs, props)
+		entityMapper = mapper.NewDatastoreMapper(ctx, &envService.ApplicationEnvs, props)
 
 		// Since only router server has the scheduled task, the task creation is here
 		scheduledTasks := tasks.New(ctx)
@@ -122,6 +121,7 @@ func runServer() error {
 	}
 }
 
+// setupEnvironment constructs the environment required by the app
 func setupEnvironment() (*environment.Environment, error) {
 	networkEnvs, err := environment.GetNetworkEnvsFromOsEnvs()
 	if err != nil {
diff --git a/playground/backend/internal/db/mapper/mapper.go b/playground/backend/internal/constants/datastore_constants.go
similarity index 55%
copy from playground/backend/internal/db/mapper/mapper.go
copy to playground/backend/internal/constants/datastore_constants.go
index ddc22434e8f..61b72800c4c 100644
--- a/playground/backend/internal/db/mapper/mapper.go
+++ b/playground/backend/internal/constants/datastore_constants.go
@@ -13,14 +13,29 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package mapper
+package constants
 
-import (
-	pb "beam.apache.org/playground/backend/internal/api/v1"
-	"beam.apache.org/playground/backend/internal/db/entity"
-)
+//Cloud Datastore constants
+const (
+	Namespace             = "Playground"
+	DatastoreNamespaceKey = "DATASTORE_NAMESPACE"
+	IDDelimiter           = "_"
+	CloudPathDelimiter    = "/"
+	UserSnippetOrigin     = "PG_USER"
+	ExampleOrigin         = "PG_EXAMPLES"
+
+	SnippetKind  = "pg_snippets"
+	SchemaKind   = "pg_schema_versions"
+	SdkKind      = "pg_sdks"
+	FileKind     = "pg_files"
+	ExampleKind  = "pg_examples"
+	PCObjectKind = "pg_pc_objects"
 
-type EntityMapper interface {
-	ToSnippet(info *pb.SaveSnippetRequest) *entity.Snippet
-	ToFileEntity(info *pb.SaveSnippetRequest, file *pb.SnippetFile) (*entity.FileEntity, error)
-}
+	PCOutputType = "OUTPUT"
+	PCLogType    = "LOG"
+	PCGraphType  = "GRAPH"
+
+	EmulatorHostKey   = "DATASTORE_EMULATOR_HOST"
+	EmulatorHostValue = "127.0.0.1:8888"
+	EmulatorProjectId = "test"
+)
diff --git a/playground/backend/internal/db/datastore/datastore_db.go b/playground/backend/internal/db/datastore/datastore_db.go
index b31882d8553..1abde270ab5 100644
--- a/playground/backend/internal/db/datastore/datastore_db.go
+++ b/playground/backend/internal/db/datastore/datastore_db.go
@@ -17,38 +17,38 @@ package datastore
 
 import (
 	"context"
-	"fmt"
 	"time"
 
-	"beam.apache.org/playground/backend/internal/db/dto"
 	"cloud.google.com/go/datastore"
 
+	"beam.apache.org/playground/backend/internal/constants"
+	"beam.apache.org/playground/backend/internal/db/dto"
+
+	pb "beam.apache.org/playground/backend/internal/api/v1"
 	"beam.apache.org/playground/backend/internal/db/entity"
+	"beam.apache.org/playground/backend/internal/db/mapper"
 	"beam.apache.org/playground/backend/internal/logger"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
 const (
-	Namespace = "Playground"
-
-	SnippetKind = "pg_snippets"
-	SchemaKind  = "pg_schema_versions"
-	SdkKind     = "pg_sdks"
-	FileKind    = "pg_files"
+	errorMsgTemplateCreatingTx = "error during creating transaction, err: %s\n"
+	errorMsgTemplateTxRollback = "error during transaction rollback, err: %s\n"
+	errorMsgTemplateTxCommit   = "error during transaction commit, err: %s\n"
 )
 
 type Datastore struct {
-	Client *datastore.Client
+	Client         *datastore.Client
+	ResponseMapper mapper.ResponseMapper
 }
 
-func New(ctx context.Context, projectId string) (*Datastore, error) {
+func New(ctx context.Context, responseMapper mapper.ResponseMapper, projectId string) (*Datastore, error) {
 	client, err := datastore.NewClient(ctx, projectId)
 	if err != nil {
 		logger.Errorf("Datastore: connection to store: error during connection, err: %s\n", err.Error())
 		return nil, err
 	}
-
-	return &Datastore{Client: client}, nil
+	return &Datastore{Client: client, ResponseMapper: responseMapper}, nil
 }
 
 // PutSnippet puts the snippet entity to datastore
@@ -57,7 +57,7 @@ func (d *Datastore) PutSnippet(ctx context.Context, snipId string, snip *entity.
 		logger.Errorf("Datastore: PutSnippet(): snippet is nil")
 		return nil
 	}
-	snipKey := utils.GetNameKey(SnippetKind, snipId, Namespace, nil)
+	snipKey := utils.GetSnippetKey(ctx, snipId)
 	tx, err := d.Client.NewTransaction(ctx)
 	if err != nil {
 		logger.Errorf("Datastore: PutSnippet(): error during the transaction creating, err: %s\n", err.Error())
@@ -73,8 +73,7 @@ func (d *Datastore) PutSnippet(ctx context.Context, snipId string, snip *entity.
 
 	var fileKeys []*datastore.Key
 	for index := range snip.Files {
-		fileId := fmt.Sprintf("%s_%d", snipId, index)
-		fileKeys = append(fileKeys, utils.GetNameKey(FileKind, fileId, Namespace, nil))
+		fileKeys = append(fileKeys, utils.GetFileKey(ctx, snipId, index))
 	}
 
 	if _, err = tx.PutMulti(fileKeys, snip.Files); err != nil {
@@ -86,7 +85,7 @@ func (d *Datastore) PutSnippet(ctx context.Context, snipId string, snip *entity.
 	}
 
 	if _, err = tx.Commit(); err != nil {
-		logger.Errorf("Datastore: PutSnippet(): error during the transaction committing, err: %s\n", err.Error())
+		logger.Errorf(errorMsgTemplateTxCommit, err.Error())
 		return err
 	}
 
@@ -95,11 +94,11 @@ func (d *Datastore) PutSnippet(ctx context.Context, snipId string, snip *entity.
 
 // GetSnippet returns the snippet entity by identifier
 func (d *Datastore) GetSnippet(ctx context.Context, id string) (*entity.SnippetEntity, error) {
-	key := utils.GetNameKey(SnippetKind, id, Namespace, nil)
+	key := utils.GetSnippetKey(ctx, id)
 	snip := new(entity.SnippetEntity)
 	tx, err := d.Client.NewTransaction(ctx)
 	if err != nil {
-		logger.Errorf("Datastore: GetSnippet(): error during the transaction creating, err: %s\n", err.Error())
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
 		return nil, err
 	}
 	if err = tx.Get(key, snip); err != nil {
@@ -119,7 +118,7 @@ func (d *Datastore) GetSnippet(ctx context.Context, id string) (*entity.SnippetE
 		return nil, err
 	}
 	if _, err = tx.Commit(); err != nil {
-		logger.Errorf("Datastore: GetSnippet(): error during the transaction committing, err: %s\n", err.Error())
+		logger.Errorf(errorMsgTemplateTxCommit, err.Error())
 		return nil, err
 	}
 	return snip, nil
@@ -131,7 +130,7 @@ func (d *Datastore) PutSchemaVersion(ctx context.Context, id string, schema *ent
 		logger.Errorf("Datastore: PutSchemaVersion(): schema version is nil")
 		return nil
 	}
-	key := utils.GetNameKey(SchemaKind, id, Namespace, nil)
+	key := utils.GetSchemaVerKey(ctx, id)
 	if _, err := d.Client.Put(ctx, key, schema); err != nil {
 		logger.Errorf("Datastore: PutSchemaVersion(): error during entity saving, err: %s\n", err.Error())
 		return err
@@ -147,7 +146,7 @@ func (d *Datastore) PutSDKs(ctx context.Context, sdks []*entity.SDKEntity) error
 	}
 	var keys []*datastore.Key
 	for _, sdk := range sdks {
-		keys = append(keys, utils.GetNameKey(SdkKind, sdk.Name, Namespace, nil))
+		keys = append(keys, utils.GetSdkKey(ctx, sdk.Name))
 	}
 	if _, err := d.Client.PutMulti(ctx, keys, sdks); err != nil {
 		logger.Errorf("Datastore: PutSDK(): error during entity saving, err: %s\n", err.Error())
@@ -163,49 +162,95 @@ func (d *Datastore) GetFiles(ctx context.Context, snipId string, numberOfFiles i
 		return []*entity.FileEntity{}, nil
 	}
 	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	defer rollback(tx)
+
 	if err != nil {
 		logger.Errorf("Datastore: GetFiles(): error during the transaction creating, err: %s\n", err.Error())
 		return nil, err
 	}
 	var fileKeys []*datastore.Key
-	for i := 0; i < numberOfFiles; i++ {
-		fileId := fmt.Sprintf("%s_%d", snipId, i)
-		fileKeys = append(fileKeys, utils.GetNameKey(FileKind, fileId, Namespace, nil))
+	for fileIndx := 0; fileIndx < numberOfFiles; fileIndx++ {
+		fileKeys = append(fileKeys, utils.GetFileKey(ctx, snipId, fileIndx))
 	}
 	var files = make([]*entity.FileEntity, numberOfFiles)
 	if err = tx.GetMulti(fileKeys, files); err != nil {
-		if rollBackErr := tx.Rollback(); rollBackErr != nil {
-			err = rollBackErr
-		}
 		logger.Errorf("Datastore: GetFiles(): error during file getting, err: %s\n", err.Error())
 		return nil, err
 	}
-	if _, err = tx.Commit(); err != nil {
-		logger.Errorf("Datastore: GetFiles(): error during the transaction committing, err: %s\n", err.Error())
+	return files, nil
+}
+
+//GetSDKs returns sdk entities by an identifier
+func (d *Datastore) GetSDKs(ctx context.Context) ([]*entity.SDKEntity, error) {
+	var sdkKeys []*datastore.Key
+	for sdkName := range pb.Sdk_value {
+		if sdkName != pb.Sdk_SDK_UNSPECIFIED.String() {
+			sdkKeys = append(sdkKeys, utils.GetSdkKey(ctx, sdkName))
+		}
+	}
+	var sdks = make([]*entity.SDKEntity, len(sdkKeys))
+	if err := d.Client.GetMulti(ctx, sdkKeys, sdks); err != nil {
+		logger.Errorf("Datastore: GetSDKs(): error during the getting sdks, err: %s\n", err.Error())
 		return nil, err
 	}
-	return files, nil
+	for sdkIndex, sdk := range sdks {
+		sdk.Name = sdkKeys[sdkIndex].Name
+	}
+	return sdks, nil
 }
 
-//GetSDK returns the sdk entity by an identifier
-func (d *Datastore) GetSDK(ctx context.Context, id string) (*entity.SDKEntity, error) {
-	sdkId := utils.GetNameKey(SdkKind, id, Namespace, nil)
-	sdk := new(entity.SDKEntity)
-	if err := d.Client.Get(ctx, sdkId, sdk); err != nil {
-		logger.Errorf("Datastore: GetSDK(): error during sdk getting, err: %s\n", err.Error())
+//GetCatalog returns all examples
+func (d *Datastore) GetCatalog(ctx context.Context, sdkCatalog []*entity.SDKEntity) ([]*pb.Categories, error) {
+	//Retrieving examples
+	exampleQuery := datastore.NewQuery(constants.ExampleKind).Namespace(utils.GetNamespace(ctx))
+	var examples []*entity.ExampleEntity
+	exampleKeys, err := d.Client.GetAll(ctx, exampleQuery, &examples)
+	if err != nil {
+		logger.Errorf("Datastore: GetCatalog(): error during the getting examples, err: %s\n", err.Error())
 		return nil, err
 	}
-	return sdk, nil
+
+	//Retrieving snippets
+	var snippetKeys []*datastore.Key
+	for _, exampleKey := range exampleKeys {
+		snippetKeys = append(snippetKeys, utils.GetSnippetKey(ctx, exampleKey.Name))
+	}
+	snippets := make([]*entity.SnippetEntity, len(snippetKeys))
+	if err = d.Client.GetMulti(ctx, snippetKeys, snippets); err != nil {
+		logger.Errorf("Datastore: GetCatalog(): error during the getting snippets, err: %s\n", err.Error())
+		return nil, err
+	}
+
+	//Retrieving files
+	var fileKeys []*datastore.Key
+	for snpIndx, snippet := range snippets {
+		for fileIndx := 0; fileIndx < snippet.NumberOfFiles; fileIndx++ {
+			fileKey := utils.GetFileKey(ctx, exampleKeys[snpIndx].Name, fileIndx)
+			fileKeys = append(fileKeys, fileKey)
+		}
+	}
+	files := make([]*entity.FileEntity, len(fileKeys))
+	if err = d.Client.GetMulti(ctx, fileKeys, files); err != nil {
+		logger.Errorf("Datastore: GetCatalog(): error during the getting files, err: %s\n", err.Error())
+		return nil, err
+	}
+
+	return d.ResponseMapper.ToArrayCategories(&dto.CatalogDTO{
+		Examples:   examples,
+		Snippets:   snippets,
+		Files:      files,
+		SdkCatalog: sdkCatalog,
+	}), nil
 }
 
 //DeleteUnusedSnippets deletes all unused snippets
 func (d *Datastore) DeleteUnusedSnippets(ctx context.Context, dayDiff int32) error {
 	var hoursDiff = dayDiff * 24
 	boundaryDate := time.Now().Add(-time.Hour * time.Duration(hoursDiff))
-	snippetQuery := datastore.NewQuery(SnippetKind).
-		Namespace(Namespace).
+	snippetQuery := datastore.NewQuery(constants.SnippetKind).
+		Namespace(utils.GetNamespace(ctx)).
 		Filter("lVisited <= ", boundaryDate).
-		Filter("origin =", "PG_USER").
+		Filter("origin =", constants.UserSnippetOrigin).
 		Project("numberOfFiles")
 	var snpDtos []*dto.SnippetDeleteDTO
 	snpKeys, err := d.Client.GetAll(ctx, snippetQuery, &snpDtos)
@@ -216,9 +261,7 @@ func (d *Datastore) DeleteUnusedSnippets(ctx context.Context, dayDiff int32) err
 	var fileKeys []*datastore.Key
 	for snpIndex, snpKey := range snpKeys {
 		for fileIndex := 0; fileIndex < snpDtos[snpIndex].NumberOfFiles; fileIndex++ {
-			fileId := fmt.Sprintf("%s_%d", snpKey.Name, fileIndex)
-			fileKey := utils.GetNameKey(FileKind, fileId, Namespace, nil)
-			fileKeys = append(fileKeys, fileKey)
+			fileKeys = append(fileKeys, utils.GetFileKey(ctx, snpKey.Name, fileIndex))
 		}
 	}
 	_, err = d.Client.RunInTransaction(ctx, func(tx *datastore.Transaction) error {
@@ -232,3 +275,10 @@ func (d *Datastore) DeleteUnusedSnippets(ctx context.Context, dayDiff int32) err
 	}
 	return nil
 }
+
+func rollback(tx *datastore.Transaction) {
+	err := tx.Rollback()
+	if err != nil {
+		logger.Errorf(errorMsgTemplateTxRollback, err.Error())
+	}
+}
diff --git a/playground/backend/internal/db/datastore/datastore_db_test.go b/playground/backend/internal/db/datastore/datastore_db_test.go
index 799c6b1fa32..fe73b29be9a 100644
--- a/playground/backend/internal/db/datastore/datastore_db_test.go
+++ b/playground/backend/internal/db/datastore/datastore_db_test.go
@@ -22,19 +22,14 @@ import (
 	"testing"
 	"time"
 
-	"cloud.google.com/go/datastore"
-
 	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/constants"
 	"beam.apache.org/playground/backend/internal/db/entity"
+	"beam.apache.org/playground/backend/internal/db/mapper"
+	"beam.apache.org/playground/backend/internal/tests/test_cleaner"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
-const (
-	datastoreEmulatorHostKey   = "DATASTORE_EMULATOR_HOST"
-	datastoreEmulatorHostValue = "127.0.0.1:8888"
-	datastoreEmulatorProjectId = "test"
-)
-
 var datastoreDb *Datastore
 var ctx context.Context
 
@@ -46,15 +41,16 @@ func TestMain(m *testing.M) {
 }
 
 func setup() {
-	datastoreEmulatorHost := os.Getenv(datastoreEmulatorHostKey)
+	datastoreEmulatorHost := os.Getenv(constants.EmulatorHostKey)
 	if datastoreEmulatorHost == "" {
-		if err := os.Setenv(datastoreEmulatorHostKey, datastoreEmulatorHostValue); err != nil {
+		if err := os.Setenv(constants.EmulatorHostKey, constants.EmulatorHostValue); err != nil {
 			panic(err)
 		}
 	}
 	ctx = context.Background()
+	ctx = context.WithValue(ctx, constants.DatastoreNamespaceKey, "datastore")
 	var err error
-	datastoreDb, err = New(ctx, datastoreEmulatorProjectId)
+	datastoreDb, err = New(ctx, mapper.NewPrecompiledObjectMapper(), constants.EmulatorProjectId)
 	if err != nil {
 		panic(err)
 	}
@@ -73,9 +69,10 @@ func TestDatastore_PutSnippet(t *testing.T) {
 		snip *entity.Snippet
 	}
 	tests := []struct {
-		name    string
-		args    args
-		wantErr bool
+		name      string
+		args      args
+		wantErr   bool
+		cleanData func()
 	}{
 		{
 			name: "PutSnippet() in the usual case",
@@ -85,10 +82,9 @@ func TestDatastore_PutSnippet(t *testing.T) {
 					IdLength: 11,
 				},
 				Snippet: &entity.SnippetEntity{
-					Sdk:           utils.GetNameKey(SdkKind, "SDK_GO", Namespace, nil),
+					Sdk:           utils.GetSdkKey(ctx, pb.Sdk_SDK_GO.String()),
 					PipeOpts:      "MOCK_OPTIONS",
-					Origin:        "PG_USER",
-					OwnerId:       "",
+					Origin:        constants.UserSnippetOrigin,
 					NumberOfFiles: 1,
 				},
 				Files: []*entity.FileEntity{{
@@ -98,6 +94,10 @@ func TestDatastore_PutSnippet(t *testing.T) {
 				}},
 			}},
 			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanFiles(ctx, t, "MOCK_ID", 1)
+				test_cleaner.CleanSnippet(ctx, t, "MOCK_ID")
+			},
 		},
 	}
 
@@ -107,11 +107,9 @@ func TestDatastore_PutSnippet(t *testing.T) {
 			if err != nil {
 				t.Error("PutSnippet() method failed")
 			}
+			tt.cleanData()
 		})
 	}
-
-	cleanData(t, FileKind, "MOCK_ID_0", nil)
-	cleanData(t, SnippetKind, "MOCK_ID", nil)
 }
 
 func TestDatastore_GetSnippet(t *testing.T) {
@@ -121,16 +119,18 @@ func TestDatastore_GetSnippet(t *testing.T) {
 		id  string
 	}
 	tests := []struct {
-		name    string
-		prepare func()
-		args    args
-		wantErr bool
+		name      string
+		prepare   func()
+		args      args
+		wantErr   bool
+		cleanData func()
 	}{
 		{
-			name:    "GetSnippet() with id that is no in the database",
-			prepare: func() {},
-			args:    args{ctx: ctx, id: "MOCK_ID"},
-			wantErr: true,
+			name:      "GetSnippet() with id that is no in the database",
+			prepare:   func() {},
+			args:      args{ctx: ctx, id: "MOCK_ID"},
+			wantErr:   true,
+			cleanData: func() {},
 		},
 		{
 			name: "GetSnippet() in the usual case",
@@ -141,11 +141,10 @@ func TestDatastore_GetSnippet(t *testing.T) {
 						IdLength: 11,
 					},
 					Snippet: &entity.SnippetEntity{
-						Sdk:           utils.GetNameKey(SdkKind, "SDK_GO", Namespace, nil),
+						Sdk:           utils.GetSdkKey(ctx, pb.Sdk_SDK_GO.String()),
 						PipeOpts:      "MOCK_OPTIONS",
 						Created:       nowDate,
-						Origin:        "PG_USER",
-						OwnerId:       "",
+						Origin:        constants.UserSnippetOrigin,
 						NumberOfFiles: 1,
 					},
 					Files: []*entity.FileEntity{{
@@ -157,6 +156,10 @@ func TestDatastore_GetSnippet(t *testing.T) {
 			},
 			args:    args{ctx: ctx, id: "MOCK_ID"},
 			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanFiles(ctx, t, "MOCK_ID", 1)
+				test_cleaner.CleanSnippet(ctx, t, "MOCK_ID")
+			},
 		},
 	}
 
@@ -169,18 +172,16 @@ func TestDatastore_GetSnippet(t *testing.T) {
 			}
 
 			if err == nil {
-				if snip.Sdk.Name != "SDK_GO" ||
+				if snip.Sdk.Name != pb.Sdk_SDK_GO.String() ||
 					snip.PipeOpts != "MOCK_OPTIONS" ||
-					snip.Origin != "PG_USER" ||
+					snip.Origin != constants.UserSnippetOrigin ||
 					snip.OwnerId != "" {
 					t.Error("GetSnippet() unexpected result")
 				}
 			}
+			tt.cleanData()
 		})
 	}
-
-	cleanData(t, FileKind, "MOCK_ID_0", nil)
-	cleanData(t, SnippetKind, "MOCK_ID", nil)
 }
 
 func TestDatastore_PutSDKs(t *testing.T) {
@@ -220,10 +221,6 @@ func TestDatastore_PutSDKs(t *testing.T) {
 			}
 		})
 	}
-
-	for _, sdk := range sdks {
-		cleanData(t, SdkKind, sdk.Name, nil)
-	}
 }
 
 func TestDatastore_PutSchemaVersion(t *testing.T) {
@@ -233,9 +230,10 @@ func TestDatastore_PutSchemaVersion(t *testing.T) {
 		schema *entity.SchemaEntity
 	}
 	tests := []struct {
-		name    string
-		args    args
-		wantErr bool
+		name      string
+		args      args
+		wantErr   bool
+		cleanData func()
 	}{
 		{
 			name: "PutSchemaVersion() in the usual case",
@@ -245,6 +243,9 @@ func TestDatastore_PutSchemaVersion(t *testing.T) {
 				schema: &entity.SchemaEntity{Descr: "MOCK_DESCRIPTION"},
 			},
 			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanSchemaVersion(ctx, t, "MOCK_ID")
+			},
 		},
 		{
 			name: "PutSchemaVersion() when input data is nil",
@@ -253,7 +254,8 @@ func TestDatastore_PutSchemaVersion(t *testing.T) {
 				id:     "MOCK_ID",
 				schema: nil,
 			},
-			wantErr: false,
+			wantErr:   false,
+			cleanData: func() {},
 		},
 	}
 
@@ -263,10 +265,9 @@ func TestDatastore_PutSchemaVersion(t *testing.T) {
 			if err != nil {
 				t.Error("PutSchemaVersion() method failed")
 			}
+			tt.cleanData()
 		})
 	}
-
-	cleanData(t, SchemaKind, "MOCK_ID", nil)
 }
 
 func TestDatastore_GetFiles(t *testing.T) {
@@ -276,41 +277,28 @@ func TestDatastore_GetFiles(t *testing.T) {
 		numberOfFiles int
 	}
 	tests := []struct {
-		name    string
-		prepare func()
-		args    args
-		wantErr bool
+		name      string
+		prepare   func()
+		args      args
+		wantErr   bool
+		cleanData func()
 	}{
 		{
-			name:    "GetFiles() with snippet id that is no in the database",
-			prepare: func() {},
-			args:    args{ctx: ctx, snipId: "MOCK_ID", numberOfFiles: 1},
-			wantErr: true,
+			name:      "GetFiles() with snippet id that is no in the database",
+			prepare:   func() {},
+			args:      args{ctx: ctx, snipId: "MOCK_ID", numberOfFiles: 1},
+			wantErr:   true,
+			cleanData: func() {},
 		},
 		{
-			name: "GetFiles() in the usual case",
-			prepare: func() {
-				_ = datastoreDb.PutSnippet(ctx, "MOCK_ID", &entity.Snippet{
-					IDMeta: &entity.IDMeta{
-						Salt:     "MOCK_SALT",
-						IdLength: 11,
-					},
-					Snippet: &entity.SnippetEntity{
-						Sdk:           utils.GetNameKey(SdkKind, "SDK_GO", Namespace, nil),
-						PipeOpts:      "MOCK_OPTIONS",
-						Origin:        "PG_USER",
-						OwnerId:       "",
-						NumberOfFiles: 1,
-					},
-					Files: []*entity.FileEntity{{
-						Name:    "MOCK_NAME",
-						Content: "MOCK_CONTENT",
-						IsMain:  false,
-					}},
-				})
-			},
+			name:    "GetFiles() in the usual case",
+			prepare: func() { saveSnippet("MOCK_ID", pb.Sdk_SDK_GO.String()) },
 			args:    args{ctx: ctx, snipId: "MOCK_ID", numberOfFiles: 1},
 			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanFiles(ctx, t, "MOCK_ID", 1)
+				test_cleaner.CleanSnippet(ctx, t, "MOCK_ID")
+			},
 		},
 	}
 
@@ -324,20 +312,18 @@ func TestDatastore_GetFiles(t *testing.T) {
 			if files != nil {
 				if len(files) != 1 ||
 					files[0].Content != "MOCK_CONTENT" ||
-					files[0].IsMain != false {
+					files[0].IsMain != true {
 					t.Error("GetFiles() unexpected result")
 				}
-				cleanData(t, FileKind, "MOCK_ID_0", nil)
-				cleanData(t, SnippetKind, "MOCK_ID", nil)
+				tt.cleanData()
 			}
 		})
 	}
 }
 
-func TestDatastore_GetSDK(t *testing.T) {
+func TestDatastore_GetSDKs(t *testing.T) {
 	type args struct {
 		ctx context.Context
-		id  string
 	}
 	sdks := getSDKs()
 	tests := []struct {
@@ -347,35 +333,99 @@ func TestDatastore_GetSDK(t *testing.T) {
 		wantErr bool
 	}{
 		{
-			name: "GetSDK() in the usual case",
-			prepare: func() {
-				_ = datastoreDb.PutSDKs(ctx, sdks)
-			},
-			args:    args{ctx: ctx, id: pb.Sdk_SDK_GO.String()},
+			name:    "GetSDKs() in the usual case",
+			prepare: func() { _ = datastoreDb.PutSDKs(ctx, sdks) },
+			args:    args{ctx: ctx},
 			wantErr: false,
 		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			sdkEntities, err := datastoreDb.GetSDKs(tt.args.ctx)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetSDKs() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if len(sdkEntities) != 4 {
+					t.Error("GetSDK unexpected result, should be four entities")
+				}
+			}
+		})
+	}
+}
+
+func TestDatastore_GetCatalog(t *testing.T) {
+	type args struct {
+		ctx        context.Context
+		sdkCatalog []*entity.SDKEntity
+	}
+	tests := []struct {
+		name      string
+		prepare   func()
+		args      args
+		wantErr   bool
+		cleanData func()
+	}{
 		{
-			name:    "GetSDK() when sdk is missing",
-			prepare: func() {},
-			args:    args{ctx: ctx, id: pb.Sdk_SDK_GO.String()},
-			wantErr: true,
+			name: "Getting catalog in the usual case",
+			prepare: func() {
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet("SDK_JAVA_MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				savePCObjs("SDK_JAVA_MOCK_EXAMPLE")
+			},
+			args: args{
+				ctx: ctx,
+				sdkCatalog: func() []*entity.SDKEntity {
+					var sdks []*entity.SDKEntity
+					for sdkName := range pb.Sdk_value {
+						sdks = append(sdks, &entity.SDKEntity{
+							Name:           sdkName,
+							DefaultExample: "MOCK_DEFAULT_EXAMPLE",
+						})
+					}
+					return sdks
+				}(),
+			},
+			wantErr: false,
+			cleanData: func() {
+				test_cleaner.CleanPCObjs(ctx, t, "SDK_JAVA_MOCK_EXAMPLE")
+				test_cleaner.CleanFiles(ctx, t, "SDK_JAVA_MOCK_EXAMPLE", 1)
+				test_cleaner.CleanSnippet(ctx, t, "SDK_JAVA_MOCK_EXAMPLE")
+				test_cleaner.CleanExample(ctx, t, "SDK_JAVA_MOCK_EXAMPLE")
+			},
 		},
 	}
 
 	for _, tt := range tests {
 		t.Run(tt.name, func(t *testing.T) {
 			tt.prepare()
-			sdkEntity, err := datastoreDb.GetSDK(tt.args.ctx, tt.args.id)
+			catalog, err := datastoreDb.GetCatalog(tt.args.ctx, tt.args.sdkCatalog)
 			if (err != nil) != tt.wantErr {
-				t.Errorf("GetSDK() error = %v, wantErr %v", err, tt.wantErr)
+				t.Errorf("GetCatalog() error = %v, wantErr %v", err, tt.wantErr)
 			}
 			if err == nil {
-				if sdkEntity.DefaultExample != "MOCK_EXAMPLE" {
-					t.Error("GetSDK() unexpected result")
+				if catalog[0].GetSdk() != pb.Sdk_SDK_JAVA {
+					t.Error("GetCatalog() unexpected result: wrong sdk")
+				}
+				actualCatName := catalog[0].GetCategories()[0].GetCategoryName()
+				actualPCObj := catalog[0].GetCategories()[0].GetPrecompiledObjects()[0]
+				if actualCatName != "MOCK_CATEGORY" {
+					t.Error("GetCatalog() unexpected result: wrong category")
 				}
-				for _, sdk := range sdks {
-					cleanData(t, SdkKind, sdk.Name, nil)
+				if actualPCObj.DefaultExample != false ||
+					actualPCObj.Multifile != false ||
+					actualPCObj.Name != "MOCK_EXAMPLE" ||
+					actualPCObj.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+					actualPCObj.CloudPath != "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/MOCK_EXAMPLE" ||
+					actualPCObj.PipelineOptions != "MOCK_OPTIONS" ||
+					actualPCObj.Description != "MOCK_DESCR" ||
+					actualPCObj.Link != "MOCK_PATH" ||
+					actualPCObj.ContextLine != 32 {
+					t.Error("GetCatalog() unexpected result: wrong precompiled obj")
 				}
+				tt.cleanData()
 			}
 		})
 	}
@@ -499,14 +549,14 @@ func TestNew(t *testing.T) {
 	}{
 		{
 			name:    "Initialize datastore database",
-			args:    args{ctx: ctx, projectId: datastoreEmulatorProjectId},
+			args:    args{ctx: ctx, projectId: constants.EmulatorProjectId},
 			wantErr: false,
 		},
 	}
 
 	for _, tt := range tests {
 		t.Run(tt.name, func(t *testing.T) {
-			_, err := New(ctx, datastoreEmulatorProjectId)
+			_, err := New(ctx, mapper.NewPrecompiledObjectMapper(), constants.EmulatorProjectId)
 			if (err != nil) != tt.wantErr {
 				t.Errorf("New() error = %v, wantErr %v", err, tt.wantErr)
 			}
@@ -514,23 +564,60 @@ func TestNew(t *testing.T) {
 	}
 }
 
-func cleanData(t *testing.T, kind, id string, parentId *datastore.Key) {
-	key := datastore.NameKey(kind, id, nil)
-	if parentId != nil {
-		key.Parent = parentId
-	}
-	key.Namespace = Namespace
-	if err := datastoreDb.Client.Delete(ctx, key); err != nil {
-		t.Errorf("Error during data cleaning after the test, err: %s", err.Error())
+func saveExample(name, sdk string) {
+	_, _ = datastoreDb.Client.Put(ctx, utils.GetExampleKey(ctx, sdk, name), &entity.ExampleEntity{
+		Name:       name,
+		Sdk:        utils.GetSdkKey(ctx, sdk),
+		Descr:      "MOCK_DESCR",
+		Cats:       []string{"MOCK_CATEGORY"},
+		Complexity: "MEDIUM",
+		Path:       "MOCK_PATH",
+		Type:       "PRECOMPILED_OBJECT_TYPE_EXAMPLE",
+		Origin:     constants.ExampleOrigin,
+		SchVer:     utils.GetSchemaVerKey(ctx, "MOCK_VERSION"),
+	})
+}
+
+func saveSnippet(snipId, sdk string) {
+	_ = datastoreDb.PutSnippet(ctx, snipId, &entity.Snippet{
+		IDMeta: &entity.IDMeta{
+			Salt:     "MOCK_SALT",
+			IdLength: 11,
+		},
+		Snippet: &entity.SnippetEntity{
+			Sdk:           utils.GetSdkKey(ctx, sdk),
+			PipeOpts:      "MOCK_OPTIONS",
+			Origin:        constants.ExampleOrigin,
+			NumberOfFiles: 1,
+		},
+		Files: []*entity.FileEntity{{
+			Name:     "MOCK_NAME",
+			Content:  "MOCK_CONTENT",
+			CntxLine: 32,
+			IsMain:   true,
+		}},
+	})
+}
+
+func savePCObjs(exampleId string) {
+	pcTypes := []string{constants.PCOutputType, constants.PCLogType, constants.PCGraphType}
+	for _, pcType := range pcTypes {
+		_, _ = datastoreDb.Client.Put(
+			ctx,
+			utils.GetPCObjectKey(ctx, exampleId, pcType),
+			&entity.PrecompiledObjectEntity{Content: "MOCK_CONTENT_" + pcType})
 	}
 }
 
 func getSDKs() []*entity.SDKEntity {
 	var sdkEntities []*entity.SDKEntity
 	for _, sdk := range pb.Sdk_name {
+		if sdk == pb.Sdk_SDK_UNSPECIFIED.String() {
+			continue
+		}
 		sdkEntities = append(sdkEntities, &entity.SDKEntity{
 			Name:           sdk,
-			DefaultExample: "MOCK_EXAMPLE",
+			DefaultExample: "MOCK_DEFAULT_EXAMPLE",
 		})
 	}
 	return sdkEntities
@@ -548,7 +635,7 @@ func putSnippet(id, origin string, lVisited time.Time, numberOfFiles int) {
 	_ = datastoreDb.PutSnippet(ctx, id, &entity.Snippet{
 		IDMeta: &entity.IDMeta{Salt: "MOCK_SALT", IdLength: 11},
 		Snippet: &entity.SnippetEntity{
-			Sdk:           utils.GetNameKey(SdkKind, pb.Sdk_SDK_GO.String(), Namespace, nil),
+			Sdk:           utils.GetSdkKey(ctx, pb.Sdk_SDK_GO.String()),
 			PipeOpts:      "MOCK_OPTIONS",
 			LVisited:      lVisited,
 			Origin:        origin,
diff --git a/playground/backend/internal/db/db.go b/playground/backend/internal/db/db.go
index 5af30f1b264..efe8f850484 100644
--- a/playground/backend/internal/db/db.go
+++ b/playground/backend/internal/db/db.go
@@ -18,12 +18,14 @@ package db
 import (
 	"context"
 
+	pb "beam.apache.org/playground/backend/internal/api/v1"
 	"beam.apache.org/playground/backend/internal/db/entity"
 )
 
 type Database interface {
 	SnippetDatabase
 	CatalogDatabase
+	ExampleDatabase
 }
 
 type SnippetDatabase interface {
@@ -41,5 +43,9 @@ type CatalogDatabase interface {
 
 	PutSDKs(ctx context.Context, sdks []*entity.SDKEntity) error
 
-	GetSDK(ctx context.Context, id string) (*entity.SDKEntity, error)
+	GetSDKs(ctx context.Context) ([]*entity.SDKEntity, error)
+}
+
+type ExampleDatabase interface {
+	GetCatalog(ctx context.Context, sdkCatalog []*entity.SDKEntity) ([]*pb.Categories, error)
 }
diff --git a/playground/backend/internal/db/dto/precompiled_object.go b/playground/backend/internal/db/dto/precompiled_object.go
new file mode 100644
index 00000000000..e64a18226e6
--- /dev/null
+++ b/playground/backend/internal/db/dto/precompiled_object.go
@@ -0,0 +1,82 @@
+// 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 dto
+
+import (
+	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/db/entity"
+)
+
+type ObjectInfo struct {
+	Name            string
+	CloudPath       string
+	Description     string                   `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"`
+	Type            pb.PrecompiledObjectType `protobuf:"varint,4,opt,name=type,proto3,enum=api.v1.PrecompiledObjectType" json:"type,omitempty"`
+	Categories      []string                 `json:"categories,omitempty"`
+	PipelineOptions string                   `protobuf:"bytes,3,opt,name=pipeline_options,proto3" json:"pipeline_options,omitempty"`
+	Link            string                   `protobuf:"bytes,3,opt,name=link,proto3" json:"link,omitempty"`
+	Multifile       bool                     `protobuf:"varint,7,opt,name=multifile,proto3" json:"multifile,omitempty"`
+	ContextLine     int32                    `protobuf:"varint,7,opt,name=context_line,proto3" json:"context_line,omitempty"`
+	DefaultExample  bool                     `protobuf:"varint,7,opt,name=default_example,json=defaultExample,proto3" json:"default_example,omitempty"`
+}
+
+type PrecompiledObjects []ObjectInfo
+type CategoryToPrecompiledObjects map[string]PrecompiledObjects
+type SdkToCategories map[string]CategoryToPrecompiledObjects
+
+type CatalogDTO struct {
+	Examples   []*entity.ExampleEntity
+	Snippets   []*entity.SnippetEntity
+	Files      []*entity.FileEntity
+	SdkCatalog []*entity.SDKEntity
+}
+
+type DefaultExamplesDTO struct {
+	Examples []*entity.ExampleEntity
+	Snippets []*entity.SnippetEntity
+	Files    []*entity.FileEntity
+}
+
+func (c *CatalogDTO) GetSdkCatalogAsMap() map[string]string {
+	sdkToExample := make(map[string]string)
+	for _, sdk := range c.SdkCatalog {
+		sdkToExample[sdk.Name] = sdk.DefaultExample
+	}
+	return sdkToExample
+}
+
+type ExampleDTO struct {
+	Example            *entity.ExampleEntity
+	Snippet            *entity.SnippetEntity
+	Files              []*entity.FileEntity
+	DefaultExampleName string
+}
+
+func (e *ExampleDTO) HasMultiFiles() bool {
+	return e.Snippet.NumberOfFiles > 1
+}
+
+func (e *ExampleDTO) IsDefault() bool {
+	return e.Example.Name == e.DefaultExampleName
+}
+
+func (e *ExampleDTO) GetType() pb.PrecompiledObjectType {
+	return pb.PrecompiledObjectType(pb.PrecompiledObjectType_value[e.Example.Type])
+}
+
+func (e *ExampleDTO) GetContextLine() int32 {
+	return e.Files[0].CntxLine
+}
diff --git a/playground/backend/internal/utils/db_utils_test.go b/playground/backend/internal/db/entity/example.go
similarity index 53%
rename from playground/backend/internal/utils/db_utils_test.go
rename to playground/backend/internal/db/entity/example.go
index 518071bb9e4..49a7f221622 100644
--- a/playground/backend/internal/utils/db_utils_test.go
+++ b/playground/backend/internal/db/entity/example.go
@@ -13,42 +13,23 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package utils
+package entity
 
-import "testing"
+import "cloud.google.com/go/datastore"
 
-func TestID(t *testing.T) {
-	type args struct {
-		salt    string
-		content string
-		length  int8
-	}
-	tests := []struct {
-		name    string
-		args    args
-		want    string
-		wantErr bool
-	}{
-		{
-			name: "ID generation in the usual case",
-			args: args{
-				salt:    "MOCK_SALT",
-				content: "MOCK_CONTENT",
-				length:  11,
-			},
-			want:    "Zl_s-8seE6k",
-			wantErr: false,
-		},
-	}
+type ExampleEntity struct {
+	Name       string         `datastore:"name"`
+	Sdk        *datastore.Key `datastore:"sdk"`
+	Descr      string         `datastore:"descr"`
+	Tags       []string       `datastore:"tags"`
+	Cats       []string       `datastore:"cats"`
+	Complexity string         `datastore:"complexity"`
+	Path       string         `datastore:"path"`
+	Type       string         `datastore:"type"`
+	Origin     string         `datastore:"origin"`
+	SchVer     *datastore.Key `datastore:"schVer"`
+}
 
-	for _, tt := range tests {
-		result, err := ID(tt.args.salt, tt.args.content, tt.args.length)
-		if (err != nil) != tt.wantErr {
-			t.Errorf("ID() error = %v, wantErr %v", err, tt.wantErr)
-			return
-		}
-		if result != tt.want {
-			t.Errorf("ID() result = %v, want %v", result, tt.want)
-		}
-	}
+type PrecompiledObjectEntity struct {
+	Content string `datastore:"content,noindex"`
 }
diff --git a/playground/backend/internal/db/entity/snippet_test.go b/playground/backend/internal/db/entity/snippet_test.go
index 294671f9340..f0e35f3829b 100644
--- a/playground/backend/internal/db/entity/snippet_test.go
+++ b/playground/backend/internal/db/entity/snippet_test.go
@@ -16,10 +16,15 @@
 package entity
 
 import (
-	"beam.apache.org/playground/backend/internal/utils"
+	"context"
 	"testing"
+
+	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/utils"
 )
 
+var ctx = context.Background()
+
 func TestSnippet_ID(t *testing.T) {
 	tests := []struct {
 		name    string
@@ -31,7 +36,7 @@ func TestSnippet_ID(t *testing.T) {
 			name: "Snippet ID() in the usual case",
 			snip: &Snippet{
 				Snippet: &SnippetEntity{
-					Sdk:      utils.GetNameKey("pg_sdks", "SDK_GO", "Playground", nil),
+					Sdk:      utils.GetSdkKey(ctx, pb.Sdk_SDK_GO.String()),
 					PipeOpts: "MOCK_OPTIONS",
 				},
 				Files: []*FileEntity{{
diff --git a/playground/backend/internal/db/mapper/datastore_mapper.go b/playground/backend/internal/db/mapper/datastore_mapper.go
index 2f5cb6a81c5..e1dcfea214f 100644
--- a/playground/backend/internal/db/mapper/datastore_mapper.go
+++ b/playground/backend/internal/db/mapper/datastore_mapper.go
@@ -16,22 +16,24 @@
 package mapper
 
 import (
+	"context"
 	"time"
 
 	pb "beam.apache.org/playground/backend/internal/api/v1"
-	datastoreDb "beam.apache.org/playground/backend/internal/db/datastore"
+	"beam.apache.org/playground/backend/internal/constants"
 	"beam.apache.org/playground/backend/internal/db/entity"
 	"beam.apache.org/playground/backend/internal/environment"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
 type DatastoreMapper struct {
+	ctx    context.Context
 	appEnv *environment.ApplicationEnvs
 	props  *environment.Properties
 }
 
-func New(appEnv *environment.ApplicationEnvs, props *environment.Properties) *DatastoreMapper {
-	return &DatastoreMapper{appEnv: appEnv, props: props}
+func NewDatastoreMapper(ctx context.Context, appEnv *environment.ApplicationEnvs, props *environment.Properties) *DatastoreMapper {
+	return &DatastoreMapper{ctx: ctx, appEnv: appEnv, props: props}
 }
 
 func (m *DatastoreMapper) ToSnippet(info *pb.SaveSnippetRequest) *entity.Snippet {
@@ -40,12 +42,12 @@ func (m *DatastoreMapper) ToSnippet(info *pb.SaveSnippetRequest) *entity.Snippet
 		IDMeta: &entity.IDMeta{Salt: m.props.Salt, IdLength: m.props.IdLength},
 		//OwnerId property will be used in Tour of Beam project
 		Snippet: &entity.SnippetEntity{
-			SchVer:        utils.GetNameKey(datastoreDb.SchemaKind, m.appEnv.SchemaVersion(), datastoreDb.Namespace, nil),
-			Sdk:           utils.GetNameKey(datastoreDb.SdkKind, info.Sdk.String(), datastoreDb.Namespace, nil),
+			SchVer:        utils.GetSchemaVerKey(m.ctx, m.appEnv.SchemaVersion()),
+			Sdk:           utils.GetSdkKey(m.ctx, info.Sdk.String()),
 			PipeOpts:      info.PipelineOptions,
 			Created:       nowDate,
 			LVisited:      nowDate,
-			Origin:        "PG_USER",
+			Origin:        constants.UserSnippetOrigin,
 			NumberOfFiles: len(info.Files),
 		},
 	}
diff --git a/playground/backend/internal/db/mapper/datastore_mapper_test.go b/playground/backend/internal/db/mapper/datastore_mapper_test.go
index b98c420c4ce..d44a1586de4 100644
--- a/playground/backend/internal/db/mapper/datastore_mapper_test.go
+++ b/playground/backend/internal/db/mapper/datastore_mapper_test.go
@@ -16,23 +16,25 @@
 package mapper
 
 import (
+	"context"
 	"os"
 	"testing"
 
 	pb "beam.apache.org/playground/backend/internal/api/v1"
-	datastoreDb "beam.apache.org/playground/backend/internal/db/datastore"
+	"beam.apache.org/playground/backend/internal/constants"
 	"beam.apache.org/playground/backend/internal/db/entity"
 	"beam.apache.org/playground/backend/internal/environment"
 	"beam.apache.org/playground/backend/internal/utils"
 )
 
 var testable *DatastoreMapper
+var datastoreMapperCtx = context.Background()
 
 func TestMain(m *testing.M) {
 	appEnv := environment.NewApplicationEnvs("/app", "", "", "", "", "", "../../../.", nil, 0)
 	appEnv.SetSchemaVersion("MOCK_SCHEMA")
 	props, _ := environment.NewProperties(appEnv.PropertyPath())
-	testable = New(appEnv, props)
+	testable = NewDatastoreMapper(datastoreMapperCtx, appEnv, props)
 	exitValue := m.Run()
 	os.Exit(exitValue)
 }
@@ -56,10 +58,10 @@ func TestEntityMapper_ToSnippet(t *testing.T) {
 					IdLength: 11,
 				},
 				Snippet: &entity.SnippetEntity{
-					SchVer:        utils.GetNameKey(datastoreDb.SchemaKind, "MOCK_SCHEMA", datastoreDb.Namespace, nil),
-					Sdk:           utils.GetNameKey(datastoreDb.SdkKind, "SDK_JAVA", datastoreDb.Namespace, nil),
+					SchVer:        utils.GetSchemaVerKey(datastoreMapperCtx, "MOCK_SCHEMA"),
+					Sdk:           utils.GetSdkKey(datastoreMapperCtx, pb.Sdk_SDK_JAVA.String()),
 					PipeOpts:      "MOCK_OPTIONS",
-					Origin:        "PG_USER",
+					Origin:        constants.UserSnippetOrigin,
 					NumberOfFiles: 1,
 				},
 				Files: []*entity.FileEntity{
@@ -81,7 +83,7 @@ func TestEntityMapper_ToSnippet(t *testing.T) {
 				result.Salt != tt.expected.Salt ||
 				result.Snippet.PipeOpts != tt.expected.Snippet.PipeOpts ||
 				result.Snippet.NumberOfFiles != 1 ||
-				result.Snippet.Origin != "PG_USER" {
+				result.Snippet.Origin != constants.UserSnippetOrigin {
 				t.Error("Unexpected result")
 			}
 		})
diff --git a/playground/backend/internal/db/mapper/mapper.go b/playground/backend/internal/db/mapper/mapper.go
index ddc22434e8f..93605dbbbc0 100644
--- a/playground/backend/internal/db/mapper/mapper.go
+++ b/playground/backend/internal/db/mapper/mapper.go
@@ -16,6 +16,8 @@
 package mapper
 
 import (
+	"beam.apache.org/playground/backend/internal/db/dto"
+
 	pb "beam.apache.org/playground/backend/internal/api/v1"
 	"beam.apache.org/playground/backend/internal/db/entity"
 )
@@ -24,3 +26,8 @@ type EntityMapper interface {
 	ToSnippet(info *pb.SaveSnippetRequest) *entity.Snippet
 	ToFileEntity(info *pb.SaveSnippetRequest, file *pb.SnippetFile) (*entity.FileEntity, error)
 }
+
+type ResponseMapper interface {
+	ToArrayCategories(catalogDTO *dto.CatalogDTO) []*pb.Categories
+	ToObjectInfo(exampleDTO *dto.ExampleDTO) *dto.ObjectInfo
+}
diff --git a/playground/backend/internal/db/mapper/precompiled_object_mapper.go b/playground/backend/internal/db/mapper/precompiled_object_mapper.go
new file mode 100644
index 00000000000..fe846328644
--- /dev/null
+++ b/playground/backend/internal/db/mapper/precompiled_object_mapper.go
@@ -0,0 +1,119 @@
+// 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 mapper
+
+import (
+	"fmt"
+
+	"beam.apache.org/playground/backend/internal/db/dto"
+
+	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/db/entity"
+)
+
+type PrecompiledObjectMapper struct {
+}
+
+func NewPrecompiledObjectMapper() *PrecompiledObjectMapper {
+	return &PrecompiledObjectMapper{}
+}
+
+func (pom *PrecompiledObjectMapper) ToObjectInfo(exampleDTO *dto.ExampleDTO) *dto.ObjectInfo {
+	return &dto.ObjectInfo{
+		Name:            exampleDTO.Example.Name,
+		CloudPath:       getCloudPath(exampleDTO.Example),
+		Description:     exampleDTO.Example.Descr,
+		Type:            exampleDTO.GetType(),
+		Categories:      exampleDTO.Example.Cats,
+		PipelineOptions: exampleDTO.Snippet.PipeOpts,
+		Link:            exampleDTO.Example.Path,
+		Multifile:       exampleDTO.HasMultiFiles(),
+		ContextLine:     exampleDTO.GetContextLine(),
+		DefaultExample:  exampleDTO.IsDefault(),
+	}
+}
+
+func (pom *PrecompiledObjectMapper) ToArrayCategories(catalogDTO *dto.CatalogDTO) []*pb.Categories {
+	sdkToExample := catalogDTO.GetSdkCatalogAsMap()
+	numberOfExamples := len(catalogDTO.Examples)
+	sdkToCategories := make(dto.SdkToCategories, 0)
+	for exampleIndx := 0; exampleIndx < numberOfExamples; exampleIndx++ {
+		example := catalogDTO.Examples[exampleIndx]
+		snippet := catalogDTO.Snippets[exampleIndx]
+		files := []*entity.FileEntity{catalogDTO.Files[exampleIndx]}
+		objInfo := pom.ToObjectInfo(&dto.ExampleDTO{
+			Example:            example,
+			Snippet:            snippet,
+			Files:              files,
+			DefaultExampleName: sdkToExample[example.Sdk.Name],
+		})
+		for _, objCategory := range objInfo.Categories {
+			appendPrecompiledObject(*objInfo, &sdkToCategories, objCategory, example.Sdk.Name)
+		}
+	}
+	sdkCategories := make([]*pb.Categories, 0)
+	for sdkName, categories := range sdkToCategories {
+		sdkCategory := pb.Categories{Sdk: pb.Sdk(pb.Sdk_value[sdkName]), Categories: make([]*pb.Categories_Category, 0)}
+		for categoryName, precompiledObjects := range categories {
+			putPrecompiledObjectsToCategory(categoryName, &precompiledObjects, &sdkCategory)
+		}
+		sdkCategories = append(sdkCategories, &sdkCategory)
+	}
+
+	return sdkCategories
+}
+
+// appendPrecompiledObject add precompiled object to the common structure of precompiled objects
+func appendPrecompiledObject(objectInfo dto.ObjectInfo, sdkToCategories *dto.SdkToCategories, categoryName string, sdk string) {
+	categoryToPrecompiledObjects, ok := (*sdkToCategories)[sdk]
+	if !ok {
+		(*sdkToCategories)[sdk] = make(dto.CategoryToPrecompiledObjects, 0)
+		categoryToPrecompiledObjects = (*sdkToCategories)[sdk]
+	}
+	objects, ok := categoryToPrecompiledObjects[categoryName]
+	if !ok {
+		categoryToPrecompiledObjects[categoryName] = make(dto.PrecompiledObjects, 0)
+		objects = categoryToPrecompiledObjects[categoryName]
+	}
+	categoryToPrecompiledObjects[categoryName] = append(objects, objectInfo)
+}
+
+// putPrecompiledObjectsToCategory adds categories with precompiled objects to protobuf object
+func putPrecompiledObjectsToCategory(categoryName string, precompiledObjects *dto.PrecompiledObjects, sdkCategory *pb.Categories) {
+	category := pb.Categories_Category{
+		CategoryName:       categoryName,
+		PrecompiledObjects: make([]*pb.PrecompiledObject, 0),
+	}
+	for _, object := range *precompiledObjects {
+		category.PrecompiledObjects = append(category.PrecompiledObjects, &pb.PrecompiledObject{
+			CloudPath:       object.CloudPath,
+			Name:            object.Name,
+			Description:     object.Description,
+			Type:            object.Type,
+			PipelineOptions: object.PipelineOptions,
+			Link:            object.Link,
+			Multifile:       object.Multifile,
+			ContextLine:     object.ContextLine,
+			DefaultExample:  object.DefaultExample,
+		})
+	}
+	sdkCategory.Categories = append(sdkCategory.Categories, &category)
+}
+
+// getCloudPath returns the cloud path by example entity
+func getCloudPath(example *entity.ExampleEntity) string {
+	return fmt.Sprintf("%s/%s/%s", example.Sdk.Name, example.Type, example.Name)
+}
diff --git a/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go b/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go
new file mode 100644
index 00000000000..a8733995922
--- /dev/null
+++ b/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go
@@ -0,0 +1,178 @@
+// 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 mapper
+
+import (
+	"context"
+	"testing"
+
+	pb "beam.apache.org/playground/backend/internal/api/v1"
+	"beam.apache.org/playground/backend/internal/constants"
+	"beam.apache.org/playground/backend/internal/db/dto"
+	"beam.apache.org/playground/backend/internal/db/entity"
+	"beam.apache.org/playground/backend/internal/utils"
+)
+
+var pcObjMapper = NewPrecompiledObjectMapper()
+var pcObjMapperCtx = context.Background()
+
+func TestPrecompiledObjectMapper_ToObjectInfo(t *testing.T) {
+	actualResult := pcObjMapper.ToObjectInfo(getExampleDTO("MOCK_NAME", "MOCK_DEFAULT_EXAMPLE", pb.Sdk_SDK_JAVA.String()))
+	if actualResult.Multifile != false ||
+		actualResult.DefaultExample != false ||
+		actualResult.Name != "MOCK_NAME" ||
+		actualResult.CloudPath != "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/MOCK_NAME" ||
+		actualResult.Description != "MOCK_DESCR" ||
+		actualResult.PipelineOptions != "MOCK_OPTIONS" ||
+		actualResult.Link != "MOCK_PATH" ||
+		actualResult.ContextLine != 32 ||
+		len(actualResult.Categories) != 3 ||
+		actualResult.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToObjectInfo() unexpected result")
+	}
+}
+
+func TestPrecompiledObjectMapper_ToArrayCategories(t *testing.T) {
+	actualResult := pcObjMapper.ToArrayCategories(getCatalogDTO())
+	javaCatalog := getCategoryBySdk(actualResult, pb.Sdk_SDK_JAVA)
+	if len(javaCatalog.Categories) != 3 ||
+		len(javaCatalog.Categories[0].PrecompiledObjects) != 1 ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].Name != "SDK_JAVA_MOCK_NAME" ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].Description != "MOCK_DESCR" ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].CloudPath != "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/SDK_JAVA_MOCK_NAME" ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].Multifile != false ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].DefaultExample != false ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].Link != "MOCK_PATH" ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].PipelineOptions != "MOCK_OPTIONS" ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].ContextLine != 32 ||
+		javaCatalog.Categories[0].PrecompiledObjects[0].Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToArrayCategories() unexpected result for Java Catalog")
+	}
+	goCatalog := getCategoryBySdk(actualResult, pb.Sdk_SDK_GO)
+	if len(goCatalog.Categories) != 3 ||
+		len(goCatalog.Categories[0].PrecompiledObjects) != 1 ||
+		goCatalog.Categories[0].PrecompiledObjects[0].Name != "SDK_GO_MOCK_NAME" ||
+		goCatalog.Categories[0].PrecompiledObjects[0].Description != "MOCK_DESCR" ||
+		goCatalog.Categories[0].PrecompiledObjects[0].CloudPath != "SDK_GO/PRECOMPILED_OBJECT_TYPE_EXAMPLE/SDK_GO_MOCK_NAME" ||
+		goCatalog.Categories[0].PrecompiledObjects[0].Multifile != false ||
+		goCatalog.Categories[0].PrecompiledObjects[0].DefaultExample != false ||
+		goCatalog.Categories[0].PrecompiledObjects[0].Link != "MOCK_PATH" ||
+		goCatalog.Categories[0].PrecompiledObjects[0].PipelineOptions != "MOCK_OPTIONS" ||
+		goCatalog.Categories[0].PrecompiledObjects[0].ContextLine != 32 ||
+		goCatalog.Categories[0].PrecompiledObjects[0].Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToArrayCategories() unexpected result for Go Catalog")
+	}
+	pythonCatalog := getCategoryBySdk(actualResult, pb.Sdk_SDK_PYTHON)
+	if len(pythonCatalog.Categories) != 3 ||
+		len(pythonCatalog.Categories[0].PrecompiledObjects) != 1 ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].Name != "SDK_PYTHON_MOCK_NAME" ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].Description != "MOCK_DESCR" ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].CloudPath != "SDK_PYTHON/PRECOMPILED_OBJECT_TYPE_EXAMPLE/SDK_PYTHON_MOCK_NAME" ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].Multifile != false ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].DefaultExample != false ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].Link != "MOCK_PATH" ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].PipelineOptions != "MOCK_OPTIONS" ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].ContextLine != 32 ||
+		pythonCatalog.Categories[0].PrecompiledObjects[0].Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToArrayCategories() unexpected result for Python Catalog")
+	}
+	scioCatalog := getCategoryBySdk(actualResult, pb.Sdk_SDK_SCIO)
+	if len(scioCatalog.Categories) != 3 ||
+		len(scioCatalog.Categories[0].PrecompiledObjects) != 1 ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].Name != "SDK_SCIO_MOCK_NAME" ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].Description != "MOCK_DESCR" ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].CloudPath != "SDK_SCIO/PRECOMPILED_OBJECT_TYPE_EXAMPLE/SDK_SCIO_MOCK_NAME" ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].Multifile != false ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].DefaultExample != false ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].Link != "MOCK_PATH" ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].PipelineOptions != "MOCK_OPTIONS" ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].ContextLine != 32 ||
+		scioCatalog.Categories[0].PrecompiledObjects[0].Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToArrayCategories() unexpected result for Scio Catalog")
+	}
+}
+
+func getCategoryBySdk(catalog []*pb.Categories, sdk pb.Sdk) *pb.Categories {
+	for _, categories := range catalog {
+		if categories.Sdk == sdk {
+			return categories
+		}
+	}
+	return nil
+}
+
+func getExampleDTO(name, defaultName, sdk string) *dto.ExampleDTO {
+	return &dto.ExampleDTO{
+		Example: &entity.ExampleEntity{
+			Name:       name,
+			Sdk:        utils.GetSdkKey(pcObjMapperCtx, sdk),
+			Descr:      "MOCK_DESCR",
+			Cats:       []string{"MOCK_CAT_1", "MOCK_CAT_2", "MOCK_CAT_3"},
+			Complexity: "MEDIUM",
+			Path:       "MOCK_PATH",
+			Type:       "PRECOMPILED_OBJECT_TYPE_EXAMPLE",
+			Origin:     constants.ExampleOrigin,
+			SchVer:     utils.GetSchemaVerKey(pcObjMapperCtx, "MOCK_VERSION"),
+		},
+		Snippet: &entity.SnippetEntity{
+			Sdk:           utils.GetSdkKey(pcObjMapperCtx, sdk),
+			PipeOpts:      "MOCK_OPTIONS",
+			Origin:        constants.ExampleOrigin,
+			SchVer:        utils.GetSchemaVerKey(pcObjMapperCtx, "MOCK_VERSION"),
+			NumberOfFiles: 1,
+		},
+		Files: []*entity.FileEntity{{
+			Name:     "MOCK_NAME",
+			Content:  "MOCK_CONTENT",
+			CntxLine: 32,
+			IsMain:   true,
+		}},
+		DefaultExampleName: defaultName,
+	}
+}
+
+func getCatalogDTO() *dto.CatalogDTO {
+	sdks := getSDKs()
+	examples := make([]*entity.ExampleEntity, 0, 4)
+	snippets := make([]*entity.SnippetEntity, 0, 4)
+	files := make([]*entity.FileEntity, 0, 4)
+	for _, sdk := range sdks {
+		if sdk.Name == pb.Sdk_SDK_UNSPECIFIED.String() {
+			continue
+		}
+		exampleDTO := getExampleDTO(utils.GetIDWithDelimiter(sdk.Name, "MOCK_NAME"), "MOCK_DEFAULT_EXAMPLE", sdk.Name)
+		examples = append(examples, exampleDTO.Example)
+		snippets = append(snippets, exampleDTO.Snippet)
+		files = append(files, exampleDTO.Files[0])
+	}
+	return &dto.CatalogDTO{
+		Examples:   examples,
+		Snippets:   snippets,
+		Files:      files,
+		SdkCatalog: sdks,
+	}
+}
+
+func getSDKs() []*entity.SDKEntity {
+	var sdkEntities []*entity.SDKEntity
+	for _, sdk := range pb.Sdk_name {
+		sdkEntities = append(sdkEntities, &entity.SDKEntity{
+			Name:           sdk,
+			DefaultExample: "MOCK_DEFAULT_EXAMPLE",
+		})
+	}
+	return sdkEntities
+}
diff --git a/playground/backend/internal/db/schema/migration/migration_v001_test.go b/playground/backend/internal/db/schema/migration/migration_v001_test.go
index 5220669da23..2faf36e285b 100644
--- a/playground/backend/internal/db/schema/migration/migration_v001_test.go
+++ b/playground/backend/internal/db/schema/migration/migration_v001_test.go
@@ -16,18 +16,15 @@
 package migration
 
 import (
-	"beam.apache.org/playground/backend/internal/db/datastore"
-	"beam.apache.org/playground/backend/internal/db/schema"
-	"beam.apache.org/playground/backend/internal/environment"
 	"context"
 	"os"
 	"testing"
-)
 
-const (
-	datastoreEmulatorHostKey   = "DATASTORE_EMULATOR_HOST"
-	datastoreEmulatorHostValue = "127.0.0.1:8888"
-	datastoreEmulatorProjectId = "test"
+	"beam.apache.org/playground/backend/internal/constants"
+	"beam.apache.org/playground/backend/internal/db/datastore"
+	"beam.apache.org/playground/backend/internal/db/mapper"
+	"beam.apache.org/playground/backend/internal/db/schema"
+	"beam.apache.org/playground/backend/internal/environment"
 )
 
 var datastoreDb *datastore.Datastore
@@ -41,15 +38,15 @@ func TestMain(m *testing.M) {
 }
 
 func setup() {
-	datastoreEmulatorHost := os.Getenv(datastoreEmulatorHostKey)
+	datastoreEmulatorHost := os.Getenv(constants.EmulatorHostKey)
 	if datastoreEmulatorHost == "" {
-		if err := os.Setenv(datastoreEmulatorHostKey, datastoreEmulatorHostValue); err != nil {
+		if err := os.Setenv(constants.EmulatorHostKey, constants.EmulatorHostValue); err != nil {
 			panic(err)
 		}
 	}
 	ctx = context.Background()
 	var err error
-	datastoreDb, err = datastore.New(ctx, datastoreEmulatorProjectId)
+	datastoreDb, err = datastore.New(ctx, mapper.NewPrecompiledObjectMapper(), constants.EmulatorProjectId)
 	if err != nil {
 		panic(err)
 	}
@@ -62,7 +59,7 @@ func teardown() {
 }
 
 func TestInitialStructure_InitiateData(t *testing.T) {
-	appEnvs := environment.NewApplicationEnvs("/app", "", "", "", "", "../../../../../sdks.yaml", "../../../../.", nil, 0)
+	appEnvs := environment.NewApplicationEnvs("/app", "", "", "", "", "../../../../../sdks-emulator.yaml", "../../../../.", nil, 0)
 	props, err := environment.NewProperties(appEnvs.PropertyPath())
 	if err != nil {
 		t.Errorf("InitiateData(): error during properties initialization, err: %s", err.Error())
diff --git a/playground/backend/internal/tests/test_cleaner/test_cleaner.go b/playground/backend/internal/tests/test_cleaner/test_cleaner.go
new file mode 100644
index 00000000000..3b779611e9e
--- /dev/null
+++ b/playground/backend/internal/tests/test_cleaner/test_cleaner.go
@@ -0,0 +1,73 @@
+// 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 test_cleaner
+
+import (
+	"context"
+	"testing"
+
+	"cloud.google.com/go/datastore"
+
+	"beam.apache.org/playground/backend/internal/constants"
+	"beam.apache.org/playground/backend/internal/utils"
+)
+
+func CleanExample(ctx context.Context, t *testing.T, exampleId string) {
+	cleanData(ctx, t, constants.ExampleKind, exampleId, nil)
+}
+
+func CleanSnippet(ctx context.Context, t *testing.T, snippetId string) {
+	cleanData(ctx, t, constants.SnippetKind, snippetId, nil)
+}
+
+func CleanPCObjs(ctx context.Context, t *testing.T, exampleId string) {
+	pcTypes := []string{constants.PCOutputType, constants.PCLogType, constants.PCGraphType}
+	for _, pcType := range pcTypes {
+		cleanData(ctx, t, constants.PCObjectKind, utils.GetIDWithDelimiter(exampleId, pcType), nil)
+	}
+}
+
+func CleanFiles(ctx context.Context, t *testing.T, snippetId string, numberOfFiles int) {
+	for fileIndx := 0; fileIndx < numberOfFiles; fileIndx++ {
+		cleanData(ctx, t, constants.FileKind, utils.GetIDWithDelimiter(snippetId, fileIndx), nil)
+	}
+}
+
+func CleanSchemaVersion(ctx context.Context, t *testing.T, schemaId string) {
+	cleanData(ctx, t, constants.SchemaKind, schemaId, nil)
+}
+
+func cleanData(ctx context.Context, t *testing.T, kind, id string, parentId *datastore.Key) {
+	client, err := datastore.NewClient(ctx, constants.EmulatorProjectId)
+	if err != nil {
+		t.Errorf("Error during datastore client creating, err: %s\n", err.Error())
+		return
+	}
+	defer func() {
+		err := client.Close()
+		if err != nil {
+			t.Errorf("Error during datastore client closing, err: %s\n", err.Error())
+		}
+	}()
+	key := datastore.NameKey(kind, id, nil)
+	if parentId != nil {
+		key.Parent = parentId
+	}
+	key.Namespace = utils.GetNamespace(ctx)
+	if err = client.Delete(ctx, key); err != nil {
+		t.Errorf("Error during data cleaning, err: %s", err.Error())
+	}
+}
diff --git a/playground/backend/internal/utils/datastore_utils.go b/playground/backend/internal/utils/datastore_utils.go
new file mode 100644
index 00000000000..b1f63aa99b5
--- /dev/null
+++ b/playground/backend/internal/utils/datastore_utils.go
@@ -0,0 +1,132 @@
+// 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 utils
+
+import (
+	"context"
+	"crypto/sha256"
+	"encoding/base64"
+	"fmt"
+	"io"
+	"os"
+	"strconv"
+	"strings"
+
+	"cloud.google.com/go/datastore"
+
+	"beam.apache.org/playground/backend/internal/constants"
+	"beam.apache.org/playground/backend/internal/errors"
+	"beam.apache.org/playground/backend/internal/logger"
+)
+
+func ID(salt, content string, length int8) (string, error) {
+	hash := sha256.New()
+	if _, err := io.WriteString(hash, salt); err != nil {
+		logger.Errorf("ID(): error during K generation: %s", err.Error())
+		return "", errors.InternalError("Error during K generation", "Error writing K and salt")
+	}
+	hash.Write([]byte(content))
+	sum := hash.Sum(nil)
+	b := make([]byte, base64.URLEncoding.EncodedLen(len(sum)))
+	base64.URLEncoding.Encode(b, sum)
+	hashLen := int(length)
+	for hashLen <= len(b) && b[hashLen-1] == '_' {
+		hashLen++
+	}
+	return string(b)[:hashLen], nil
+}
+
+func GetExampleKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.ExampleKind, id, nil)
+}
+
+func GetSdkKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.SdkKind, id, nil)
+}
+
+func GetFileKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.FileKind, id, nil)
+}
+
+func GetSchemaVerKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.SchemaKind, id, nil)
+}
+
+func GetSnippetKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.SnippetKind, id, nil)
+}
+
+func GetPCObjectKey(ctx context.Context, values ...interface{}) *datastore.Key {
+	id := GetIDWithDelimiter(values...)
+	return getNameKey(ctx, constants.PCObjectKind, id, nil)
+}
+
+func GetExampleID(cloudPath string) (string, error) {
+	cloudPathParams := strings.Split(cloudPath, constants.CloudPathDelimiter)
+	if len(cloudPathParams) < 3 {
+		logger.Error("the wrong cloud path from a client")
+		return "", fmt.Errorf("cloud path doesn't have all options. The minimum size must be 3")
+	}
+	return GetIDWithDelimiter(cloudPathParams[0], cloudPathParams[2]), nil
+}
+
+func GetIDWithDelimiter(values ...interface{}) string {
+	valuesAsStr := make([]string, 0)
+	for _, value := range values {
+		switch value.(type) {
+		case int:
+			valuesAsStr = append(valuesAsStr, strconv.Itoa(value.(int)))
+		case int8:
+			valuesAsStr = append(valuesAsStr, strconv.Itoa(int(value.(int8))))
+		case int16:
+			valuesAsStr = append(valuesAsStr, strconv.Itoa(int(value.(int16))))
+		case int32:
+			valuesAsStr = append(valuesAsStr, strconv.Itoa(int(value.(int32))))
+		case int64:
+			valuesAsStr = append(valuesAsStr, strconv.Itoa(int(value.(int64))))
+		default:
+			valuesAsStr = append(valuesAsStr, value.(string))
+		}
+	}
+	return strings.Join(valuesAsStr, constants.IDDelimiter)
+}
+
+// getNameKey returns the datastore key
+func getNameKey(ctx context.Context, kind, id string, parentId *datastore.Key) *datastore.Key {
+	key := datastore.NameKey(kind, id, nil)
+	if parentId != nil {
+		key.Parent = parentId
+	}
+	key.Namespace = GetNamespace(ctx)
+	return key
+}
+
+func GetNamespace(ctx context.Context) string {
+	namespace, ok := ctx.Value(constants.DatastoreNamespaceKey).(string)
+	if !ok {
+		namespace, ok = os.LookupEnv(constants.DatastoreNamespaceKey)
+		if !ok {
+			return constants.Namespace
+		}
+		return namespace
+	}
+	return namespace
+}
diff --git a/playground/backend/internal/utils/datastore_utils_test.go b/playground/backend/internal/utils/datastore_utils_test.go
new file mode 100644
index 00000000000..2f7bb444033
--- /dev/null
+++ b/playground/backend/internal/utils/datastore_utils_test.go
@@ -0,0 +1,206 @@
+// 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 utils
+
+import (
+	"context"
+	"testing"
+
+	"beam.apache.org/playground/backend/internal/constants"
+)
+
+var ctx = context.Background()
+
+func TestID(t *testing.T) {
+	type args struct {
+		salt    string
+		content string
+		length  int8
+	}
+	tests := []struct {
+		name    string
+		args    args
+		want    string
+		wantErr bool
+	}{
+		{
+			name: "ID generation in the usual case",
+			args: args{
+				salt:    "MOCK_SALT",
+				content: "MOCK_CONTENT",
+				length:  11,
+			},
+			want:    "Zl_s-8seE6k",
+			wantErr: false,
+		},
+	}
+
+	for _, tt := range tests {
+		result, err := ID(tt.args.salt, tt.args.content, tt.args.length)
+		if (err != nil) != tt.wantErr {
+			t.Errorf("ID() error = %v, wantErr %v", err, tt.wantErr)
+			return
+		}
+		if result != tt.want {
+			t.Errorf("ID() result = %v, want %v", result, tt.want)
+		}
+	}
+}
+
+func TestGetExampleKey(t *testing.T) {
+	exampleKey := GetExampleKey(ctx, "MOCK_ID")
+	if exampleKey.Namespace != constants.Namespace ||
+		exampleKey.Parent != nil ||
+		exampleKey.Kind != constants.ExampleKind ||
+		exampleKey.Name != "MOCK_ID" {
+		t.Error("GetExampleKey() unexpected result")
+	}
+}
+
+func TestGetSdkKey(t *testing.T) {
+	sdkKey := GetSdkKey(ctx, "MOCK_ID")
+	if sdkKey.Namespace != constants.Namespace ||
+		sdkKey.Parent != nil ||
+		sdkKey.Kind != constants.SdkKind ||
+		sdkKey.Name != "MOCK_ID" {
+		t.Error("GetSdkKey() unexpected result")
+	}
+}
+
+func TestGetFileKey(t *testing.T) {
+	sdkKey := GetFileKey(ctx, "MOCK_ID")
+	if sdkKey.Namespace != constants.Namespace ||
+		sdkKey.Parent != nil ||
+		sdkKey.Kind != constants.FileKind ||
+		sdkKey.Name != "MOCK_ID" {
+		t.Error("GetFileKey() unexpected result")
+	}
+}
+
+func TestGetSchemaVerKey(t *testing.T) {
+	sdkKey := GetSchemaVerKey(ctx, "MOCK_ID")
+	if sdkKey.Namespace != constants.Namespace ||
+		sdkKey.Parent != nil ||
+		sdkKey.Kind != constants.SchemaKind ||
+		sdkKey.Name != "MOCK_ID" {
+		t.Error("GetSchemaVerKey() unexpected result")
+	}
+}
+
+func TestGetSnippetKey(t *testing.T) {
+	sdkKey := GetSnippetKey(ctx, "MOCK_ID")
+	if sdkKey.Namespace != constants.Namespace ||
+		sdkKey.Parent != nil ||
+		sdkKey.Kind != constants.SnippetKind ||
+		sdkKey.Name != "MOCK_ID" {
+		t.Error("GetSnippetKey() unexpected result")
+	}
+}
+
+func TestGetPCObjectKey(t *testing.T) {
+	sdkKey := GetPCObjectKey(ctx, "MOCK_ID")
+	if sdkKey.Namespace != constants.Namespace ||
+		sdkKey.Parent != nil ||
+		sdkKey.Kind != constants.PCObjectKind ||
+		sdkKey.Name != "MOCK_ID" {
+		t.Error("GetPCObjectKey() unexpected result")
+	}
+}
+
+func TestGetExampleID(t *testing.T) {
+	tests := []struct {
+		name              string
+		cloudPath         string
+		expectedExampleId string
+		wantErr           bool
+	}{
+		{
+			name:              "Getting example identifier in the usual case",
+			cloudPath:         "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/MOCK_EXAMPLE",
+			expectedExampleId: "SDK_JAVA_MOCK_EXAMPLE",
+			wantErr:           false,
+		},
+		{
+			name:              "Getting example identifier when invalid the cloud path",
+			cloudPath:         "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE",
+			expectedExampleId: "SDK_JAVA_MOCK_EXAMPLE",
+			wantErr:           true,
+		},
+		{
+			name:              "Getting example identifier when the cloud path is empty",
+			cloudPath:         "",
+			expectedExampleId: "SDK_JAVA_MOCK_EXAMPLE",
+			wantErr:           true,
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			exampleId, err := GetExampleID(tt.cloudPath)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExampleID() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if exampleId != tt.expectedExampleId {
+					t.Error("GetExampleID() unexpected result")
+				}
+			}
+		})
+	}
+}
+
+func TestGetIDWithDelimiter(t *testing.T) {
+	tests := []struct {
+		name           string
+		values         []interface{}
+		expectedResult string
+	}{
+		{
+			name:           "Getting ID separated by a delimiter in the usual case",
+			values:         []interface{}{"SDK_JAVA", "MOCK_EXAMPLE"},
+			expectedResult: "SDK_JAVA_MOCK_EXAMPLE",
+		},
+		{
+			name:           "Getting ID separated by a delimiter when an input data has only one parameter",
+			values:         []interface{}{"MOCK_EXAMPLE"},
+			expectedResult: "MOCK_EXAMPLE",
+		},
+		{
+			name:           "Getting ID separated by a delimiter when an input data is empty",
+			values:         []interface{}{""},
+			expectedResult: "",
+		},
+		{
+			name:           "Getting ID separated by a delimiter when an input data is nil",
+			values:         nil,
+			expectedResult: "",
+		},
+		{
+			name:           "Getting ID separated by a delimiter when an input data has string and integer",
+			values:         []interface{}{"MOCK_EXAMPLE", 2},
+			expectedResult: "MOCK_EXAMPLE_2",
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			actualResult := GetIDWithDelimiter(tt.values...)
+			if actualResult != tt.expectedResult {
+				t.Errorf("GetIDWithDelimiter() unexpected result")
+			}
+		})
+	}
+}
diff --git a/playground/backend/internal/utils/db_utils.go b/playground/backend/internal/utils/db_utils.go
deleted file mode 100644
index b1d957d6e39..00000000000
--- a/playground/backend/internal/utils/db_utils.go
+++ /dev/null
@@ -1,52 +0,0 @@
-// 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 utils
-
-import (
-	"beam.apache.org/playground/backend/internal/errors"
-	"beam.apache.org/playground/backend/internal/logger"
-	"cloud.google.com/go/datastore"
-	"crypto/sha256"
-	"encoding/base64"
-	"io"
-)
-
-func ID(salt, content string, length int8) (string, error) {
-	hash := sha256.New()
-	if _, err := io.WriteString(hash, salt); err != nil {
-		logger.Errorf("ID(): error during K generation: %s", err.Error())
-		return "", errors.InternalError("Error during K generation", "Error writing K and salt")
-	}
-	hash.Write([]byte(content))
-	sum := hash.Sum(nil)
-	b := make([]byte, base64.URLEncoding.EncodedLen(len(sum)))
-	base64.URLEncoding.Encode(b, sum)
-	hashLen := int(length)
-	for hashLen <= len(b) && b[hashLen-1] == '_' {
-		hashLen++
-	}
-	return string(b)[:hashLen], nil
-}
-
-// GetNameKey returns the datastore key
-func GetNameKey(kind, id, namespace string, parentId *datastore.Key) *datastore.Key {
-	key := datastore.NameKey(kind, id, nil)
-	if parentId != nil {
-		key.Parent = parentId
-	}
-	key.Namespace = namespace
-	return key
-}
diff --git a/playground/sdks-emulator.yaml b/playground/sdks-emulator.yaml
new file mode 100644
index 00000000000..709cc6f50cb
--- /dev/null
+++ b/playground/sdks-emulator.yaml
@@ -0,0 +1,28 @@
+# 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.
+
+sdks:
+  SDK_GO:
+    default-example: MOCK_DEFAULT_EXAMPLE
+  SDK_JAVA:
+    default-example: MOCK_DEFAULT_EXAMPLE
+  SDK_PYTHON:
+    default-example: MOCK_DEFAULT_EXAMPLE
+  SDK_SCIO:
+    default-example: MOCK_DEFAULT_EXAMPLE
+
+