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/13 14:52:52 UTC

[beam] branch master updated: [Playground] [Backend] Datastore queries and mappers to get precompiled objects (#22868)

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 c654e41cb40 [Playground] [Backend] Datastore queries and mappers to get precompiled objects (#22868)
c654e41cb40 is described below

commit c654e41cb40acad026a2a4665383b60c0227f694
Author: Vladislav Chunikhin <10...@users.noreply.github.com>
AuthorDate: Tue Sep 13 17:52:44 2022 +0300

    [Playground] [Backend] Datastore queries and mappers to get precompiled objects (#22868)
    
    * [Playground] [Backend] added the datastore query to get an example catalog
    
    * [Playground] [Backend] added datastore queries to get precompiled objects
    
    * [Playground] [Backend] refactoring for datastore db
    
    * [Playground] [Backend] updated complexity field
    
    * [Playground] [Backend] updated complexity field
    
    * [Playground] [Backend] fixed integration tests for datastore
    
    * [Playground] [Backend] fixed integration tests for datastore
    
    * [Playground] [Backend] added testing data cleaner
    
    * [Playground] [Backend] added testing data cleaner
    
    * [Playground] [Backend] refactoring for integration tests
    
    * [Playground] [Backend] edited getting datastore key
    
    * [Playground] [Backend] edited getting datastore key
    
    * [Playground] [Backend] updated unit tests for diferent namespaces
    
    * [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
    
    * [Playground] [Backend] added genreics for one method
    
    * fixing incorrect merge
    
    Co-authored-by: oborysevych <ol...@akvelon.com>
---
 .../backend/internal/db/datastore/datastore_db.go  | 210 ++++++++++++
 .../internal/db/datastore/datastore_db_test.go     | 369 ++++++++++++++++++++-
 playground/backend/internal/db/db.go               |  12 +
 .../internal/db/entity/{id_meta.go => common.go}   |   4 +
 playground/backend/internal/db/mapper/mapper.go    |   2 +
 .../db/mapper/precompiled_object_mapper.go         |  32 ++
 .../db/mapper/precompiled_object_mapper_test.go    |  95 ++++++
 .../db/schema/migration/migration_v001_test.go     |   1 +
 8 files changed, 715 insertions(+), 10 deletions(-)

diff --git a/playground/backend/internal/db/datastore/datastore_db.go b/playground/backend/internal/db/datastore/datastore_db.go
index 1abde270ab5..ef99f065a13 100644
--- a/playground/backend/internal/db/datastore/datastore_db.go
+++ b/playground/backend/internal/db/datastore/datastore_db.go
@@ -18,6 +18,7 @@ package datastore
 import (
 	"context"
 	"time"
+	"fmt"
 
 	"cloud.google.com/go/datastore"
 
@@ -243,6 +244,191 @@ func (d *Datastore) GetCatalog(ctx context.Context, sdkCatalog []*entity.SDKEnti
 	}), nil
 }
 
+//GetDefaultExamples returns the default examples
+func (d *Datastore) GetDefaultExamples(ctx context.Context, sdks []*entity.SDKEntity) (map[pb.Sdk]*pb.PrecompiledObject, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return nil, err
+	}
+	defer rollback(tx)
+
+	//Retrieving examples
+	var exampleKeys []*datastore.Key
+	for _, sdk := range sdks {
+		exampleKeys = append(exampleKeys, utils.GetExampleKey(ctx, sdk.Name, sdk.DefaultExample))
+	}
+	examples, err := getEntities[entity.ExampleEntity](tx, exampleKeys)
+	if err != nil {
+		return nil, err
+	}
+
+	if len(examples) == 0 {
+		logger.Error("no examples")
+		return nil, fmt.Errorf("no examples")
+	}
+
+	//Retrieving snippets
+	var snippetKeys []*datastore.Key
+	for _, exampleKey := range exampleKeys {
+		snippetKeys = append(snippetKeys, utils.GetSnippetKey(ctx, exampleKey.Name))
+	}
+	snippets, err := getEntities[entity.SnippetEntity](tx, snippetKeys)
+	if err != nil {
+		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, examples[snpIndx].Sdk.Name, examples[snpIndx].Name, fileIndx)
+			fileKeys = append(fileKeys, fileKey)
+		}
+	}
+	files := make([]*entity.FileEntity, len(fileKeys))
+	if err = tx.GetMulti(fileKeys, files); err != nil {
+		logger.Errorf("error during the getting files, err: %s\n", err.Error())
+		return nil, err
+	}
+
+	return d.ResponseMapper.ToDefaultPrecompiledObjects(&dto.DefaultExamplesDTO{
+		Examples: examples,
+		Snippets: snippets,
+		Files:    files,
+	}), nil
+}
+
+func (d *Datastore) GetExample(ctx context.Context, id string, sdks []*entity.SDKEntity) (*pb.PrecompiledObject, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return nil, err
+	}
+	defer rollback(tx)
+
+	exampleKey := utils.GetExampleKey(ctx, id)
+	var example = new(entity.ExampleEntity)
+	if err = tx.Get(exampleKey, example); err != nil {
+		if err == datastore.ErrNoSuchEntity {
+			logger.Warnf("error during getting example by identifier, err: %s", err.Error())
+			return nil, err
+		}
+		logger.Errorf("error during getting example by identifier, err: %s", err.Error())
+		return nil, err
+	}
+
+	snpKey := utils.GetSnippetKey(ctx, id)
+	var snippet = new(entity.SnippetEntity)
+	if err = tx.Get(snpKey, snippet); err != nil {
+		logger.Errorf("error during getting snippet by identifier, err: %s", err.Error())
+		return nil, err
+	}
+
+	fileKey := utils.GetFileKey(ctx, id, 0)
+	var file = new(entity.FileEntity)
+	if err = tx.Get(fileKey, file); err != nil {
+		logger.Errorf("error during getting file by identifier, err: %s", err.Error())
+		return nil, err
+	}
+
+	sdkToExample := make(map[string]string)
+	for _, sdk := range sdks {
+		sdkToExample[sdk.Name] = sdk.DefaultExample
+	}
+
+	return d.ResponseMapper.ToPrecompiledObj(&dto.ExampleDTO{
+		Example:            example,
+		Snippet:            snippet,
+		Files:              []*entity.FileEntity{file},
+		DefaultExampleName: sdkToExample[example.Sdk.Name],
+	}), err
+}
+
+func (d *Datastore) GetExampleCode(ctx context.Context, id string) (string, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return "", err
+	}
+	defer rollback(tx)
+
+	fileKey := utils.GetFileKey(ctx, id, 0)
+	var file = new(entity.FileEntity)
+	if err = tx.Get(fileKey, file); err != nil {
+		if err == datastore.ErrNoSuchEntity {
+			logger.Warnf("error during getting example code by identifier, err: %s", err.Error())
+			return "", err
+		}
+		logger.Errorf("error during getting example code by identifier, err: %s", err.Error())
+		return "", err
+	}
+	return file.Content, nil
+}
+
+func (d *Datastore) GetExampleOutput(ctx context.Context, id string) (string, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return "", err
+	}
+	defer rollback(tx)
+
+	pcObjKey := utils.GetPCObjectKey(ctx, id, constants.PCOutputType)
+	var pcObj = new(entity.PrecompiledObjectEntity)
+	if err = tx.Get(pcObjKey, pcObj); err != nil {
+		if err == datastore.ErrNoSuchEntity {
+			logger.Warnf("error during getting example output by identifier, err: %s", err.Error())
+			return "", err
+		}
+		logger.Errorf("error during getting example output by identifier, err: %s", err.Error())
+		return "", err
+	}
+	return pcObj.Content, nil
+}
+
+func (d *Datastore) GetExampleLogs(ctx context.Context, id string) (string, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return "", err
+	}
+	defer rollback(tx)
+
+	pcObjKey := utils.GetPCObjectKey(ctx, id, constants.PCLogType)
+	var pcObj = new(entity.PrecompiledObjectEntity)
+	if err = tx.Get(pcObjKey, pcObj); err != nil {
+		if err == datastore.ErrNoSuchEntity {
+			logger.Warnf("error during getting example logs by identifier, err: %s", err.Error())
+			return "", err
+		}
+		logger.Errorf("error during getting example logs by identifier, err: %s", err.Error())
+		return "", err
+	}
+	return pcObj.Content, nil
+}
+
+func (d *Datastore) GetExampleGraph(ctx context.Context, id string) (string, error) {
+	tx, err := d.Client.NewTransaction(ctx, datastore.ReadOnly)
+	if err != nil {
+		logger.Errorf(errorMsgTemplateCreatingTx, err.Error())
+		return "", err
+	}
+	defer rollback(tx)
+
+	pcObjKey := utils.GetPCObjectKey(ctx, id, constants.PCGraphType)
+	var pcObj = new(entity.PrecompiledObjectEntity)
+	if err = tx.Get(pcObjKey, pcObj); err != nil {
+		if err == datastore.ErrNoSuchEntity {
+			logger.Warnf("error during getting example graph by identifier, err: %s", err.Error())
+			return "", err
+		}
+		logger.Errorf("error during getting example graph by identifier, err: %s", err.Error())
+		return "", err
+	}
+	return pcObj.Content, nil
+}
+
 //DeleteUnusedSnippets deletes all unused snippets
 func (d *Datastore) DeleteUnusedSnippets(ctx context.Context, dayDiff int32) error {
 	var hoursDiff = dayDiff * 24
@@ -282,3 +468,27 @@ func rollback(tx *datastore.Transaction) {
 		logger.Errorf(errorMsgTemplateTxRollback, err.Error())
 	}
 }
+
+func getEntities[V entity.DatastoreEntity](tx *datastore.Transaction, keys []*datastore.Key) ([]*V, error) {
+	var examplesWithNils = make([]*V, len(keys))
+	examples := make([]*V, 0)
+	if err := tx.GetMulti(keys, examplesWithNils); err != nil {
+		if errors, ok := err.(datastore.MultiError); ok {
+			for _, errVal := range errors {
+				if errVal == datastore.ErrNoSuchEntity {
+					for _, exampleVal := range examplesWithNils {
+						if exampleVal != nil {
+							examples = append(examples, exampleVal)
+						}
+					}
+				}
+			}
+		} else {
+			logger.Errorf("error during the getting entities, err: %s\n", err.Error())
+			return nil, err
+		}
+	} else {
+		examples = examplesWithNils
+	}
+	return examples, nil
+}
diff --git a/playground/backend/internal/db/datastore/datastore_db_test.go b/playground/backend/internal/db/datastore/datastore_db_test.go
index fe73b29be9a..f5615755c91 100644
--- a/playground/backend/internal/db/datastore/datastore_db_test.go
+++ b/playground/backend/internal/db/datastore/datastore_db_test.go
@@ -431,6 +431,355 @@ func TestDatastore_GetCatalog(t *testing.T) {
 	}
 }
 
+func TestDatastore_GetDefaultExamples(t *testing.T) {
+	type args struct {
+		ctx  context.Context
+		sdks []*entity.SDKEntity
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting default examples in the usual case",
+			prepare: func() {
+				for sdk := range pb.Sdk_value {
+					exampleId := utils.GetIDWithDelimiter(sdk, "MOCK_DEFAULT_EXAMPLE")
+					saveExample("MOCK_DEFAULT_EXAMPLE", sdk)
+					saveSnippet(exampleId, sdk)
+					savePCObjs(exampleId)
+				}
+			},
+			args: args{
+				ctx:  ctx,
+				sdks: getSDKs(),
+			},
+			wantErr: false,
+			clean: func() {
+				for sdk := range pb.Sdk_value {
+					exampleId := utils.GetIDWithDelimiter(sdk, "MOCK_DEFAULT_EXAMPLE")
+					test_cleaner.CleanPCObjs(ctx, t, exampleId)
+					test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+					test_cleaner.CleanSnippet(ctx, t, exampleId)
+					test_cleaner.CleanExample(ctx, t, exampleId)
+				}
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			defaultExamples, err := datastoreDb.GetDefaultExamples(tt.args.ctx, tt.args.sdks)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetDefaultExamples() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if len(defaultExamples) < 4 {
+					t.Errorf("GetDefaultExamples() unexpected result: the length of example should be at least 4")
+				}
+				for _, example := range defaultExamples {
+					if example.DefaultExample != true ||
+						example.Name != "MOCK_DEFAULT_EXAMPLE" ||
+						example.ContextLine != 32 ||
+						example.Multifile != false ||
+						example.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+						example.PipelineOptions != "MOCK_OPTIONS" ||
+						example.Description != "MOCK_DESCR" ||
+						example.Link != "MOCK_PATH" {
+						t.Errorf("GetDefaultExamples() unexpected result: wrong precompiled obj")
+					}
+				}
+				tt.clean()
+			}
+		})
+	}
+}
+
+func TestDatastore_GetExample(t *testing.T) {
+	type args struct {
+		ctx       context.Context
+		exampleId string
+		sdks      []*entity.SDKEntity
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting an example in the usual case",
+			prepare: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet(exampleId, pb.Sdk_SDK_JAVA.String())
+				savePCObjs(exampleId)
+			},
+			args: args{
+				ctx:       ctx,
+				exampleId: utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE"),
+				sdks:      getSDKs(),
+			},
+			wantErr: false,
+			clean: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				test_cleaner.CleanPCObjs(ctx, t, exampleId)
+				test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+				test_cleaner.CleanSnippet(ctx, t, exampleId)
+				test_cleaner.CleanExample(ctx, t, exampleId)
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			example, err := datastoreDb.GetExample(tt.args.ctx, tt.args.exampleId, tt.args.sdks)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExample() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if example.DefaultExample != false ||
+					example.Name != "MOCK_EXAMPLE" ||
+					example.Multifile != false ||
+					example.ContextLine != 32 ||
+					example.Description != "MOCK_DESCR" ||
+					example.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+					example.Link != "MOCK_PATH" ||
+					example.PipelineOptions != "MOCK_OPTIONS" ||
+					example.CloudPath != "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/MOCK_EXAMPLE" {
+					t.Errorf("GetExample() unexpected result: wrong precompiled obj")
+				}
+				tt.clean()
+			}
+		})
+	}
+}
+
+func TestDatastore_GetExampleCode(t *testing.T) {
+	type args struct {
+		ctx       context.Context
+		exampleId string
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting an example code in the usual case",
+			prepare: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet(exampleId, pb.Sdk_SDK_JAVA.String())
+				savePCObjs(exampleId)
+			},
+			args: args{
+				ctx:       ctx,
+				exampleId: utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE"),
+			},
+			wantErr: false,
+			clean: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				test_cleaner.CleanPCObjs(ctx, t, exampleId)
+				test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+				test_cleaner.CleanSnippet(ctx, t, exampleId)
+				test_cleaner.CleanExample(ctx, t, exampleId)
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			code, err := datastoreDb.GetExampleCode(tt.args.ctx, tt.args.exampleId)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExampleCode() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if code != "MOCK_CONTENT" {
+					t.Errorf("GetExampleCode() unexpected result: wrong precompiled obj")
+				}
+				tt.clean()
+			}
+		})
+	}
+}
+
+func TestDatastore_GetExampleOutput(t *testing.T) {
+	type args struct {
+		ctx       context.Context
+		exampleId string
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting an example output in the usual case",
+			prepare: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet(exampleId, pb.Sdk_SDK_JAVA.String())
+				savePCObjs(exampleId)
+			},
+			args: args{
+				ctx:       ctx,
+				exampleId: utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE"),
+			},
+			wantErr: false,
+			clean: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				test_cleaner.CleanPCObjs(ctx, t, exampleId)
+				test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+				test_cleaner.CleanSnippet(ctx, t, exampleId)
+				test_cleaner.CleanExample(ctx, t, exampleId)
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			output, err := datastoreDb.GetExampleOutput(tt.args.ctx, tt.args.exampleId)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExampleOutput() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if output != "MOCK_CONTENT_OUTPUT" {
+					t.Errorf("GetExampleOutput() unexpected result: wrong precompiled obj")
+				}
+				tt.clean()
+			}
+		})
+	}
+}
+
+func TestDatastore_GetExampleLogs(t *testing.T) {
+	type args struct {
+		ctx       context.Context
+		exampleId string
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting an example logs in the usual case",
+			prepare: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet(exampleId, pb.Sdk_SDK_JAVA.String())
+				savePCObjs(exampleId)
+			},
+			args: args{
+				ctx:       ctx,
+				exampleId: utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE"),
+			},
+			wantErr: false,
+			clean: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				test_cleaner.CleanPCObjs(ctx, t, exampleId)
+				test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+				test_cleaner.CleanSnippet(ctx, t, exampleId)
+				test_cleaner.CleanExample(ctx, t, exampleId)
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			output, err := datastoreDb.GetExampleLogs(tt.args.ctx, tt.args.exampleId)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExampleLogs() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if output != "MOCK_CONTENT_LOG" {
+					t.Errorf("GetExampleLogs() unexpected result: wrong precompiled obj")
+				}
+				tt.clean()
+			}
+		})
+	}
+}
+
+func TestDatastore_GetExampleGraph(t *testing.T) {
+	type args struct {
+		ctx       context.Context
+		exampleId string
+	}
+	tests := []struct {
+		name    string
+		prepare func()
+		args    args
+		wantErr bool
+		clean   func()
+	}{
+		{
+			name: "Getting an example graph in the usual case",
+			prepare: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				saveExample("MOCK_EXAMPLE", pb.Sdk_SDK_JAVA.String())
+				saveSnippet(exampleId, pb.Sdk_SDK_JAVA.String())
+				savePCObjs(exampleId)
+			},
+			args: args{
+				ctx:       ctx,
+				exampleId: utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE"),
+			},
+			wantErr: false,
+			clean: func() {
+				exampleId := utils.GetIDWithDelimiter(pb.Sdk_SDK_JAVA.String(), "MOCK_EXAMPLE")
+				test_cleaner.CleanPCObjs(ctx, t, exampleId)
+				test_cleaner.CleanFiles(ctx, t, exampleId, 1)
+				test_cleaner.CleanSnippet(ctx, t, exampleId)
+				test_cleaner.CleanExample(ctx, t, exampleId)
+			},
+		},
+	}
+
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			tt.prepare()
+			output, err := datastoreDb.GetExampleGraph(tt.args.ctx, tt.args.exampleId)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("GetExampleGraph() error = %v, wantErr %v", err, tt.wantErr)
+			}
+			if err == nil {
+				if output != "MOCK_CONTENT_GRAPH" {
+					t.Errorf("GetExampleGraph() unexpected result: wrong precompiled obj")
+				}
+				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()
+			}
+		})
+	}
+}
+
 func TestDatastore_DeleteUnusedSnippets(t *testing.T) {
 	type args struct {
 		ctx     context.Context
@@ -448,25 +797,25 @@ func TestDatastore_DeleteUnusedSnippets(t *testing.T) {
 			args: args{ctx: ctx, dayDiff: 10},
 			prepare: func() {
 				//last visit date is now - 7 days
-				putSnippet("MOCK_ID0", "PG_USER", now.Add(-time.Hour*24*7), 2)
+				putSnippet("MOCK_ID0", constants.UserSnippetOrigin, now.Add(-time.Hour*24*7), 2)
 				//last visit date is now - 10 days
-				putSnippet("MOCK_ID1", "PG_USER", now.Add(-time.Hour*24*10), 4)
+				putSnippet("MOCK_ID1", constants.UserSnippetOrigin, now.Add(-time.Hour*24*10), 4)
 				//last visit date is now - 15 days
-				putSnippet("MOCK_ID2", "PG_USER", now.Add(-time.Hour*24*15), 8)
+				putSnippet("MOCK_ID2", constants.UserSnippetOrigin, now.Add(-time.Hour*24*15), 8)
 				//last visit date is now
-				putSnippet("MOCK_ID3", "PG_USER", now, 1)
+				putSnippet("MOCK_ID3", constants.UserSnippetOrigin, now, 1)
 				//last visit date is now + 2 days
-				putSnippet("MOCK_ID4", "PG_USER", now.Add(time.Hour*24*2), 2)
+				putSnippet("MOCK_ID4", constants.UserSnippetOrigin, now.Add(time.Hour*24*2), 2)
 				//last visit date is now + 10 days
-				putSnippet("MOCK_ID5", "PG_USER", now.Add(time.Hour*24*10), 2)
+				putSnippet("MOCK_ID5", constants.UserSnippetOrigin, now.Add(time.Hour*24*10), 2)
 				//last visit date is now - 18 days
-				putSnippet("MOCK_ID6", "PG_USER", now.Add(-time.Hour*24*18), 3)
+				putSnippet("MOCK_ID6", constants.UserSnippetOrigin, now.Add(-time.Hour*24*18), 3)
 				//last visit date is now - 18 days and origin != PG_USER
-				putSnippet("MOCK_ID7", "PG_EXAMPLES", now.Add(-time.Hour*24*18), 2)
+				putSnippet("MOCK_ID7", constants.ExampleOrigin, now.Add(-time.Hour*24*18), 2)
 				//last visit date is now - 9 days
-				putSnippet("MOCK_ID8", "PG_USER", now.Add(-time.Hour*24*9), 2)
+				putSnippet("MOCK_ID8", constants.UserSnippetOrigin, now.Add(-time.Hour*24*9), 2)
 				//last visit date is now - 11 days
-				putSnippet("MOCK_ID9", "PG_USER", now.Add(-time.Hour*24*11), 2)
+				putSnippet("MOCK_ID9", constants.UserSnippetOrigin, now.Add(-time.Hour*24*11), 2)
 			},
 			wantErr: false,
 		},
diff --git a/playground/backend/internal/db/db.go b/playground/backend/internal/db/db.go
index efe8f850484..43e7b3235c4 100644
--- a/playground/backend/internal/db/db.go
+++ b/playground/backend/internal/db/db.go
@@ -48,4 +48,16 @@ type CatalogDatabase interface {
 
 type ExampleDatabase interface {
 	GetCatalog(ctx context.Context, sdkCatalog []*entity.SDKEntity) ([]*pb.Categories, error)
+
+	GetDefaultExamples(ctx context.Context, sdks []*entity.SDKEntity) (map[pb.Sdk]*pb.PrecompiledObject, error)
+
+	GetExample(ctx context.Context, id string, sdks []*entity.SDKEntity) (*pb.PrecompiledObject, error)
+
+	GetExampleCode(ctx context.Context, id string) (string, error)
+
+	GetExampleOutput(ctx context.Context, id string) (string, error)
+
+	GetExampleLogs(ctx context.Context, id string) (string, error)
+
+	GetExampleGraph(ctx context.Context, id string) (string, error)
 }
diff --git a/playground/backend/internal/db/entity/id_meta.go b/playground/backend/internal/db/entity/common.go
similarity index 92%
rename from playground/backend/internal/db/entity/id_meta.go
rename to playground/backend/internal/db/entity/common.go
index 1ad70fa622d..1fc48c55340 100644
--- a/playground/backend/internal/db/entity/id_meta.go
+++ b/playground/backend/internal/db/entity/common.go
@@ -19,3 +19,7 @@ type IDMeta struct {
 	Salt     string
 	IdLength int8
 }
+
+type DatastoreEntity interface {
+	ExampleEntity | SnippetEntity
+}
diff --git a/playground/backend/internal/db/mapper/mapper.go b/playground/backend/internal/db/mapper/mapper.go
index 93605dbbbc0..402da10942b 100644
--- a/playground/backend/internal/db/mapper/mapper.go
+++ b/playground/backend/internal/db/mapper/mapper.go
@@ -30,4 +30,6 @@ type EntityMapper interface {
 type ResponseMapper interface {
 	ToArrayCategories(catalogDTO *dto.CatalogDTO) []*pb.Categories
 	ToObjectInfo(exampleDTO *dto.ExampleDTO) *dto.ObjectInfo
+	ToDefaultPrecompiledObjects(defaultExamplesDTO *dto.DefaultExamplesDTO) map[pb.Sdk]*pb.PrecompiledObject
+	ToPrecompiledObj(exampleDTO *dto.ExampleDTO) *pb.PrecompiledObject
 }
diff --git a/playground/backend/internal/db/mapper/precompiled_object_mapper.go b/playground/backend/internal/db/mapper/precompiled_object_mapper.go
index fe846328644..cdc39152a8f 100644
--- a/playground/backend/internal/db/mapper/precompiled_object_mapper.go
+++ b/playground/backend/internal/db/mapper/precompiled_object_mapper.go
@@ -76,6 +76,38 @@ func (pom *PrecompiledObjectMapper) ToArrayCategories(catalogDTO *dto.CatalogDTO
 	return sdkCategories
 }
 
+func (pom *PrecompiledObjectMapper) ToDefaultPrecompiledObjects(defaultExamplesDTO *dto.DefaultExamplesDTO) map[pb.Sdk]*pb.PrecompiledObject {
+	result := make(map[pb.Sdk]*pb.PrecompiledObject)
+	for exampleIndx, example := range defaultExamplesDTO.Examples {
+		result[pb.Sdk(pb.Sdk_value[example.Sdk.Name])] = &pb.PrecompiledObject{
+			CloudPath:       getCloudPath(example),
+			Name:            example.Name,
+			Description:     example.Descr,
+			Type:            pb.PrecompiledObjectType(pb.PrecompiledObjectType_value[example.Type]),
+			PipelineOptions: defaultExamplesDTO.Snippets[exampleIndx].PipeOpts,
+			Link:            example.Path,
+			Multifile:       false,
+			ContextLine:     defaultExamplesDTO.Files[exampleIndx].CntxLine,
+			DefaultExample:  true,
+		}
+	}
+	return result
+}
+
+func (pom *PrecompiledObjectMapper) ToPrecompiledObj(exampleDTO *dto.ExampleDTO) *pb.PrecompiledObject {
+	return &pb.PrecompiledObject{
+		CloudPath:       getCloudPath(exampleDTO.Example),
+		Name:            exampleDTO.Example.Name,
+		Description:     exampleDTO.Example.Descr,
+		Type:            exampleDTO.GetType(),
+		PipelineOptions: exampleDTO.Snippet.PipeOpts,
+		Link:            exampleDTO.Example.Path,
+		Multifile:       exampleDTO.HasMultiFiles(),
+		ContextLine:     exampleDTO.GetContextLine(),
+		DefaultExample:  exampleDTO.IsDefault(),
+	}
+}
+
 // 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]
diff --git a/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go b/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go
index a8733995922..c3a27142d5d 100644
--- a/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go
+++ b/playground/backend/internal/db/mapper/precompiled_object_mapper_test.go
@@ -45,6 +45,81 @@ func TestPrecompiledObjectMapper_ToObjectInfo(t *testing.T) {
 	}
 }
 
+func TestPrecompiledObjectMapper_ToPrecompiledObj(t *testing.T) {
+	actualResult := pcObjMapper.ToPrecompiledObj(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 ||
+		actualResult.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" {
+		t.Error("ToPrecompiledObj() unexpected result")
+	}
+}
+
+func TestPrecompiledObjectMapper_ToDefaultPrecompiledObjects(t *testing.T) {
+	actualResult := pcObjMapper.ToDefaultPrecompiledObjects(getDefaultExamplesDTO())
+	javaPCObj, ok := actualResult[pb.Sdk_SDK_JAVA]
+	if !ok ||
+		javaPCObj.DefaultExample != true ||
+		javaPCObj.Name != "1_MOCK_DEFAULT_EXAMPLE" ||
+		javaPCObj.Multifile != false ||
+		javaPCObj.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+		javaPCObj.ContextLine != 32 ||
+		javaPCObj.Link != "MOCK_PATH" ||
+		javaPCObj.Description != "MOCK_DESCR" ||
+		javaPCObj.PipelineOptions != "MOCK_OPTIONS" ||
+		javaPCObj.CloudPath != "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/1_MOCK_DEFAULT_EXAMPLE" {
+		t.Error("ToDefaultPrecompiledObjects() unexpected result for SDK_JAVA")
+	}
+	goPCObj, ok := actualResult[pb.Sdk_SDK_GO]
+	if !ok ||
+		goPCObj.DefaultExample != true ||
+		goPCObj.Name != "2_MOCK_DEFAULT_EXAMPLE" ||
+		goPCObj.Multifile != false ||
+		goPCObj.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+		goPCObj.ContextLine != 32 ||
+		goPCObj.Link != "MOCK_PATH" ||
+		goPCObj.Description != "MOCK_DESCR" ||
+		goPCObj.PipelineOptions != "MOCK_OPTIONS" ||
+		goPCObj.CloudPath != "SDK_GO/PRECOMPILED_OBJECT_TYPE_EXAMPLE/2_MOCK_DEFAULT_EXAMPLE" {
+		t.Error("ToDefaultPrecompiledObjects() unexpected result for SDK_GO")
+	}
+	scioPCObj, ok := actualResult[pb.Sdk_SDK_SCIO]
+	if !ok ||
+		scioPCObj.DefaultExample != true ||
+		scioPCObj.Name != "4_MOCK_DEFAULT_EXAMPLE" ||
+		scioPCObj.Multifile != false ||
+		scioPCObj.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+		scioPCObj.ContextLine != 32 ||
+		scioPCObj.Link != "MOCK_PATH" ||
+		scioPCObj.Description != "MOCK_DESCR" ||
+		scioPCObj.PipelineOptions != "MOCK_OPTIONS" ||
+		scioPCObj.CloudPath != "SDK_SCIO/PRECOMPILED_OBJECT_TYPE_EXAMPLE/4_MOCK_DEFAULT_EXAMPLE" {
+		t.Error("ToDefaultPrecompiledObjects() unexpected result for SDK_SCIO")
+	}
+	pythonPCObj, ok := actualResult[pb.Sdk_SDK_PYTHON]
+	if !ok ||
+		pythonPCObj.DefaultExample != true ||
+		pythonPCObj.Name != "3_MOCK_DEFAULT_EXAMPLE" ||
+		pythonPCObj.Multifile != false ||
+		pythonPCObj.Type.String() != "PRECOMPILED_OBJECT_TYPE_EXAMPLE" ||
+		pythonPCObj.ContextLine != 32 ||
+		pythonPCObj.Link != "MOCK_PATH" ||
+		pythonPCObj.Description != "MOCK_DESCR" ||
+		pythonPCObj.PipelineOptions != "MOCK_OPTIONS" ||
+		pythonPCObj.CloudPath != "SDK_PYTHON/PRECOMPILED_OBJECT_TYPE_EXAMPLE/3_MOCK_DEFAULT_EXAMPLE" {
+		t.Error("ToDefaultPrecompiledObjects() unexpected result for SDK_PYTHON")
+	}
+	_, ok = actualResult[pb.Sdk_SDK_UNSPECIFIED]
+	if ok {
+		t.Error("ToDefaultPrecompiledObjects() unexpected result for SDK_UNSPECIFIED")
+	}
+}
+
 func TestPrecompiledObjectMapper_ToArrayCategories(t *testing.T) {
 	actualResult := pcObjMapper.ToArrayCategories(getCatalogDTO())
 	javaCatalog := getCategoryBySdk(actualResult, pb.Sdk_SDK_JAVA)
@@ -144,6 +219,26 @@ func getExampleDTO(name, defaultName, sdk string) *dto.ExampleDTO {
 	}
 }
 
+func getDefaultExamplesDTO() *dto.DefaultExamplesDTO {
+	examples := make([]*entity.ExampleEntity, 0, 4)
+	snippets := make([]*entity.SnippetEntity, 0, 4)
+	files := make([]*entity.FileEntity, 0, 4)
+	for sdk, sdkNum := range pb.Sdk_value {
+		if sdk == pb.Sdk_SDK_UNSPECIFIED.String() {
+			continue
+		}
+		exampleDTO := getExampleDTO(utils.GetIDWithDelimiter(sdkNum, "MOCK_DEFAULT_EXAMPLE"), "MOCK_DEFAULT_EXAMPLE", sdk)
+		examples = append(examples, exampleDTO.Example)
+		snippets = append(snippets, exampleDTO.Snippet)
+		files = append(files, exampleDTO.Files[0])
+	}
+	return &dto.DefaultExamplesDTO{
+		Examples: examples,
+		Snippets: snippets,
+		Files:    files,
+	}
+}
+
 func getCatalogDTO() *dto.CatalogDTO {
 	sdks := getSDKs()
 	examples := make([]*entity.ExampleEntity, 0, 4)
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 2faf36e285b..2fc1aa78d52 100644
--- a/playground/backend/internal/db/schema/migration/migration_v001_test.go
+++ b/playground/backend/internal/db/schema/migration/migration_v001_test.go
@@ -45,6 +45,7 @@ func setup() {
 		}
 	}
 	ctx = context.Background()
+	ctx = context.WithValue(ctx, constants.DatastoreNamespaceKey, "migration")
 	var err error
 	datastoreDb, err = datastore.New(ctx, mapper.NewPrecompiledObjectMapper(), constants.EmulatorProjectId)
 	if err != nil {