You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by zk...@apache.org on 2023/05/25 08:47:15 UTC

[incubator-devlake] branch main updated: feat(github): adds authentication via github app (#5077)

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

zky pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git


The following commit(s) were added to refs/heads/main by this push:
     new 16f97a1a7 feat(github): adds authentication via github app (#5077)
16f97a1a7 is described below

commit 16f97a1a7605d5ce7cf391bbec1270eec7c77b6e
Author: Marek Magdziak <ma...@hotmail.cz>
AuthorDate: Thu May 25 10:47:09 2023 +0200

    feat(github): adds authentication via github app (#5077)
    
    The change adds new authentication option in addition to PAT. The main motivation is that Github Apps have higher rate limits compared to PATs.
    
    Closes issue 4801
    
    Co-authored-by: Marek Magdziak <ma...@telia.no>
    Co-authored-by: Louis.z <lo...@gmail.com>
---
 backend/plugins/github/api/blueprint_v200.go       |   8 +
 backend/plugins/github/api/connection.go           | 166 +++++++++++-----
 backend/plugins/github/models/connection.go        | 113 ++++++++++-
 .../migrationscripts/20230428_add_multi_auth.go    |  63 ++++++
 .../github/models/migrationscripts/register.go     |   1 +
 config-ui/src/plugins/register/github/api.ts       |   8 +
 .../miller-columns/use-miller-columns.ts           | 121 ++++++++----
 config-ui/src/plugins/register/github/config.tsx   |  40 +++-
 .../github/connection-fields/authentication.tsx}   |  51 ++---
 .../github/connection-fields/githubapp.tsx         | 217 +++++++++++++++++++++
 .../register/github/connection-fields/index.ts     |   2 +
 .../register/github/connection-fields/styled.ts    |   2 +-
 .../register/github/connection-fields/token.tsx    |   9 +-
 config-ui/src/store/connections/api.ts             |   2 +
 config-ui/src/store/connections/context.tsx        |   8 +
 config-ui/src/store/connections/types.ts           |   2 +
 16 files changed, 676 insertions(+), 137 deletions(-)

diff --git a/backend/plugins/github/api/blueprint_v200.go b/backend/plugins/github/api/blueprint_v200.go
index cd8def7c8..fac202d45 100644
--- a/backend/plugins/github/api/blueprint_v200.go
+++ b/backend/plugins/github/api/blueprint_v200.go
@@ -18,6 +18,7 @@ limitations under the License.
 package api
 
 import (
+	"context"
 	"fmt"
 	"net/url"
 	"strings"
@@ -47,6 +48,13 @@ func MakeDataSourcePipelinePlanV200(subtaskMetas []plugin.SubTaskMeta, connectio
 		return nil, nil, err
 	}
 
+	// needed for the connection to populate its access tokens
+	// if AppKey authentication method is selected
+	_, err = helper.NewApiClientFromConnection(context.TODO(), basicRes, connection)
+	if err != nil {
+		return nil, nil, err
+	}
+
 	plan := make(plugin.PipelinePlan, len(bpScopes))
 	plan, err = makeDataSourcePipelinePlanV200(subtaskMetas, plan, bpScopes, connection, syncPolicy)
 	if err != nil {
diff --git a/backend/plugins/github/api/connection.go b/backend/plugins/github/api/connection.go
index 76e612701..74b28a7bd 100644
--- a/backend/plugins/github/api/connection.go
+++ b/backend/plugins/github/api/connection.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
 	"github.com/apache/incubator-devlake/plugins/github/models"
 	"github.com/apache/incubator-devlake/server/api/shared"
+	"github.com/mitchellh/mapstructure"
 )
 
 var publicPermissions = []string{"repo:status", "repo_deployment", "read:user", "read:org"}
@@ -53,8 +54,9 @@ func findMissingPerms(userPerms map[string]bool, requiredPerms []string) []strin
 
 type GithubTestConnResponse struct {
 	shared.ApiBody
-	Login   string `json:"login"`
-	Warning bool   `json:"warning"`
+	Login         string                         `json:"login"`
+	Warning       bool                           `json:"warning"`
+	Installations []models.GithubAppInstallation `json:"installations"`
 }
 
 // @Summary test github connection
@@ -68,72 +70,130 @@ type GithubTestConnResponse struct {
 func TestConnection(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
 	// process input
 	var conn models.GithubConn
-	err := api.Decode(input.Body, &conn, vld)
-	if err != nil {
-		return nil, err
+	e := mapstructure.Decode(input.Body, &conn)
+	if e != nil {
+		return nil, errors.Convert(e)
+	}
+	e = vld.StructExcept(conn, "GithubAppKey", "GithubAccessToken")
+	if e != nil {
+		return nil, errors.Convert(e)
 	}
 
 	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, &conn)
 	if err != nil {
 		return nil, err
 	}
-	res, err := apiClient.Get("user", nil, nil)
-	if err != nil {
-		return nil, errors.BadInput.Wrap(err, "verify token failed")
-	}
 
-	if res.StatusCode == http.StatusUnauthorized {
-		return nil, errors.HttpStatus(http.StatusBadRequest).New("StatusUnauthorized error when testing connection")
-	}
+	githubApiResponse := &GithubTestConnResponse{}
 
-	if res.StatusCode != http.StatusOK {
-		return nil, errors.HttpStatus(res.StatusCode).New("unexpected status code while testing connection")
-	}
+	if conn.AuthMethod == "AppKey" {
+		jwt, err := conn.GithubAppKey.CreateJwt()
+		if err != nil {
+			return nil, err
+		}
 
-	githubUserOfToken := &models.GithubUserOfToken{}
-	err = api.UnmarshalResponse(res, githubUserOfToken)
-	if err != nil {
-		return nil, errors.BadInput.Wrap(err, "verify token failed")
-	} else if githubUserOfToken.Login == "" {
-		return nil, errors.BadInput.Wrap(err, "invalid token")
-	}
+		res, err := apiClient.Get("app", nil, http.Header{
+			"Authorization": []string{fmt.Sprintf("Bearer %s", jwt)},
+		})
+
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
+		}
+		if res.StatusCode != http.StatusOK {
+			return nil, errors.HttpStatus(res.StatusCode).New("unexpected status code while testing connection")
+		}
 
-	success := false
-	warning := false
-	messages := []string{}
-	// for github classic token, check permission
-	if strings.HasPrefix(conn.Token, "ghp_") {
-		scopes := res.Header.Get("X-OAuth-Scopes")
-		// convert "X-OAuth-Scopes" header to user permissions map
-		userPerms := map[string]bool{}
-		for _, userPerm := range strings.Split(scopes, ", ") {
-			userPerms[userPerm] = true
+		githubApp := &models.GithubApp{}
+		err = api.UnmarshalResponse(res, githubApp)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
+		} else if githubApp.Slug == "" {
+			return nil, errors.BadInput.Wrap(err, "invalid token")
 		}
-		// check public repo permission
-		missingPubPerms := findMissingPerms(userPerms, publicPermissions)
-		success = len(missingPubPerms) == 0
-		if !success {
-			messages = append(messages, fmt.Sprintf(
-				"%s is/are required to collect data from Public Repos",
-				strings.Join(missingPubPerms, ", "),
-			))
+
+		res, err = apiClient.Get("app/installations", nil, http.Header{
+			"Authorization": []string{fmt.Sprintf("Bearer %s", jwt)},
+		})
+
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
 		}
-		// check private repo permission
-		missingPriPerms := findMissingPerms(userPerms, privatePermissions)
-		warning = len(missingPriPerms) > 0
-		if warning {
-			messages = append(messages, fmt.Sprintf(
-				"%s is/are required to collect data from Private Repos",
-				strings.Join(missingPriPerms, ", "),
-			))
+		if res.StatusCode != http.StatusOK {
+			return nil, errors.HttpStatus(res.StatusCode).New("unexpected status code while testing connection")
 		}
+
+		githubAppInstallations := &[]models.GithubAppInstallation{}
+		err = api.UnmarshalResponse(res, githubAppInstallations)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
+		}
+
+		githubApiResponse.Success = true
+		githubApiResponse.Message = "success"
+		githubApiResponse.Login = githubApp.Slug
+		githubApiResponse.Installations = *githubAppInstallations
+
+	} else if conn.AuthMethod == "AccessToken" {
+		res, err := apiClient.Get("user", nil, nil)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
+		}
+
+		if res.StatusCode == http.StatusUnauthorized {
+			return nil, errors.HttpStatus(http.StatusBadRequest).New("StatusUnauthorized error when testing connection")
+		}
+
+		if res.StatusCode != http.StatusOK {
+			return nil, errors.HttpStatus(res.StatusCode).New("unexpected status code while testing connection")
+		}
+
+		githubUserOfToken := &models.GithubUserOfToken{}
+		err = api.UnmarshalResponse(res, githubUserOfToken)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "verify token failed")
+		} else if githubUserOfToken.Login == "" {
+			return nil, errors.BadInput.Wrap(err, "invalid token")
+		}
+
+		success := false
+		warning := false
+		messages := []string{}
+		// for github classic token, check permission
+		if strings.HasPrefix(conn.Token, "ghp_") {
+			scopes := res.Header.Get("X-OAuth-Scopes")
+			// convert "X-OAuth-Scopes" header to user permissions map
+			userPerms := map[string]bool{}
+			for _, userPerm := range strings.Split(scopes, ", ") {
+				userPerms[userPerm] = true
+			}
+			// check public repo permission
+			missingPubPerms := findMissingPerms(userPerms, publicPermissions)
+			success = len(missingPubPerms) == 0
+			if !success {
+				messages = append(messages, fmt.Sprintf(
+					"%s is/are required to collect data from Public Repos",
+					strings.Join(missingPubPerms, ", "),
+				))
+			}
+			// check private repo permission
+			missingPriPerms := findMissingPerms(userPerms, privatePermissions)
+			warning = len(missingPriPerms) > 0
+			if warning {
+				messages = append(messages, fmt.Sprintf(
+					"%s is/are required to collect data from Private Repos",
+					strings.Join(missingPriPerms, ", "),
+				))
+			}
+		}
+
+		githubApiResponse.Success = success
+		githubApiResponse.Warning = warning
+		githubApiResponse.Message = strings.Join(messages, ";\n")
+		githubApiResponse.Login = githubUserOfToken.Login
+	} else {
+		return nil, errors.BadInput.New("invalid authentication method")
 	}
 
-	githubApiResponse := &GithubTestConnResponse{}
-	githubApiResponse.Success = success
-	githubApiResponse.Warning = warning
-	githubApiResponse.Message = strings.Join(messages, ";\n")
-	githubApiResponse.Login = githubUserOfToken.Login
 	return &plugin.ApiResourceOutput{Body: githubApiResponse, Status: http.StatusOK}, nil
 }
 
diff --git a/backend/plugins/github/models/connection.go b/backend/plugins/github/models/connection.go
index d634c4889..b321ff86b 100644
--- a/backend/plugins/github/models/connection.go
+++ b/backend/plugins/github/models/connection.go
@@ -18,13 +18,17 @@ limitations under the License.
 package models
 
 import (
+	"encoding/json"
 	"fmt"
+	"io"
 	"net/http"
 	"strings"
+	"time"
 
 	"github.com/apache/incubator-devlake/core/errors"
 	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
+	"github.com/golang-jwt/jwt/v5"
 )
 
 // GithubAccessToken supports fetching data with multiple tokens
@@ -34,28 +38,51 @@ type GithubAccessToken struct {
 	tokenIndex         int      `gorm:"-" json:"-" mapstructure:"-"`
 }
 
+type GithubAppKey struct {
+	helper.AppKey  `mapstructure:",squash"`
+	InstallationID int `mapstructure:"installationId" validate:"required" json:"installationId"`
+}
+
 // GithubConn holds the essential information to connect to the Github API
 type GithubConn struct {
 	helper.RestConnection `mapstructure:",squash"`
+	helper.MultiAuth      `mapstructure:",squash"`
 	GithubAccessToken     `mapstructure:",squash"`
+	GithubAppKey          `mapstructure:",squash"`
 }
 
 // PrepareApiClient splits Token to tokens for SetupAuthentication to utilize
 func (conn *GithubConn) PrepareApiClient(apiClient apihelperabstract.ApiClientAbstract) errors.Error {
-	conn.tokens = strings.Split(conn.Token, ",")
+
+	if conn.AuthMethod == "AccessToken" {
+		conn.tokens = strings.Split(conn.Token, ",")
+	}
+
+	if conn.AuthMethod == "AppKey" && conn.InstallationID != 0 {
+		token, err := conn.getInstallationAccessToken(apiClient)
+		if err != nil {
+			return err
+		}
+
+		conn.Token = token.Token
+		conn.tokens = []string{token.Token}
+	}
+
 	return nil
 }
 
 // SetupAuthentication sets up the HTTP Request Authentication
-func (gat *GithubAccessToken) SetupAuthentication(req *http.Request) errors.Error {
+func (conn *GithubConn) SetupAuthentication(req *http.Request) errors.Error {
 	// Rotates token on each request.
-	req.Header.Set("Authorization", fmt.Sprintf("Bearer %v", gat.tokens[gat.tokenIndex]))
-	// Set next token index
-	gat.tokenIndex = (gat.tokenIndex + 1) % len(gat.tokens)
+	if len(conn.tokens) > 0 {
+		req.Header.Set("Authorization", fmt.Sprintf("Bearer %v", conn.tokens[conn.tokenIndex]))
+		// Set next token index
+		conn.tokenIndex = (conn.tokenIndex + 1) % len(conn.tokens)
+	}
+
 	return nil
 }
 
-// GetTokensCount returns total number of tokens
 func (gat *GithubAccessToken) GetTokensCount() int {
 	return len(gat.tokens)
 }
@@ -75,3 +102,77 @@ func (GithubConnection) TableName() string {
 type GithubUserOfToken struct {
 	Login string `json:"login"`
 }
+
+type InstallationToken struct {
+	Token string `json:"token"`
+}
+
+type GithubApp struct {
+	ID   int32  `json:"id"`
+	Slug string `json:"slug"`
+}
+
+type GithubAppInstallation struct {
+	Id      int `json:"id"`
+	Account struct {
+		Login string `json:"login"`
+	} `json:"account"`
+}
+
+type GithubAppInstallationWithToken struct {
+	GithubAppInstallation
+	Token string
+}
+
+func (gak *GithubAppKey) CreateJwt() (string, errors.Error) {
+	token := jwt.New(jwt.SigningMethodRS256)
+	t := time.Now().Unix()
+
+	token.Claims = jwt.MapClaims{
+		"iat": t,
+		"exp": t + (10 * 60),
+		"iss": gak.AppId,
+	}
+
+	privateKey, err := jwt.ParseRSAPrivateKeyFromPEM([]byte(gak.SecretKey))
+	if err != nil {
+		return "", errors.BadInput.Wrap(err, "invalid private key")
+	}
+
+	tokenString, err := token.SignedString(privateKey)
+	if err != nil {
+		return "", errors.BadInput.Wrap(err, "invalid private key")
+	}
+
+	return tokenString, nil
+}
+
+func (gak *GithubAppKey) getInstallationAccessToken(
+	apiClient apihelperabstract.ApiClientAbstract,
+) (*InstallationToken, errors.Error) {
+
+	jwt, err := gak.CreateJwt()
+	if err != nil {
+		return nil, err
+	}
+
+	resp, err := apiClient.Post(fmt.Sprintf("/app/installations/%d/access_tokens", gak.InstallationID), nil, nil, http.Header{
+		"Authorization": []string{fmt.Sprintf("Bearer %s", jwt)},
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	body, err := errors.Convert01(io.ReadAll(resp.Body))
+	if err != nil {
+		return nil, err
+	}
+
+	var installationToken InstallationToken
+	err = errors.Convert(json.Unmarshal(body, &installationToken))
+	if err != nil {
+		return nil, err
+	}
+
+	return &installationToken, nil
+}
diff --git a/backend/plugins/github/models/migrationscripts/20230428_add_multi_auth.go b/backend/plugins/github/models/migrationscripts/20230428_add_multi_auth.go
new file mode 100644
index 000000000..ca0b6c912
--- /dev/null
+++ b/backend/plugins/github/models/migrationscripts/20230428_add_multi_auth.go
@@ -0,0 +1,63 @@
+/*
+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 migrationscripts
+
+import (
+	"github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/errors"
+)
+
+type githubMultiAuth20230428 struct {
+	AppId          string
+	SecretKey      string
+	AuthMethod     string
+	InstallationId int
+}
+
+func (githubMultiAuth20230428) TableName() string {
+	return "_tool_github_connections"
+}
+
+type addGithubMultiAuth struct{}
+
+func (*addGithubMultiAuth) Up(res context.BasicRes) errors.Error {
+	db := res.GetDal()
+	err := db.AutoMigrate(&githubMultiAuth20230428{})
+	if err != nil {
+		return err
+	}
+	err = db.UpdateColumn(
+		&GithubConnection20221111{},
+		`auth_method`,
+		"AccessToken",
+		dal.Where(`token IS NOT NULL`),
+	)
+	if err != nil {
+		return err
+	}
+	return err
+}
+
+func (*addGithubMultiAuth) Version() uint64 {
+	return 20230428000010
+}
+
+func (*addGithubMultiAuth) Name() string {
+	return "UpdateSchemas for addGithubMultiAuth"
+}
diff --git a/backend/plugins/github/models/migrationscripts/register.go b/backend/plugins/github/models/migrationscripts/register.go
index 5139a53f8..f11587f34 100644
--- a/backend/plugins/github/models/migrationscripts/register.go
+++ b/backend/plugins/github/models/migrationscripts/register.go
@@ -38,5 +38,6 @@ func All() []plugin.MigrationScript {
 		new(addEnvToRunAndJob),
 		new(addGithubCommitAuthorInfo),
 		new(fixRunNameToText),
+		new(addGithubMultiAuth),
 	}
 }
diff --git a/config-ui/src/plugins/register/github/api.ts b/config-ui/src/plugins/register/github/api.ts
index b0edde388..e202ffee7 100644
--- a/config-ui/src/plugins/register/github/api.ts
+++ b/config-ui/src/plugins/register/github/api.ts
@@ -25,6 +25,14 @@ type PaginationParams = {
 
 export const getUser = (prefix: string) => request(`${prefix}/user`);
 
+export const getInstallationRepos = (prefix: string, params: PaginationParams) =>
+  request(`${prefix}/installation/repositories`, {
+    method: 'get',
+    data: {
+      ...params,
+    },
+  });
+
 export const getUserOrgs = (prefix: string, params: PaginationParams) =>
   request(`${prefix}/user/orgs`, {
     method: 'get',
diff --git a/config-ui/src/plugins/register/github/components/miller-columns/use-miller-columns.ts b/config-ui/src/plugins/register/github/components/miller-columns/use-miller-columns.ts
index e93098581..04f003218 100644
--- a/config-ui/src/plugins/register/github/components/miller-columns/use-miller-columns.ts
+++ b/config-ui/src/plugins/register/github/components/miller-columns/use-miller-columns.ts
@@ -23,6 +23,7 @@ import { useProxyPrefix } from '@/hooks';
 
 import type { ScopeItemType } from '../../types';
 import * as API from '../../api';
+import { getConnection } from '@/pages/blueprint/connection-detail/api';
 
 const DEFAULT_PAGE_SIZE = 30;
 
@@ -83,42 +84,75 @@ export const useMillerColumns = ({ connectionId }: UseMillerColumnsProps) => {
 
   useEffect(() => {
     (async () => {
-      const user = await API.getUser(prefix);
-      const orgs = await API.getUserOrgs(prefix, {
-        page: 1,
-        per_page: DEFAULT_PAGE_SIZE,
-      });
-
-      const loaded = !orgs.length || orgs.length < DEFAULT_PAGE_SIZE;
-
-      setUser(user);
-      setLoaded(loaded, 'root', 2);
-      setItems([
-        {
-          parentId: null,
-          id: user.login,
-          title: user.login,
-          type: 'org',
-        },
-        ...formatOrgs(orgs),
-      ]);
+      const connection = await getConnection('github', connectionId);
+
+      if (connection.authMethod === 'AppKey') {
+        const appInstallationRepos = await API.getInstallationRepos(prefix, {
+          page: 1,
+          per_page: 1,
+        });
+
+        setUser(null);
+        setLoaded(true, 'root', 2);
+
+        if (appInstallationRepos.total_count === 0) {
+          setItems([]);
+        } else {
+          setItems([
+            {
+              parentId: null,
+              id: appInstallationRepos.repositories[0].owner.login,
+              title: appInstallationRepos.repositories[0].owner.login,
+              type: 'org',
+            } as any,
+          ])
+        }
+      } else {
+        const user = await API.getUser(prefix);
+        const orgs = await API.getUserOrgs(prefix, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+
+        const loaded = !orgs.length || orgs.length < DEFAULT_PAGE_SIZE;
+
+        setUser(user);
+        setLoaded(loaded, 'root', 2);
+        setItems([
+          {
+            parentId: null,
+            id: user.login,
+            title: user.login,
+            type: 'org',
+          },
+          ...formatOrgs(orgs),
+        ]);
+    }
     })();
   }, [prefix]);
 
   const onExpand = useCallback(
     async (id: McsID) => {
       const item = items.find((it) => it.id === id) as McsItem<ExtraType>;
-
-      const isUser = id === user.login;
-      const repos = isUser
-        ? await API.getUserRepos(prefix, {
-            page: 1,
-            per_page: DEFAULT_PAGE_SIZE,
-          })
-        : await API.getOrgRepos(prefix, item.title, {
-            page: 1,
-            per_page: DEFAULT_PAGE_SIZE,
-          });
+      let repos = [];
+
+      if (user && id === user.login) {
+        repos = await API.getUserRepos(prefix, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+      } else if (user) {
+        repos = await API.getOrgRepos(prefix, item.title, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+      } else {
+        const response = await API.getInstallationRepos(prefix, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+        repos = response.repositories;
+      }
 
       const loaded = !repos.length || repos.length < DEFAULT_PAGE_SIZE;
       setLoaded(loaded, id, 2);
@@ -134,18 +168,25 @@ export const useMillerColumns = ({ connectionId }: UseMillerColumnsProps) => {
     let loaded = false;
 
     if (id) {
-      const isUser = id === user.login;
       const item = items.find((it) => it.id === id) as McsItem<ExtraType>;
 
-      repos = isUser
-        ? await API.getUserRepos(prefix, {
-            page,
-            per_page: DEFAULT_PAGE_SIZE,
-          })
-        : await API.getOrgRepos(prefix, item.title, {
-            page,
-            per_page: DEFAULT_PAGE_SIZE,
-          });
+      if (user && id === user.login) {
+        repos = await API.getUserRepos(prefix, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+      } else if (user) {
+        repos = await API.getOrgRepos(prefix, item.title, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+      } else {
+        const response = await API.getInstallationRepos(prefix, {
+          page: 1,
+          per_page: DEFAULT_PAGE_SIZE,
+        });
+        repos = response.repositories;
+      }
 
       loaded = !repos.length || repos.length < DEFAULT_PAGE_SIZE;
     } else {
diff --git a/config-ui/src/plugins/register/github/config.tsx b/config-ui/src/plugins/register/github/config.tsx
index 85698f4bd..43a9db085 100644
--- a/config-ui/src/plugins/register/github/config.tsx
+++ b/config-ui/src/plugins/register/github/config.tsx
@@ -18,9 +18,10 @@
 
 import type { PluginConfigType } from '../../types';
 import { PluginType } from '../../types';
+import { pick } from 'lodash';
 
 import Icon from './assets/icon.svg';
-import { Token, Graphql } from './connection-fields';
+import { Token, Graphql, GithubApp, Authentication } from './connection-fields';
 
 export const GitHubConfig: PluginConfigType = {
   type: PluginType.Connection,
@@ -32,6 +33,7 @@ export const GitHubConfig: PluginConfigType = {
     docLink: 'https://devlake.apache.org/docs/Configuration/GitHub',
     initialValues: {
       endpoint: 'https://api.github.com/',
+      authMethod: 'AccessToken',
       enableGraphql: true,
     },
     fields: [
@@ -44,17 +46,35 @@ export const GitHubConfig: PluginConfigType = {
         },
       },
       ({ initialValues, values, errors, setValues, setErrors }: any) => (
-        <Token
-          key="token"
-          endpoint={values.endpoint}
-          proxy={values.proxy}
-          initialValue={initialValues.token ?? ''}
-          value={values.token ?? ''}
-          error={errors.token ?? ''}
-          setValue={(value) => setValues({ token: value })}
-          setError={(value) => setErrors({ token: value })}
+        <Authentication
+          key="authMethod"
+          initialValue={initialValues.authMethod ?? ''}
+          value={values.authMethod ?? ''}
+          setValue={(value) => setValues({ authMethod: value })}
         />
       ),
+      ({ initialValues, values, errors, setValues, setErrors }: any) =>
+        (values.authMethod || initialValues.authMethod) == 'AccessToken' ? (
+          <Token
+            endpoint={values.endpoint}
+            proxy={values.proxy}
+            initialValue={initialValues.token ?? ''}
+            value={values.token ?? ''}
+            error={errors.token ?? ''}
+            setValue={(value) => setValues({ token: value })}
+            setError={(value) => setErrors({ token: value })}
+          />
+        ) : (
+          <GithubApp
+            endpoint={values.endpoint}
+            proxy={values.proxy}
+            initialValue={initialValues ? pick(initialValues, ['appId', 'secretKey', 'installationId']) : {}}
+            value={values ? pick(values, ['appId', 'secretKey', 'installationId']) : {}}
+            error={errors ?? {}}
+            setValue={(value) => setValues(value)}
+            setError={(value) => setErrors(value)}
+          />
+        ),
       'proxy',
       ({ initialValues, values, setValues }: any) => (
         <Graphql
diff --git a/config-ui/src/store/connections/api.ts b/config-ui/src/plugins/register/github/connection-fields/authentication.tsx
similarity index 50%
copy from config-ui/src/store/connections/api.ts
copy to config-ui/src/plugins/register/github/connection-fields/authentication.tsx
index d84d04852..8ddb6547d 100644
--- a/config-ui/src/store/connections/api.ts
+++ b/config-ui/src/plugins/register/github/connection-fields/authentication.tsx
@@ -16,32 +16,33 @@
  *
  */
 
-import { request } from '@/utils';
+import React, { useEffect } from 'react';
+import { FormGroup, RadioGroup, Radio } from '@blueprintjs/core';
 
-type GetConnectionRes = {
-  id: ID;
-  name: string;
-  endpoint: string;
-  proxy: string;
-  token?: string;
-  username?: string;
-  password?: string;
-  authMethod?: string;
-};
+import * as S from './styled';
 
-export const getConnection = (plugin: string): Promise<GetConnectionRes[]> => request(`/plugins/${plugin}/connections`);
+interface Props {
+  initialValue: string;
+  value: string;
+  setValue: (value: string) => void;
+}
 
-type TestConnectionPayload = {
-  endpoint: string;
-  proxy: string;
-  token?: string;
-  username?: string;
-  password?: string;
-  authMethod?: string;
-};
+export const Authentication = ({ initialValue, value, setValue }: Props) => {
 
-export const testConnection = (plugin: string, data: TestConnectionPayload) =>
-  request(`/plugins/${plugin}/test`, {
-    method: 'post',
-    data,
-  });
+  useEffect(() => {
+    setValue(initialValue);
+  }, [initialValue]);
+
+  return (
+    <FormGroup label={<S.Label>Authentication type</S.Label>} labelInfo={<S.LabelInfo>*</S.LabelInfo>}>
+      <RadioGroup inline selectedValue={value || initialValue} onChange={(e) => {
+        setValue((e.target as any).value);
+      }}>
+        <Radio value="AccessToken">Github Access Token</Radio>
+        <Radio value="AppKey">
+          Github App
+        </Radio>
+      </RadioGroup>
+    </FormGroup>
+  );
+};
diff --git a/config-ui/src/plugins/register/github/connection-fields/githubapp.tsx b/config-ui/src/plugins/register/github/connection-fields/githubapp.tsx
new file mode 100644
index 000000000..6691c9025
--- /dev/null
+++ b/config-ui/src/plugins/register/github/connection-fields/githubapp.tsx
@@ -0,0 +1,217 @@
+/*
+ * 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.
+ *
+ */
+
+import React, { useEffect, useState } from 'react';
+import { Button, FormGroup, InputGroup, MenuItem, TextArea } from '@blueprintjs/core';
+import { Select2 } from '@blueprintjs/select';
+
+import { ExternalLink } from '@/components';
+
+import * as API from '../api';
+
+import * as S from './styled';
+
+interface Props {
+  endpoint?: string;
+  proxy?: string;
+  initialValue: any;
+  value: any;
+  error: string;
+  setValue: (value: any) => void;
+  setError: (error: any) => void;
+}
+
+interface GithubAppSettings {
+  appId?: string;
+  secretKey?: string;
+  installationId?: number;
+
+  status: 'idle' | 'valid' | 'invalid';
+  from?: string;
+  installations?: GithubInstallation[];
+}
+
+interface GithubInstallation {
+  id: number;
+  account: {
+    login: string;
+  };
+}
+
+export const GithubApp = ({ endpoint, proxy, initialValue, value, error, setValue, setError }: Props) => {
+  const [settings, setSettings] = useState<GithubAppSettings>({ status: 'idle' });
+
+  useEffect(() => {
+    setError({
+      appId: value.appId ? '' : 'AppId is required',
+      secretKey: value.secretKey ? '' : 'SecretKey is required',
+      installationId: value.installationId ? '' : 'InstallationId is required',
+    });
+
+    return () => {
+      setError({
+        appId: '',
+        secretKey: '',
+        installationId: '',
+      });
+    }
+  }, [value.appId, value.secretKey, value.installationId]);
+
+  const testConfiguration = async (appId?: string, secretKey?: string, installationId?: number): Promise<GithubAppSettings> => {
+    if (!endpoint || !appId || !secretKey) {
+      return {
+        appId,
+        secretKey,
+        installationId,
+        status: 'idle',
+      };
+    }
+
+    try {
+      const res = await API.testConnection({
+        authMethod: 'AppKey',
+        endpoint,
+        proxy,
+        appId,
+        secretKey,
+        token: '',
+      });
+      return {
+        appId,
+        secretKey,
+        installationId,
+        status: 'valid',
+        from: res.login,
+        installations: res.installations,
+      };
+    } catch {
+      return {
+        appId,
+        secretKey,
+        installationId,
+        status: 'invalid',
+      };
+    }
+  };
+
+  const handleChangeAppId = (value: string) => {
+    setSettings({ ...settings, appId: value });;
+  };
+
+  const handleChangeClientSecret = (value: string) => {
+    setSettings({ ...settings, secretKey: value });
+  };
+
+  const handleTestConfiguration = async () => {
+    const res = await testConfiguration(settings.appId, settings.secretKey, settings.installationId);
+    setSettings(res);
+  };
+
+  const checkConfig = async (appId: string, secretKey: string, installationId: number) => {
+    const res = await testConfiguration(appId, secretKey, installationId);
+    setSettings(res);
+  };
+
+
+  useEffect(() => {
+    checkConfig(initialValue.appId, initialValue.secretKey, initialValue.installationId);
+  }, [initialValue.appId, initialValue.secretKey, initialValue.installationId, endpoint]);
+
+  useEffect(() => {
+    setValue({ appId: settings.appId, secretKey: settings.secretKey, installationId: settings.installationId });
+  }, [settings.appId, settings.secretKey, settings.installationId]);
+
+
+  return (
+    <FormGroup
+      label={<S.Label>Github App settings</S.Label>}
+      labelInfo={<S.LabelInfo>*</S.LabelInfo>}
+      subLabel={
+        <S.LabelDescription>
+          Input information about your Github App{' '}
+          <ExternalLink link="https://TODO">
+            Learn how to create a github app
+          </ExternalLink>
+        </S.LabelDescription>
+      }
+    >
+      <S.Input>
+        <div className="input">
+          <InputGroup
+            placeholder="App Id"
+            type="text"
+            value={settings.appId ?? ''}
+            onChange={(e) => handleChangeAppId(e.target.value)}
+            onBlur={() => handleTestConfiguration()}
+          />
+          <div className="info">
+            {settings.status === 'invalid' && <span className="error">Invalid</span>}
+            {settings.status === 'valid' && <span className="success">Valid From: {settings.from}</span>}
+          </div>
+        </div>
+      </S.Input>
+      <S.Input>
+        <div className="input">
+          <TextArea
+            cols={90}
+            rows={15}
+            placeholder="Private key"
+            value={settings.secretKey ?? ''}
+            onChange={(e) => handleChangeClientSecret(e.target.value)}
+            onBlur={() => handleTestConfiguration()}
+          />
+          <div className="info">
+            {settings.status === 'invalid' && <span className="error">Invalid</span>}
+            {settings.status === 'valid' && <span className="success">Valid From: {settings.from}</span>}
+          </div>
+        </div>
+      </S.Input>
+      <S.Input>
+        <Select2
+          items={settings.installations ?? []}
+          activeItem={settings.installations?.find(e => e.id === settings.installationId)}
+          itemPredicate={(query, item) => item.account.login.toLowerCase().includes(query.toLowerCase())}
+          itemRenderer={(item, { handleClick, handleFocus, modifiers }) => {
+            return (
+              <MenuItem
+                active={modifiers.active}
+                disabled={modifiers.disabled}
+                key={item.id}
+                label={item.id.toString()}
+                onClick={handleClick}
+                onFocus={handleFocus}
+                roleStructure="listoption"
+                text={item.account.login}
+            />
+            );
+          }}
+          onItemSelect={(item) => {
+            setSettings({ ...settings, installationId: item.id });
+          }}
+          noResults={<option disabled={true}>No results</option>}
+          popoverProps={{ minimal: true }}
+        >
+          <Button
+            text={settings.installations?.find(e => e.id === settings.installationId)?.account.login ?? 'Select App installation'}
+            rightIcon="double-caret-vertical"
+            placeholder="Select App installation" />
+        </Select2>
+      </S.Input>
+    </FormGroup>
+  );
+};
diff --git a/config-ui/src/plugins/register/github/connection-fields/index.ts b/config-ui/src/plugins/register/github/connection-fields/index.ts
index fc074058f..8e0a150a7 100644
--- a/config-ui/src/plugins/register/github/connection-fields/index.ts
+++ b/config-ui/src/plugins/register/github/connection-fields/index.ts
@@ -18,3 +18,5 @@
 
 export * from './token';
 export * from './graphql';
+export * from './githubapp';
+export * from './authentication';
\ No newline at end of file
diff --git a/config-ui/src/plugins/register/github/connection-fields/styled.ts b/config-ui/src/plugins/register/github/connection-fields/styled.ts
index a4119ee73..757d449c2 100644
--- a/config-ui/src/plugins/register/github/connection-fields/styled.ts
+++ b/config-ui/src/plugins/register/github/connection-fields/styled.ts
@@ -38,7 +38,7 @@ export const Endpoint = styled.div`
   }
 `;
 
-export const Token = styled.div`
+export const Input = styled.div`
   margin-bottom: 8px;
 
   .input {
diff --git a/config-ui/src/plugins/register/github/connection-fields/token.tsx b/config-ui/src/plugins/register/github/connection-fields/token.tsx
index a42d4bc26..b811a072d 100644
--- a/config-ui/src/plugins/register/github/connection-fields/token.tsx
+++ b/config-ui/src/plugins/register/github/connection-fields/token.tsx
@@ -55,6 +55,7 @@ export const Token = ({ endpoint, proxy, initialValue, value, error, setValue, s
 
     try {
       const res = await API.testConnection({
+        authMethod: 'AccessToken',
         endpoint,
         proxy,
         token,
@@ -84,6 +85,10 @@ export const Token = ({ endpoint, proxy, initialValue, value, error, setValue, s
 
   useEffect(() => {
     setError(value ? '' : 'token is required');
+
+    return () => {
+      setError('');
+    }
   }, [value]);
 
   useEffect(() => {
@@ -120,7 +125,7 @@ export const Token = ({ endpoint, proxy, initialValue, value, error, setValue, s
       }
     >
       {tokens.map(({ value, status, from, message }, i) => (
-        <S.Token key={i}>
+        <S.Input key={i}>
           <div className="input">
             <InputGroup
               placeholder="Token"
@@ -141,7 +146,7 @@ export const Token = ({ endpoint, proxy, initialValue, value, error, setValue, s
               {message}
             </div>
           )}
-        </S.Token>
+        </S.Input>
       ))}
       <div className="action">
         <Button outlined small intent={Intent.PRIMARY} text="Another Token" icon="plus" onClick={handleCreateToken} />
diff --git a/config-ui/src/store/connections/api.ts b/config-ui/src/store/connections/api.ts
index d84d04852..b7c443272 100644
--- a/config-ui/src/store/connections/api.ts
+++ b/config-ui/src/store/connections/api.ts
@@ -38,6 +38,8 @@ type TestConnectionPayload = {
   username?: string;
   password?: string;
   authMethod?: string;
+  appId?: string;
+  secretKey?: string;
 };
 
 export const testConnection = (plugin: string, data: TestConnectionPayload) =>
diff --git a/config-ui/src/store/connections/context.tsx b/config-ui/src/store/connections/context.tsx
index e0b03adc8..b9a1fcea0 100644
--- a/config-ui/src/store/connections/context.tsx
+++ b/config-ui/src/store/connections/context.tsx
@@ -70,6 +70,8 @@ export const ConnectionContextProvider = ({ children, ...props }: Props) => {
     username,
     password,
     authMethod,
+    secretKey,
+    appId,
   }: ConnectionItemType) => {
     try {
       const res = await API.testConnection(plugin, {
@@ -79,6 +81,8 @@ export const ConnectionContextProvider = ({ children, ...props }: Props) => {
         username,
         password,
         authMethod,
+        secretKey,
+        appId,
       });
       return res.success ? ConnectionStatusEnum.ONLINE : ConnectionStatusEnum.OFFLINE;
     } catch {
@@ -103,9 +107,13 @@ export const ConnectionContextProvider = ({ children, ...props }: Props) => {
       username: it.username,
       password: it.password,
       authMethod: it.authMethod,
+      secretKey: it.secretKey,
+      appId: it.appId,
     }));
   };
 
+
+
   const handleGet = (unique: string) => {
     return connections.find((cs) => cs.unique === unique) as ConnectionItemType;
   };
diff --git a/config-ui/src/store/connections/types.ts b/config-ui/src/store/connections/types.ts
index cbf96b32a..68847166f 100644
--- a/config-ui/src/store/connections/types.ts
+++ b/config-ui/src/store/connections/types.ts
@@ -39,4 +39,6 @@ export type ConnectionItemType = {
   username?: string;
   password?: string;
   authMethod?: string;
+  appId?: string;
+  secretKey?: string;
 };