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/12/16 14:36:44 UTC

[beam] branch master updated: [Tour Of Beam] delete user progress (#24657)

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 7c691b7449f [Tour Of Beam] delete user progress (#24657)
7c691b7449f is described below

commit 7c691b7449f910f29e63b1a75c8b19d5d970cbae
Author: Evgeny Antyshev <ea...@gmail.com>
AuthorDate: Fri Dec 16 17:36:36 2022 +0300

    [Tour Of Beam] delete user progress (#24657)
    
    * README
    
    * refactor
    
    * aio
    
    * log
    
    * workflow
---
 .../workflows/tour_of_beam_backend_integration.yml |  3 ++
 learning/tour-of-beam/backend/README.md            |  6 ++++
 learning/tour-of-beam/backend/auth.go              | 16 ++++++----
 learning/tour-of-beam/backend/function.go          | 36 ++++++++++++++++++----
 .../backend/integration_tests/auth_test.go         | 26 ++++++++++++++++
 .../backend/integration_tests/client.go            |  7 +++++
 .../backend/integration_tests/function_test.go     | 13 ++++----
 .../backend/integration_tests/local.sh             |  2 ++
 .../backend/internal/service/content.go            |  5 +++
 .../backend/internal/storage/datastore.go          | 28 +++++++++++++++++
 .../tour-of-beam/backend/internal/storage/iface.go |  2 ++
 .../tour-of-beam/backend/internal/storage/mock.go  |  4 +++
 learning/tour-of-beam/backend/middleware.go        | 25 ++++++++++++---
 13 files changed, 150 insertions(+), 23 deletions(-)

diff --git a/.github/workflows/tour_of_beam_backend_integration.yml b/.github/workflows/tour_of_beam_backend_integration.yml
index ce6950890fb..d43af660221 100644
--- a/.github/workflows/tour_of_beam_backend_integration.yml
+++ b/.github/workflows/tour_of_beam_backend_integration.yml
@@ -59,6 +59,7 @@ env:
   PORT_GET_USER_PROGRESS: 8804
   PORT_POST_UNIT_COMPLETE: 8805
   PORT_POST_USER_CODE: 8806
+  PORT_POST_DELETE_PROGRESS: 8807
 
 
 jobs:
@@ -102,6 +103,8 @@ jobs:
         run: PORT=${{ env.PORT_POST_UNIT_COMPLETE }} FUNCTION_TARGET=postUnitComplete ./tob_function &
       - name: Run postUserCode in background
         run: PORT=${{ env.PORT_POST_USER_CODE }} FUNCTION_TARGET=postUserCode ./tob_function &
+      - name: Run postDeleteProgress in background
+        run: PORT=${{ env.PORT_POST_DELETE_PROGRESS }} FUNCTION_TARGET=postDeleteProgress ./tob_function &
 
       # 3. Load data in datastore: run CD step on samples/learning-content
       - name: Run CI/CD to populate datastore
diff --git a/learning/tour-of-beam/backend/README.md b/learning/tour-of-beam/backend/README.md
index 04c4e938a28..a2928e1f957 100644
--- a/learning/tour-of-beam/backend/README.md
+++ b/learning/tour-of-beam/backend/README.md
@@ -182,3 +182,9 @@ request body:
 $ curl -X POST -H "Authorization: Bearer $token" \
   "https://$REGION-$PROJECT_ID.cloudfunctions.net/postUserCode?sdk=python&id=challenge1" -d @request.json
 ```
+
+### Delete user progress
+```
+$ curl -X POST -H "Authorization: Bearer $token" \
+  "https://$REGION-$PROJECT_ID.cloudfunctions.net/postDeleteProgress" -d '{}'
+```
\ No newline at end of file
diff --git a/learning/tour-of-beam/backend/auth.go b/learning/tour-of-beam/backend/auth.go
index e307c00fe29..2d5a3c241c0 100644
--- a/learning/tour-of-beam/backend/auth.go
+++ b/learning/tour-of-beam/backend/auth.go
@@ -23,13 +23,16 @@ import (
 	"net/http"
 	"strings"
 
-	tob "beam.apache.org/learning/tour-of-beam/backend/internal"
 	"beam.apache.org/learning/tour-of-beam/backend/internal/storage"
 	firebase "firebase.google.com/go/v4"
 )
 
-// HandleFunc enriched with sdk and authenticated user uid.
-type HandlerFuncAuthWithSdk func(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid string)
+// helper to extract uid from context
+// set by ParseAuthHeader middleware
+// panics if key is not found
+func getContextUid(r *http.Request) string {
+	return r.Context().Value(CONTEXT_KEY_UID).(string)
+}
 
 const BEARER_SCHEMA = "Bearer "
 
@@ -53,8 +56,8 @@ func MakeAuthorizer(ctx context.Context, repo storage.Iface) *Authorizer {
 }
 
 // middleware to parse authorization header, verify the ID token and extract uid.
-func (a *Authorizer) ParseAuthHeader(next HandlerFuncAuthWithSdk) HandlerFuncWithSdk {
-	return func(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
+func (a *Authorizer) ParseAuthHeader(next http.HandlerFunc) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
 		ctx := r.Context()
 		header := r.Header.Get("authorization") // returns "" if no header
 		if !strings.HasPrefix(header, BEARER_SCHEMA) {
@@ -87,6 +90,7 @@ func (a *Authorizer) ParseAuthHeader(next HandlerFuncAuthWithSdk) HandlerFuncWit
 			return
 		}
 
-		next(w, r, sdk, uid)
+		ctx = context.WithValue(ctx, CONTEXT_KEY_UID, uid)
+		next(w, r.WithContext(ctx))
 	}
 }
diff --git a/learning/tour-of-beam/backend/function.go b/learning/tour-of-beam/backend/function.go
index 2e89cd0c9b0..8c761ab63aa 100644
--- a/learning/tour-of-beam/backend/function.go
+++ b/learning/tour-of-beam/backend/function.go
@@ -115,6 +115,7 @@ func init() {
 	functions.HTTP("getUserProgress", commonGet(ParseSdkParam(auth.ParseAuthHeader(getUserProgress))))
 	functions.HTTP("postUnitComplete", commonPost(ParseSdkParam(auth.ParseAuthHeader(postUnitComplete))))
 	functions.HTTP("postUserCode", commonPost(ParseSdkParam(auth.ParseAuthHeader(postUserCode))))
+	functions.HTTP("postDeleteProgress", commonPost(auth.ParseAuthHeader(postDeleteProgress)))
 }
 
 // Get list of SDK names
@@ -132,7 +133,8 @@ func getSdkList(w http.ResponseWriter, r *http.Request) {
 
 // Get the content tree for a given SDK
 // Required to be wrapped into ParseSdkParam middleware.
-func getContentTree(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
+func getContentTree(w http.ResponseWriter, r *http.Request) {
+	sdk := getContextSdk(r)
 	tree, err := svc.GetContentTree(r.Context(), sdk)
 	if err != nil {
 		log.Println("Get content tree error:", err)
@@ -152,7 +154,8 @@ func getContentTree(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
 // 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) {
+func getUnitContent(w http.ResponseWriter, r *http.Request) {
+	sdk := getContextSdk(r)
 	unitId := r.URL.Query().Get("id")
 
 	unit, err := svc.GetUnitContent(r.Context(), sdk, unitId)
@@ -174,8 +177,11 @@ func getUnitContent(w http.ResponseWriter, r *http.Request, sdk tob.Sdk) {
 	}
 }
 
-// Get user progress
-func getUserProgress(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid string) {
+// Get user progress by sdk and uid
+func getUserProgress(w http.ResponseWriter, r *http.Request) {
+	sdk := getContextSdk(r)
+	uid := getContextUid(r)
+
 	progress, err := svc.GetUserProgress(r.Context(), sdk, uid)
 
 	if err != nil {
@@ -193,7 +199,9 @@ func getUserProgress(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid st
 }
 
 // Mark unit completed
-func postUnitComplete(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid string) {
+func postUnitComplete(w http.ResponseWriter, r *http.Request) {
+	sdk := getContextSdk(r)
+	uid := getContextUid(r)
 	unitId := r.URL.Query().Get("id")
 
 	err := svc.SetUnitComplete(r.Context(), sdk, unitId, uid)
@@ -211,7 +219,9 @@ func postUnitComplete(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid s
 }
 
 // Save user code for unit
-func postUserCode(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid string) {
+func postUserCode(w http.ResponseWriter, r *http.Request) {
+	sdk := getContextSdk(r)
+	uid := getContextUid(r)
 	unitId := r.URL.Query().Get("id")
 
 	var userCodeRequest tob.UserCodeRequest
@@ -239,3 +249,17 @@ func postUserCode(w http.ResponseWriter, r *http.Request, sdk tob.Sdk, uid strin
 
 	fmt.Fprint(w, "{}")
 }
+
+// Delete user progress
+func postDeleteProgress(w http.ResponseWriter, r *http.Request) {
+	uid := getContextUid(r)
+
+	err := svc.DeleteProgress(r.Context(), uid)
+	if err != nil {
+		log.Println("Delete progress error:", err)
+		finalizeErrResponse(w, http.StatusInternalServerError, INTERNAL_ERROR, "storage error")
+		return
+	}
+
+	fmt.Fprint(w, "{}")
+}
diff --git a/learning/tour-of-beam/backend/integration_tests/auth_test.go b/learning/tour-of-beam/backend/integration_tests/auth_test.go
index 6b0e4df98d2..021b97a03bf 100644
--- a/learning/tour-of-beam/backend/integration_tests/auth_test.go
+++ b/learning/tour-of-beam/backend/integration_tests/auth_test.go
@@ -84,6 +84,13 @@ func TestSaveGetProgress(t *testing.T) {
 	}
 	getUserProgressURL := "http://localhost:" + port
 
+	// postDeleteProgressURL
+	port = os.Getenv(PORT_POST_DELETE_PROGRESS)
+	if port == "" {
+		t.Fatal(PORT_POST_DELETE_PROGRESS, "env not set")
+	}
+	postDeleteProgressURL := "http://localhost:" + port
+
 	t.Run("save_complete_no_unit", func(t *testing.T) {
 		resp, err := PostUnitComplete(postUnitCompleteURL, "python", "unknown_unit_id_1", idToken)
 		checkBadHttpCode(t, err, http.StatusNotFound)
@@ -142,6 +149,25 @@ func TestSaveGetProgress(t *testing.T) {
 		exp.Units[1].UserSnippetId = resp.Units[1].UserSnippetId
 		assert.Equal(t, exp, resp)
 	})
+	t.Run("delete_progress", func(t *testing.T) {
+		_, err := PostDeleteProgress(postDeleteProgressURL, idToken)
+		if err != nil {
+			t.Fatal(err)
+		}
+	})
+	t.Run("delete_progress_retry", func(t *testing.T) {
+		_, err := PostDeleteProgress(postDeleteProgressURL, idToken)
+		if err != nil {
+			t.Fatal(err)
+		}
+	})
+	t.Run("get_deleted", func(t *testing.T) {
+		resp, err := GetUserProgress(getUserProgressURL, "python", idToken)
+		if err != nil {
+			t.Fatal(err)
+		}
+		assert.Equal(t, 0, len(resp.Units))
+	})
 }
 
 func TestUserCode(t *testing.T) {
diff --git a/learning/tour-of-beam/backend/integration_tests/client.go b/learning/tour-of-beam/backend/integration_tests/client.go
index 5058929eca4..956c67dde5d 100644
--- a/learning/tour-of-beam/backend/integration_tests/client.go
+++ b/learning/tour-of-beam/backend/integration_tests/client.go
@@ -112,6 +112,13 @@ func PostUserCode(url, sdk, unitId, token string, body UserCodeRequest) (ErrorRe
 	return result, err
 }
 
+func PostDeleteProgress(url, token string) (ErrorResponse, error) {
+	var result ErrorResponse
+	err := Do(&result, http.MethodPost, url, nil,
+		map[string]string{"Authorization": "Bearer " + token}, nil)
+	return result, err
+}
+
 func Post(dst interface{}, url string, queryParams, headers map[string]string, body io.Reader) error {
 	if err := Options(http.MethodPost, url, queryParams); err != nil {
 		return fmt.Errorf("pre-flight request error: %w", err)
diff --git a/learning/tour-of-beam/backend/integration_tests/function_test.go b/learning/tour-of-beam/backend/integration_tests/function_test.go
index ee99769f8e8..813bbdbedd4 100644
--- a/learning/tour-of-beam/backend/integration_tests/function_test.go
+++ b/learning/tour-of-beam/backend/integration_tests/function_test.go
@@ -26,12 +26,13 @@ import (
 )
 
 const (
-	PORT_SDK_LIST           = "PORT_SDK_LIST"
-	PORT_GET_CONTENT_TREE   = "PORT_GET_CONTENT_TREE"
-	PORT_GET_UNIT_CONTENT   = "PORT_GET_UNIT_CONTENT"
-	PORT_GET_USER_PROGRESS  = "PORT_GET_USER_PROGRESS"
-	PORT_POST_UNIT_COMPLETE = "PORT_POST_UNIT_COMPLETE"
-	PORT_POST_USER_CODE     = "PORT_POST_USER_CODE"
+	PORT_SDK_LIST             = "PORT_SDK_LIST"
+	PORT_GET_CONTENT_TREE     = "PORT_GET_CONTENT_TREE"
+	PORT_GET_UNIT_CONTENT     = "PORT_GET_UNIT_CONTENT"
+	PORT_GET_USER_PROGRESS    = "PORT_GET_USER_PROGRESS"
+	PORT_POST_UNIT_COMPLETE   = "PORT_POST_UNIT_COMPLETE"
+	PORT_POST_USER_CODE       = "PORT_POST_USER_CODE"
+	PORT_POST_DELETE_PROGRESS = "PORT_POST_DELETE_PROGRESS"
 )
 
 // scenarios:
diff --git a/learning/tour-of-beam/backend/integration_tests/local.sh b/learning/tour-of-beam/backend/integration_tests/local.sh
index a28032ac0cb..35eb1f77719 100644
--- a/learning/tour-of-beam/backend/integration_tests/local.sh
+++ b/learning/tour-of-beam/backend/integration_tests/local.sh
@@ -31,6 +31,7 @@ export PORT_GET_UNIT_CONTENT=8803
 export PORT_GET_USER_PROGRESS=8804
 export PORT_POST_UNIT_COMPLETE=8805
 export PORT_POST_USER_CODE=8806
+export PORT_POST_DELETE_PROGRESS=8807
 
 mkdir "$DATASTORE_EMULATOR_DATADIR"
 
@@ -44,6 +45,7 @@ PORT=$PORT_GET_UNIT_CONTENT FUNCTION_TARGET=getUnitContent  ./tob_function &
 PORT=$PORT_GET_USER_PROGRESS FUNCTION_TARGET=getUserProgress ./tob_function &
 PORT=$PORT_POST_UNIT_COMPLETE FUNCTION_TARGET=postUnitComplete ./tob_function &
 PORT=$PORT_POST_USER_CODE FUNCTION_TARGET=postUserCode ./tob_function &
+PORT=$PORT_POST_DELETE_PROGRESS FUNCTION_TARGET=postDeleteProgress ./tob_function &
 
 sleep 5
 
diff --git a/learning/tour-of-beam/backend/internal/service/content.go b/learning/tour-of-beam/backend/internal/service/content.go
index 4dffa6da731..b62be44d961 100644
--- a/learning/tour-of-beam/backend/internal/service/content.go
+++ b/learning/tour-of-beam/backend/internal/service/content.go
@@ -31,6 +31,7 @@ type IContent interface {
 	GetUserProgress(ctx context.Context, sdk tob.Sdk, userId string) (tob.SdkProgress, error)
 	SetUnitComplete(ctx context.Context, sdk tob.Sdk, unitId, uid string) error
 	SaveUserCode(ctx context.Context, sdk tob.Sdk, unitId, uid string, userRequest tob.UserCodeRequest) error
+	DeleteProgress(ctx context.Context, uid string) error
 }
 
 type Svc struct {
@@ -102,3 +103,7 @@ func (s *Svc) SaveUserCode(ctx context.Context, sdk tob.Sdk, unitId, uid string,
 
 	return s.Repo.SaveUserSnippetId(ctx, sdk, unitId, uid, savePgSnippet)
 }
+
+func (s *Svc) DeleteProgress(ctx context.Context, uid string) error {
+	return s.Repo.DeleteProgress(ctx, uid)
+}
diff --git a/learning/tour-of-beam/backend/internal/storage/datastore.go b/learning/tour-of-beam/backend/internal/storage/datastore.go
index 6f01332c50f..4c95384a9a0 100644
--- a/learning/tour-of-beam/backend/internal/storage/datastore.go
+++ b/learning/tour-of-beam/backend/internal/storage/datastore.go
@@ -373,5 +373,33 @@ func (d *DatastoreDb) SaveUserSnippetId(
 	return d.upsertUnitProgress(ctx, sdk, unitId, uid, applyChanges)
 }
 
+func (d *DatastoreDb) DeleteProgress(ctx context.Context, uid string) error {
+	userKey := pgNameKey(TbUserKind, uid, nil)
+
+	_, err := d.Client.RunInTransaction(ctx, func(tx *datastore.Transaction) error {
+		query := datastore.NewQuery(TbUserProgressKind).
+			Namespace(PgNamespace).
+			Ancestor(userKey).
+			KeysOnly().
+			Transaction(tx)
+		keys, err := d.Client.GetAll(ctx, query, nil)
+		if err != nil {
+			return fmt.Errorf("query tb_user_progress: %w", err)
+		}
+		log.Printf("deleting %v tb_user_progress entities\n", len(keys))
+		if err := tx.DeleteMulti(keys); err != nil {
+			return fmt.Errorf("delete %v enitities tb_user_progress: %w", len(keys), err)
+		}
+		if err := tx.Delete(userKey); err != nil {
+			return fmt.Errorf("delete tb_user: %w", err)
+		}
+		return nil
+	})
+	if err != nil {
+		return fmt.Errorf("failed to commit: %w", err)
+	}
+	return 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 1cfc6f631d5..bd1c617dd76 100644
--- a/learning/tour-of-beam/backend/internal/storage/iface.go
+++ b/learning/tour-of-beam/backend/internal/storage/iface.go
@@ -37,4 +37,6 @@ type Iface interface {
 		ctx context.Context, sdk tob.Sdk, unitId, uid string,
 		externalSave func(string) (string, error),
 	) error
+
+	DeleteProgress(ctx context.Context, uid string) error
 }
diff --git a/learning/tour-of-beam/backend/internal/storage/mock.go b/learning/tour-of-beam/backend/internal/storage/mock.go
index d93e1e32e3f..2a39c774bd2 100644
--- a/learning/tour-of-beam/backend/internal/storage/mock.go
+++ b/learning/tour-of-beam/backend/internal/storage/mock.go
@@ -89,3 +89,7 @@ func (d *Mock) SaveUserSnippetId(
 ) error {
 	return nil
 }
+
+func (d *Mock) DeleteProgress(ctx context.Context, uid string) error {
+	return nil
+}
diff --git a/learning/tour-of-beam/backend/middleware.go b/learning/tour-of-beam/backend/middleware.go
index 71a43c6229f..1f8c9f52fa8 100644
--- a/learning/tour-of-beam/backend/middleware.go
+++ b/learning/tour-of-beam/backend/middleware.go
@@ -18,6 +18,7 @@
 package tob
 
 import (
+	"context"
 	"log"
 	"net/http"
 
@@ -31,6 +32,22 @@ const (
 	UNAUTHORIZED   = "UNAUTHORIZED"
 )
 
+// this subtypes here to pass go-staticcheck
+type _ContextKeyTypeSdk string
+type _ContextKeyTypeUid string
+
+const (
+	CONTEXT_KEY_SDK _ContextKeyTypeSdk = "sdk"
+	CONTEXT_KEY_UID _ContextKeyTypeUid = "uid"
+)
+
+// helper to extract sdk from context
+// set by ParseSdkParam middleware
+// panics if key is not found
+func getContextSdk(r *http.Request) tob.Sdk {
+	return r.Context().Value(CONTEXT_KEY_SDK).(tob.Sdk)
+}
+
 // 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)
@@ -94,11 +111,8 @@ func Common(method string) func(http.HandlerFunc) http.HandlerFunc {
 	}
 }
 
-// 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 {
+func ParseSdkParam(next http.HandlerFunc) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		sdkStr := r.URL.Query().Get("sdk")
 		sdk := tob.ParseSdk(sdkStr)
@@ -109,6 +123,7 @@ func ParseSdkParam(next HandlerFuncWithSdk) http.HandlerFunc {
 			return
 		}
 
-		next(w, r, sdk)
+		ctx := context.WithValue(r.Context(), CONTEXT_KEY_SDK, sdk)
+		next(w, r.WithContext(ctx))
 	}
 }