You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2022/09/02 10:54:47 UTC

[beam] branch master updated: [Tour Of Beam][backend] get unit content (#22967)

This is an automated email from the ASF dual-hosted git repository.

damccorm 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 378cf40a75f [Tour Of Beam][backend] get unit content (#22967)
378cf40a75f is described below

commit 378cf40a75fb3bfad78c22286734c3ef6ca57ccd
Author: Evgeny Antyshev <ea...@gmail.com>
AuthorDate: Fri Sep 2 13:54:40 2022 +0300

    [Tour Of Beam][backend] get unit content (#22967)
    
    * get_unit_content
    
    * final
    
    * aio
    
    * tests
    
    * Revert "tests"
    
    This reverts commit edd00c31435a3fd2204832acf8e8dd2d0b69a2fb.
---
 learning/tour-of-beam/backend/function.go          | 127 ++++++++++++++++-----
 learning/tour-of-beam/backend/internal/entity.go   |   2 +-
 learning/tour-of-beam/backend/internal/json.go     |  12 ++
 .../backend/internal/service/content.go            |  18 ++-
 .../tour-of-beam/backend/internal/service/mock.go  |  10 +-
 .../backend/internal/storage/datastore.go          |  29 +++++
 .../tour-of-beam/backend/internal/storage/iface.go |   2 +
 .../backend/samples/api/get_unit_content.json      |  15 ++-
 8 files changed, 173 insertions(+), 42 deletions(-)

diff --git a/learning/tour-of-beam/backend/function.go b/learning/tour-of-beam/backend/function.go
index be740f20f70..6c6e34bd311 100644
--- a/learning/tour-of-beam/backend/function.go
+++ b/learning/tour-of-beam/backend/function.go
@@ -32,13 +32,70 @@ import (
 	"github.com/GoogleCloudPlatform/functions-framework-go/functions"
 )
 
-var svc service.IContent
-
 const (
 	BAD_FORMAT     = "BAD_FORMAT"
 	INTERNAL_ERROR = "INTERNAL_ERROR"
+	NOT_FOUND      = "NOT_FOUND"
 )
 
+// Middleware-maker for setting a header
+// We also make this less generic: it works with HandlerFunc's
+// so that to be convertible to func(w http ResponseWriter, r *http.Request)
+// and be accepted by functions.HTTP.
+func AddHeader(header, value string) func(http.HandlerFunc) http.HandlerFunc {
+	return func(next http.HandlerFunc) http.HandlerFunc {
+		return func(w http.ResponseWriter, r *http.Request) {
+			w.Header().Add(header, value)
+			next(w, r)
+		}
+	}
+}
+
+// Middleware to check http method.
+func EnsureMethod(method string) func(http.HandlerFunc) http.HandlerFunc {
+	return func(next http.HandlerFunc) http.HandlerFunc {
+		return func(w http.ResponseWriter, r *http.Request) {
+			if r.Method == method {
+				next(w, r)
+			} else {
+				w.WriteHeader(http.StatusMethodNotAllowed)
+			}
+		}
+	}
+}
+
+// HandleFunc enriched with sdk.
+type HandlerFuncWithSdk func(w http.ResponseWriter, r *http.Request, sdk tob.Sdk)
+
+// middleware to parse sdk query param and pass it as additional handler param.
+func ParseSdkParam(next HandlerFuncWithSdk) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		sdkStr := r.URL.Query().Get("sdk")
+		sdk := tob.ParseSdk(sdkStr)
+
+		if sdk == tob.SDK_UNDEFINED {
+			log.Printf("Bad sdk: %v", sdkStr)
+
+			message := fmt.Sprintf("Sdk not in: %v", tob.SdksList())
+			finalizeErrResponse(w, http.StatusBadRequest, BAD_FORMAT, message)
+
+			return
+		}
+
+		next(w, r, sdk)
+	}
+}
+
+// Helper to format http error messages.
+func finalizeErrResponse(w http.ResponseWriter, status int, code, message string) {
+	resp := tob.CodeMessage{Code: code, Message: message}
+
+	w.WriteHeader(status)
+	_ = json.NewEncoder(w).Encode(resp)
+}
+
+var svc service.IContent
+
 func init() {
 	// dependencies
 	// required:
@@ -57,53 +114,63 @@ func init() {
 		svc = &service.Svc{Repo: &storage.DatastoreDb{Client: client}}
 	}
 
-	// functions framework
-	functions.HTTP("sdkList", sdkList)
-	functions.HTTP("getContentTree", getContentTree)
-}
+	addHeader := AddHeader("Content-Type", "application/json")
+	ensureGet := EnsureMethod(http.MethodGet)
 
-func finalizeErrResponse(w http.ResponseWriter, status int, code, message string) {
-	w.WriteHeader(status)
-	resp := tob.CodeMessage{Code: code, Message: message}
-	_ = json.NewEncoder(w).Encode(resp)
+	// functions framework
+	functions.HTTP("sdkList", ensureGet(addHeader(sdkList)))
+	functions.HTTP("getContentTree", ensureGet(addHeader(ParseSdkParam(getContentTree))))
+	functions.HTTP("getUnitContent", ensureGet(addHeader(ParseSdkParam(getUnitContent))))
 }
 
+// Get list of SDK names
+// Used in both representation and accessing content.
 func sdkList(w http.ResponseWriter, r *http.Request) {
-	if r.Method != "GET" {
-		w.WriteHeader(http.StatusMethodNotAllowed)
-		return
-	}
-	w.Header().Add("Content-Type", "application/json")
 	fmt.Fprint(w, `{"names": ["Java", "Python", "Go"]}`)
 }
 
-func getContentTree(w http.ResponseWriter, r *http.Request) {
-	w.Header().Add("Content-Type", "application/json")
-	if r.Method != "GET" {
-		w.WriteHeader(http.StatusMethodNotAllowed)
+// Get the content tree for a given SDK and user
+// Merges info from the default tree and per-user information:
+// user code snippets and progress
+// Required to be wrapped into ParseSdkParam middleware.
+func getContentTree(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
+	tree, err := svc.GetContentTree(r.Context(), sdk, nil /*TODO userId*/)
+	if err != nil {
+		log.Println("Get content tree error:", err)
+		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "storage error")
 		return
 	}
 
-	sdkStr := r.URL.Query().Get("sdk")
-	sdk := tob.ParseSdk(sdkStr)
-	if sdk == tob.SDK_UNDEFINED {
-		log.Printf("Bad sdk: %v", sdkStr)
-		message := fmt.Sprintf("Sdk not in: %v", tob.SdksList())
-		finalizeErrResponse(w, http.StatusBadRequest, BAD_FORMAT, message)
+	err = json.NewEncoder(w).Encode(tree)
+	if err != nil {
+		log.Println("Format content tree error:", err)
+		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "format content tree")
 		return
 	}
+}
 
-	tree, err := svc.GetContentTree(r.Context(), sdk, nil /*TODO userId*/)
+// Get unit content
+// Everything needed to render a learning unit:
+// description, hints, code snippets
+// Required to be wrapped into ParseSdkParam middleware.
+func getUnitContent(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
+	unitId := r.URL.Query().Get("unitId")
+
+	unit, err := svc.GetUnitContent(r.Context(), sdk, unitId, nil /*TODO userId*/)
+	if err == service.ErrNoUnit {
+		finalizeErrResponse(w, http.StatusNotFound, NOT_FOUND, "unit not found")
+		return
+	}
 	if err != nil {
-		log.Println("Get content tree error:", err)
+		log.Println("Get unit content error:", err)
 		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "storage error")
 		return
 	}
 
-	err = json.NewEncoder(w).Encode(tree)
+	err = json.NewEncoder(w).Encode(unit)
 	if err != nil {
-		log.Println("Format content tree error:", err)
-		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "format content tree")
+		log.Println("Format unit content error:", err)
+		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "format unit content")
 		return
 	}
 }
diff --git a/learning/tour-of-beam/backend/internal/entity.go b/learning/tour-of-beam/backend/internal/entity.go
index 53871530345..22c48f77ee9 100644
--- a/learning/tour-of-beam/backend/internal/entity.go
+++ b/learning/tour-of-beam/backend/internal/entity.go
@@ -29,7 +29,7 @@ type Unit struct {
 
 	// optional, user-specific
 	UserSnippetId string `json:"userSnippetId,omitempty"`
-	IsCompleted   string `json:"isCompleted,omitempty"`
+	IsCompleted   bool   `json:"isCompleted,omitempty"`
 }
 
 type NodeType int
diff --git a/learning/tour-of-beam/backend/internal/json.go b/learning/tour-of-beam/backend/internal/json.go
index 793f04b213c..e3ee3383904 100644
--- a/learning/tour-of-beam/backend/internal/json.go
+++ b/learning/tour-of-beam/backend/internal/json.go
@@ -15,6 +15,18 @@
 
 package internal
 
+func (nt *NodeType) UnmarshalText(text []byte) error {
+	switch string(text) {
+	case "unit":
+		*nt = NODE_UNIT
+	case "group":
+		*nt = NODE_GROUP
+	default:
+		panic("can't parse node type")
+	}
+	return nil
+}
+
 func (nt NodeType) MarshalText() ([]byte, error) {
 	var typ string
 	switch nt {
diff --git a/learning/tour-of-beam/backend/internal/service/content.go b/learning/tour-of-beam/backend/internal/service/content.go
index 90a7fe0fcd7..2edf2e87048 100644
--- a/learning/tour-of-beam/backend/internal/service/content.go
+++ b/learning/tour-of-beam/backend/internal/service/content.go
@@ -17,14 +17,17 @@ package service
 
 import (
 	"context"
+	"errors"
 
 	tob "beam.apache.org/learning/tour-of-beam/backend/internal"
 	"beam.apache.org/learning/tour-of-beam/backend/internal/storage"
 )
 
+var ErrNoUnit = errors.New("unit not found")
+
 type IContent interface {
 	GetContentTree(ctx context.Context, sdk tob.Sdk, userId *string) (tob.ContentTree, error)
-	// GetUnitContent(ctx context.Context, unitId string, userId *string) (tob.UnitContent, error)
+	GetUnitContent(ctx context.Context, sdk tob.Sdk, unitId string, userId *string) (tob.Unit, error)
 }
 
 type Svc struct {
@@ -32,5 +35,18 @@ type Svc struct {
 }
 
 func (s *Svc) GetContentTree(ctx context.Context, sdk tob.Sdk, userId *string) (ct tob.ContentTree, err error) {
+	// TODO enrich tree with user-specific state (isCompleted)
 	return s.Repo.GetContentTree(ctx, sdk)
 }
+
+func (s *Svc) GetUnitContent(ctx context.Context, sdk tob.Sdk, unitId string, userId *string) (tob.Unit, error) {
+	// TODO enrich unit with user-specific state: isCompleted, userSnippetId
+	unit, err := s.Repo.GetUnitContent(ctx, sdk, unitId)
+	if err != nil {
+		return tob.Unit{}, err
+	}
+	if unit == nil {
+		return tob.Unit{}, ErrNoUnit
+	}
+	return *unit, nil
+}
diff --git a/learning/tour-of-beam/backend/internal/service/mock.go b/learning/tour-of-beam/backend/internal/service/mock.go
index c9344a6dbec..dd9fac6cc95 100644
--- a/learning/tour-of-beam/backend/internal/service/mock.go
+++ b/learning/tour-of-beam/backend/internal/service/mock.go
@@ -36,13 +36,13 @@ type Mock struct{}
 var _ IContent = &Mock{}
 
 func (d *Mock) GetContentTree(_ context.Context, sdk tob.Sdk, userId *string) (ct tob.ContentTree, err error) {
-	content, _ := ioutil.ReadFile(path.Join(getSamplesPath(), "content_tree.json"))
+	content, _ := ioutil.ReadFile(path.Join(getSamplesPath(), "get_content_tree.json"))
 	_ = json.Unmarshal(content, &ct)
 	return ct, nil
 }
 
-func (d *Mock) GetUnitContent(_ context.Context, unitId string, userId *string) (u tob.Unit) {
-	content, _ := ioutil.ReadFile(path.Join(getSamplesPath(), "unit.json"))
-	_ = json.Unmarshal(content, &u)
-	return u
+func (d *Mock) GetUnitContent(_ context.Context, sdk tob.Sdk, unitId string, userId *string) (u tob.Unit, err error) {
+	content, _ := ioutil.ReadFile(path.Join(getSamplesPath(), "get_unit_content.json"))
+	err = json.Unmarshal(content, &u)
+	return u, err
 }
diff --git a/learning/tour-of-beam/backend/internal/storage/datastore.go b/learning/tour-of-beam/backend/internal/storage/datastore.go
index ca2f2484dff..62c55b2ba5b 100644
--- a/learning/tour-of-beam/backend/internal/storage/datastore.go
+++ b/learning/tour-of-beam/backend/internal/storage/datastore.go
@@ -227,5 +227,34 @@ func (d *DatastoreDb) SaveContentTrees(ctx context.Context, trees []tob.ContentT
 	return nil
 }
 
+// Get learning unit content by unitId
+func (d *DatastoreDb) GetUnitContent(ctx context.Context, sdk tob.Sdk, unitId string) (unit *tob.Unit, err error) {
+	var tbNodes []TbLearningNode
+	rootKey := pgNameKey(TbLearningPathKind, sdkToKey(sdk), nil)
+
+	query := datastore.NewQuery(TbLearningNodeKind).
+		Namespace(PgNamespace).
+		Ancestor(rootKey).
+		FilterField("id", "=", unitId)
+
+	_, err = d.Client.GetAll(ctx, query, &tbNodes)
+	if err != nil {
+		return nil, fmt.Errorf("query unit failed: %w", err)
+	}
+
+	switch {
+	case len(tbNodes) == 0:
+		return nil, nil
+	case len(tbNodes) > 1:
+		return nil, fmt.Errorf("query by unitId returned %v units", len(tbNodes))
+	}
+
+	node := FromDatastoreNode(tbNodes[0])
+	if node.Type != tob.NODE_UNIT {
+		return nil, fmt.Errorf("wrong node type: %v, unit expected", node.Type)
+	}
+	return node.Unit, nil
+}
+
 // check if the interface is implemented.
 var _ Iface = &DatastoreDb{}
diff --git a/learning/tour-of-beam/backend/internal/storage/iface.go b/learning/tour-of-beam/backend/internal/storage/iface.go
index 8695adf05d4..f81a28e4ce5 100644
--- a/learning/tour-of-beam/backend/internal/storage/iface.go
+++ b/learning/tour-of-beam/backend/internal/storage/iface.go
@@ -24,4 +24,6 @@ import (
 type Iface interface {
 	GetContentTree(ctx context.Context, sdk tob.Sdk) (tob.ContentTree, error)
 	SaveContentTrees(ctx context.Context, trees []tob.ContentTree) error
+
+	GetUnitContent(ctx context.Context, sdk tob.Sdk, unitId string) (*tob.Unit, error)
 }
diff --git a/learning/tour-of-beam/backend/samples/api/get_unit_content.json b/learning/tour-of-beam/backend/samples/api/get_unit_content.json
index a18305870ce..0774f09116f 100644
--- a/learning/tour-of-beam/backend/samples/api/get_unit_content.json
+++ b/learning/tour-of-beam/backend/samples/api/get_unit_content.json
@@ -2,9 +2,14 @@
     "unitId": "1.1",
     "name": "Basic concepts",
     "description": "Lorem ipsum...",
-    "hint": "Try to use this method....",
-    "assignment": "assignmentSnippetId",
-    "solution": "solutionSnippetId",
-    "userSnippet": "userSnippetId",
-    "completed": true
+    "hints" : [
+        "## Hint 1\n\napply yourself :)",
+        "## Hint 2\n\napply more"
+     ],
+
+    "taskSnippetId": "taskSnippetId",
+    "solutionSnippetId": "solutionSnippetId",
+
+    "userSnippetId": "userSnippetId",
+    "isCompleted": true
 }