Git LFS support v2 (#122)

* Import github.com/git-lfs/lfs-test-server as lfs module base

Imported commit is 3968aac269a77b73924649b9412ae03f7ccd3198

Removed:

Dockerfile CONTRIBUTING.md mgmt* script/ vendor/ kvlogger.go
.dockerignore .gitignore README.md

* Remove config, add JWT support from github.com/mgit-at/lfs-test-server

Imported commit f0cdcc5a01599c5a955dc1bbf683bb4acecdba83

* Add LFS settings

* Add LFS meta object model

* Add LFS routes and initialization

* Import github.com/dgrijalva/jwt-go into vendor/

* Adapt LFS module: handlers, routing, meta store

* Move LFS routes to /user/repo/info/lfs/*

* Add request header checks to LFS BatchHandler / PostHandler

* Implement LFS basic authentication

* Rework JWT secret generation / load

* Implement LFS SSH token authentication with JWT

Specification: https://github.com/github/git-lfs/tree/master/docs/api

* Integrate LFS settings into install process

* Remove LFS objects when repository is deleted

Only removes objects from content store when deleted repo is the only
referencing repository

* Make LFS module stateless

Fixes bug where LFS would not work after installation without
restarting Gitea

* Change 500 'Internal Server Error' to 400 'Bad Request'

* Change sql query to xorm call

* Remove unneeded type from LFS module

* Change internal imports to code.gitea.io/gitea/

* Add Gitea authors copyright

* Change basic auth realm to "gitea-lfs"

* Add unique indexes to LFS model

* Use xorm count function in LFS check on repository delete

* Return io.ReadCloser from content store and close after usage

* Add LFS info to runWeb()

* Export LFS content store base path

* LFS file download from UI

* Work around git-lfs client issue with unauthenticated requests

Returning a dummy Authorization header for unauthenticated requests
lets git-lfs client skip asking for auth credentials
See: https://github.com/github/git-lfs/issues/1088

* Fix unauthenticated UI downloads from public repositories

* Authentication check order, Finish LFS file view logic

* Ignore LFS hooks if installed for current OS user

Fixes Gitea UI actions for repositories tracking LFS files.
Checks for minimum needed git version by parsing the semantic version
string.

* Hide LFS metafile diff from commit view, marking as binary

* Show LFS notice if file in commit view is tracked

* Add notbefore/nbf JWT claim

* Correct lint suggestions - comments for structs and functions

- Add comments to LFS model
- Function comment for GetRandomBytesAsBase64
- LFS server function comments and lint variable suggestion

* Move secret generation code out of conditional

Ensures no LFS code may run with an empty secret

* Do not hand out JWT tokens if LFS server support is disabled
release/v1.1
Fabian Zaremba 7 years ago committed by Lunny Xiao
parent 4b7594d9fa
commit 2e7ccecfe6

@ -7,6 +7,7 @@ package cmd
import (
"crypto/tls"
"encoding/json"
"fmt"
"os"
"os/exec"
@ -21,12 +22,14 @@ import (
"code.gitea.io/gitea/modules/log"
"code.gitea.io/gitea/modules/setting"
"github.com/Unknwon/com"
"github.com/dgrijalva/jwt-go"
gouuid "github.com/satori/go.uuid"
"github.com/urfave/cli"
)
const (
accessDenied = "Repository does not exist or you do not have access"
accessDenied = "Repository does not exist or you do not have access"
lfsAuthenticateVerb = "git-lfs-authenticate"
)
// CmdServ represents the available serv sub-command.
@ -73,6 +76,7 @@ var (
"git-upload-pack": models.AccessModeRead,
"git-upload-archive": models.AccessModeRead,
"git-receive-pack": models.AccessModeWrite,
lfsAuthenticateVerb: models.AccessModeNone,
}
)
@ -161,6 +165,21 @@ func runServ(c *cli.Context) error {
}
verb, args := parseCmd(cmd)
var lfsVerb string
if verb == lfsAuthenticateVerb {
if !setting.LFS.StartServer {
fail("Unknown git command", "LFS authentication request over SSH denied, LFS support is disabled")
}
if strings.Contains(args, " ") {
argsSplit := strings.SplitN(args, " ", 2)
args = strings.TrimSpace(argsSplit[0])
lfsVerb = strings.TrimSpace(argsSplit[1])
}
}
repoPath := strings.ToLower(strings.Trim(args, "'"))
rr := strings.SplitN(repoPath, "/", 2)
if len(rr) != 2 {
@ -196,6 +215,14 @@ func runServ(c *cli.Context) error {
fail("Unknown git command", "Unknown git command %s", verb)
}
if verb == lfsAuthenticateVerb {
if lfsVerb == "upload" {
requestedMode = models.AccessModeWrite
} else {
requestedMode = models.AccessModeRead
}
}
// Prohibit push to mirror repositories.
if requestedMode > models.AccessModeRead && repo.IsMirror {
fail("mirror repository is read-only", "")
@ -261,6 +288,41 @@ func runServ(c *cli.Context) error {
}
}
//LFS token authentication
if verb == lfsAuthenticateVerb {
url := fmt.Sprintf("%s%s/%s.git/info/lfs", setting.AppURL, repoUser.Name, repo.Name)
now := time.Now()
token := jwt.NewWithClaims(jwt.SigningMethodHS256, jwt.MapClaims{
"repo": repo.ID,
"op": lfsVerb,
"exp": now.Add(5 * time.Minute).Unix(),
"nbf": now.Unix(),
})
// Sign and get the complete encoded token as a string using the secret
tokenString, err := token.SignedString(setting.LFS.JWTSecretBytes)
if err != nil {
fail("Internal error", "Failed to sign JWT token: %v", err)
}
tokenAuthentication := &models.LFSTokenResponse{
Header: make(map[string]string),
Href: url,
}
tokenAuthentication.Header["Authorization"] = fmt.Sprintf("Bearer %s", tokenString)
enc := json.NewEncoder(os.Stdout)
err = enc.Encode(tokenAuthentication)
if err != nil {
fail("Internal error", "Failed to encode LFS json response: %v", err)
}
return nil
}
uuid := gouuid.NewV4().String()
os.Setenv("GITEA_UUID", uuid)
// Keep the old env variable name for backward compability

@ -17,6 +17,7 @@ import (
"code.gitea.io/gitea/models"
"code.gitea.io/gitea/modules/auth"
"code.gitea.io/gitea/modules/context"
"code.gitea.io/gitea/modules/lfs"
"code.gitea.io/gitea/modules/log"
"code.gitea.io/gitea/modules/options"
"code.gitea.io/gitea/modules/public"
@ -29,6 +30,7 @@ import (
"code.gitea.io/gitea/routers/org"
"code.gitea.io/gitea/routers/repo"
"code.gitea.io/gitea/routers/user"
"github.com/go-macaron/binding"
"github.com/go-macaron/cache"
"github.com/go-macaron/captcha"
@ -564,6 +566,12 @@ func runWeb(ctx *cli.Context) error {
}, ignSignIn, context.RepoAssignment(true), context.RepoRef())
m.Group("/:reponame", func() {
m.Group("/info/lfs", func() {
m.Post("/objects/batch", lfs.BatchHandler)
m.Get("/objects/:oid/:filename", lfs.ObjectOidHandler)
m.Any("/objects/:oid", lfs.ObjectOidHandler)
m.Post("/objects", lfs.PostHandler)
}, ignSignInAndCsrf)
m.Any("/*", ignSignInAndCsrf, repo.HTTP)
m.Head("/tasks/trigger", repo.TriggerTask)
})
@ -600,6 +608,10 @@ func runWeb(ctx *cli.Context) error {
}
log.Info("Listen: %v://%s%s", setting.Protocol, listenAddr, setting.AppSubURL)
if setting.LFS.StartServer {
log.Info("LFS server enabled")
}
var err error
switch setting.Protocol {
case setting.HTTP:

@ -200,6 +200,7 @@ type DiffFile struct {
IsCreated bool
IsDeleted bool
IsBin bool
IsLFSFile bool
IsRenamed bool
IsSubmodule bool
Sections []*DiffSection
@ -245,6 +246,7 @@ func ParsePatch(maxLines, maxLineCharacteres, maxFiles int, reader io.Reader) (*
leftLine, rightLine int
lineCount int
curFileLinesCount int
curFileLFSPrefix bool
)
input := bufio.NewReader(reader)
@ -268,6 +270,28 @@ func ParsePatch(maxLines, maxLineCharacteres, maxFiles int, reader io.Reader) (*
continue
}
trimLine := strings.Trim(line, "+- ")
if trimLine == LFSMetaFileIdentifier {
curFileLFSPrefix = true
}
if curFileLFSPrefix && strings.HasPrefix(trimLine, LFSMetaFileOidPrefix) {
oid := strings.TrimPrefix(trimLine, LFSMetaFileOidPrefix)
if len(oid) == 64 {
m := &LFSMetaObject{Oid: oid}
count, err := x.Count(m)
if err == nil && count > 0 {
curFile.IsBin = true
curFile.IsLFSFile = true
curSection.Lines = nil
break
}
}
}
curFileLinesCount++
lineCount++
@ -354,6 +378,7 @@ func ParsePatch(maxLines, maxLineCharacteres, maxFiles int, reader io.Reader) (*
break
}
curFileLinesCount = 0
curFileLFSPrefix = false
// Check file diff type and is submodule.
for {

@ -0,0 +1,122 @@
package models
import (
"errors"
"github.com/go-xorm/xorm"
"time"
)
// LFSMetaObject stores metadata for LFS tracked files.
type LFSMetaObject struct {
ID int64 `xorm:"pk autoincr"`
Oid string `xorm:"UNIQUE(s) INDEX NOT NULL"`
Size int64 `xorm:"NOT NULL"`
RepositoryID int64 `xorm:"UNIQUE(s) INDEX NOT NULL"`
Existing bool `xorm:"-"`
Created time.Time `xorm:"-"`
CreatedUnix int64
}
// LFSTokenResponse defines the JSON structure in which the JWT token is stored.
// This structure is fetched via SSH and passed by the Git LFS client to the server
// endpoint for authorization.
type LFSTokenResponse struct {
Header map[string]string `json:"header"`
Href string `json:"href"`
}
var (
// ErrLFSObjectNotExist is returned from lfs models functions in order
// to differentiate between database and missing object errors.
ErrLFSObjectNotExist = errors.New("LFS Meta object does not exist")
)
const (
// LFSMetaFileIdentifier is the string appearing at the first line of LFS pointer files.
// https://github.com/git-lfs/git-lfs/blob/master/docs/spec.md
LFSMetaFileIdentifier = "version https://git-lfs.github.com/spec/v1"
// LFSMetaFileOidPrefix appears in LFS pointer files on a line before the sha256 hash.
LFSMetaFileOidPrefix = "oid sha256:"
)
// NewLFSMetaObject stores a given populated LFSMetaObject structure in the database
// if it is not already present.
func NewLFSMetaObject(m *LFSMetaObject) (*LFSMetaObject, error) {
var err error
has, err := x.Get(m)
if err != nil {
return nil, err
}
if has {
m.Existing = true
return m, nil
}
sess := x.NewSession()
defer sessionRelease(sess)
if err = sess.Begin(); err != nil {
return nil, err
}
if _, err = sess.Insert(m); err != nil {
return nil, err
}
return m, sess.Commit()
}
// GetLFSMetaObjectByOid selects a LFSMetaObject entry from database by its OID.
// It may return ErrLFSObjectNotExist or a database error. If the error is nil,
// the returned pointer is a valid LFSMetaObject.
func GetLFSMetaObjectByOid(oid string) (*LFSMetaObject, error) {
if len(oid) == 0 {
return nil, ErrLFSObjectNotExist
}
m := &LFSMetaObject{Oid: oid}
has, err := x.Get(m)
if err != nil {
return nil, err
} else if !has {
return nil, ErrLFSObjectNotExist
}
return m, nil
}
// RemoveLFSMetaObjectByOid removes a LFSMetaObject entry from database by its OID.
// It may return ErrLFSObjectNotExist or a database error.
func RemoveLFSMetaObjectByOid(oid string) error {
if len(oid) == 0 {
return ErrLFSObjectNotExist
}
sess := x.NewSession()
defer sessionRelease(sess)
if err := sess.Begin(); err != nil {
return err
}
m := &LFSMetaObject{Oid: oid}
if _, err := sess.Delete(m); err != nil {
return err
}
return sess.Commit()
}
// BeforeInsert sets the time at which the LFSMetaObject was created.
func (m *LFSMetaObject) BeforeInsert() {
m.CreatedUnix = time.Now().Unix()
}
// AfterSet stores the LFSMetaObject creation time in the database as local time.
func (m *LFSMetaObject) AfterSet(colName string, _ xorm.Cell) {
switch colName {
case "created_unix":
m.Created = time.Unix(m.CreatedUnix, 0).Local()
}
}

@ -79,7 +79,7 @@ func init() {
new(Mirror), new(Release), new(LoginSource), new(Webhook),
new(UpdateTask), new(HookTask),
new(Team), new(OrgUser), new(TeamUser), new(TeamRepo),
new(Notice), new(EmailAddress))
new(Notice), new(EmailAddress), new(LFSMetaObject))
gonicNames := []string{"SSL", "UID"}
for _, name := range gonicNames {

@ -1480,6 +1480,35 @@ func DeleteRepository(uid, repoID int64) error {
RemoveAllWithNotice("Delete attachment", attachmentPaths[i])
}
// Remove LFS objects
var lfsObjects []*LFSMetaObject
if err = sess.Where("repository_id=?", repoID).Find(&lfsObjects); err != nil {
return err
}
for _, v := range lfsObjects {
count, err := sess.Count(&LFSMetaObject{Oid: v.Oid})
if err != nil {
return err
}
if count > 1 {
continue
}
oidPath := filepath.Join(v.Oid[0:2], v.Oid[2:4], v.Oid[4:len(v.Oid)])
err = os.Remove(filepath.Join(setting.LFS.ContentPath, oidPath))
if err != nil {
return err
}
}
if _, err := sess.Delete(&LFSMetaObject{RepositoryID: repoID}); err != nil {
return err
}
if err = sess.Commit(); err != nil {
return fmt.Errorf("Commit: %v", err)
}
@ -2240,6 +2269,34 @@ func ForkRepository(u *User, oldRepo *Repository, name, desc string) (_ *Reposit
return nil, fmt.Errorf("createUpdateHook: %v", err)
}
//Commit repo to get Fork ID
err = sess.Commit()
if err != nil {
return nil, err
}
sessionRelease(sess)
// Copy LFS meta objects in new session
sess = x.NewSession()
defer sessionRelease(sess)
if err = sess.Begin(); err != nil {
return nil, err
}
var lfsObjects []*LFSMetaObject
if err = sess.Where("repository_id=?", oldRepo.ID).Find(&lfsObjects); err != nil {
return nil, err
}
for _, v := range lfsObjects {
v.ID = 0
v.RepositoryID = repo.ID
if _, err = sess.Insert(v); err != nil {
return nil, err
}
}
return repo, sess.Commit()
}

@ -23,6 +23,7 @@ type InstallForm struct {
AppName string `binding:"Required" locale:"install.app_name"`
RepoRootPath string `binding:"Required"`
LFSRootPath string
RunUser string `binding:"Required"`
Domain string `binding:"Required"`
SSHPort int

@ -12,6 +12,7 @@ import (
"encoding/hex"
"fmt"
"html/template"
"io"
"math"
"math/big"
"net/http"
@ -103,6 +104,18 @@ func GetRandomString(n int) (string, error) {
return string(buffer), nil
}
// GetRandomBytesAsBase64 generates a random base64 string from n bytes
func GetRandomBytesAsBase64(n int) string {
bytes := make([]byte, 32)
_, err := io.ReadFull(rand.Reader, bytes)
if err != nil {
log.Fatal(4, "Error reading random bytes: %s", err)
}
return base64.RawURLEncoding.EncodeToString(bytes)
}
func randomInt(max *big.Int) (int, error) {
rand, err := rand.Int(rand.Reader, max)
if err != nil {

@ -0,0 +1,20 @@
Copyright (c) 2016 The Gitea Authors
Copyright (c) GitHub, Inc. and LFS Test Server contributors
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.

@ -0,0 +1,94 @@
package lfs
import (
"code.gitea.io/gitea/models"
"crypto/sha256"
"encoding/hex"
"errors"
"io"
"os"
"path/filepath"
)
var (
errHashMismatch = errors.New("Content hash does not match OID")
errSizeMismatch = errors.New("Content size does not match")
)
// ContentStore provides a simple file system based storage.
type ContentStore struct {
BasePath string
}
// Get takes a Meta object and retreives the content from the store, returning
// it as an io.Reader. If fromByte > 0, the reader starts from that byte
func (s *ContentStore) Get(meta *models.LFSMetaObject, fromByte int64) (io.ReadCloser, error) {
path := filepath.Join(s.BasePath, transformKey(meta.Oid))
f, err := os.Open(path)
if err != nil {
return nil, err
}
if fromByte > 0 {
_, err = f.Seek(fromByte, os.SEEK_CUR)
}
return f, err
}
// Put takes a Meta object and an io.Reader and writes the content to the store.
func (s *ContentStore) Put(meta *models.LFSMetaObject, r io.Reader) error {
path := filepath.Join(s.BasePath, transformKey(meta.Oid))
tmpPath := path + ".tmp"
dir := filepath.Dir(path)
if err := os.MkdirAll(dir, 0750); err != nil {
return err
}
file, err := os.OpenFile(tmpPath, os.O_CREATE|os.O_WRONLY|os.O_EXCL, 0640)
if err != nil {
return err
}
defer os.Remove(tmpPath)
hash := sha256.New()
hw := io.MultiWriter(hash, file)
written, err := io.Copy(hw, r)
if err != nil {
file.Close()
return err
}
file.Close()
if written != meta.Size {
return errSizeMismatch
}
shaStr := hex.EncodeToString(hash.Sum(nil))
if shaStr != meta.Oid {
return errHashMismatch
}
if err := os.Rename(tmpPath, path); err != nil {
return err
}
return nil
}
// Exists returns true if the object exists in the content store.
func (s *ContentStore) Exists(meta *models.LFSMetaObject) bool {
path := filepath.Join(s.BasePath, transformKey(meta.Oid))
if _, err := os.Stat(path); os.IsNotExist(err) {
return false
}
return true
}
func transformKey(key string) string {
if len(key) < 5 {
return key
}
return filepath.Join(key[0:2], key[2:4], key[4:len(key)])
}

@ -0,0 +1,549 @@
package lfs
import (
"encoding/base64"
"encoding/json"
"fmt"
"io"
"net/http"
"regexp"
"strconv"
"strings"
"time"
"code.gitea.io/gitea/models"
"code.gitea.io/gitea/modules/context"
"code.gitea.io/gitea/modules/log"
"code.gitea.io/gitea/modules/setting"
"github.com/dgrijalva/jwt-go"
"gopkg.in/macaron.v1"
)
const (
contentMediaType = "application/vnd.git-lfs"
metaMediaType = contentMediaType + "+json"
)
// RequestVars contain variables from the HTTP request. Variables from routing, json body decoding, and
// some headers are stored.
type RequestVars struct {
Oid string
Size int64
User string
Password string
Repo string
Authorization string
}
// BatchVars contains multiple RequestVars processed in one batch operation.
// https://github.com/git-lfs/git-lfs/blob/master/docs/api/batch.md
type BatchVars struct {
Transfers []string `json:"transfers,omitempty"`
Operation string `json:"operation"`
Objects []*RequestVars `json:"objects"`
}
// BatchResponse contains multiple object metadata Representation structures
// for use with the batch API.
type BatchResponse struct {
Transfer string `json:"transfer,omitempty"`
Objects []*Representation `json:"objects"`
}
// Representation is object medata as seen by clients of the lfs server.
type Representation struct {
Oid string `json:"oid"`
Size int64 `json:"size"`
Actions map[string]*link `json:"actions"`
Error *ObjectError `json:"error,omitempty"`
}
// ObjectError defines the JSON structure returned to the client in case of an error
type ObjectError struct {
Code int `json:"code"`
Message string `json:"message"`
}
// ObjectLink builds a URL linking to the object.
func (v *RequestVars) ObjectLink() string {
return fmt.Sprintf("%s%s/%s/info/lfs/objects/%s", setting.AppURL, v.User, v.Repo, v.Oid)
}
// link provides a structure used to build a hypermedia representation of an HTTP link.
type link struct {
Href string `json:"href"`
Header map[string]string `json:"header,omitempty"`
ExpiresAt time.Time `json:"expires_at,omitempty"`
}
// ObjectOidHandler is the main request routing entry point into LFS server functions
func ObjectOidHandler(ctx *context.Context) {
if !setting.LFS.StartServer {
writeStatus(ctx, 404)
return
}
if ctx.Req.Method == "GET" || ctx.Req.Method == "HEAD" {
if MetaMatcher(ctx.Req) {
GetMetaHandler(ctx)
return
}
if ContentMatcher(ctx.Req) || len(ctx.Params("filename")) > 0 {
GetContentHandler(ctx)
return
}
} else if ctx.Req.Method == "PUT" && ContentMatcher(ctx.Req) {
PutHandler(ctx)
return
}
}
// GetContentHandler gets the content from the content store
func GetContentHandler(ctx *context.Context) {
rv := unpack(ctx)
meta, err := models.GetLFSMetaObjectByOid(rv.Oid)
if err != nil {
writeStatus(ctx, 404)
return
}
repository, err := models.GetRepositoryByID(meta.RepositoryID)
if err != nil {
writeStatus(ctx, 404)
return
}
if !authenticate(ctx, repository, rv.Authorization, false) {
requireAuth(ctx)
return
}
// Support resume download using Range header
var fromByte int64
statusCode := 200
if rangeHdr := ctx.Req.Header.Get("Range"); rangeHdr != "" {
regex := regexp.MustCompile(`bytes=(\d+)\-.*`)
match := regex.FindStringSubmatch(rangeHdr)
if match != nil && len(match) > 1 {
statusCode = 206
fromByte, _ = strconv.ParseInt(match[1], 10, 32)
ctx.Resp.Header().Set("Content-Range", fmt.Sprintf("bytes %d-%d/%d", fromByte, meta.Size-1, int64(meta.Size)-fromByte))
}
}
contentStore := &ContentStore{BasePath: setting.LFS.ContentPath}
content, err := contentStore.Get(meta, fromByte)
if err != nil {
writeStatus(ctx, 404)
return
}
ctx.Resp.Header().Set("Content-Length", strconv.FormatInt(meta.Size, 10))
ctx.Resp.Header().Set("Content-Type", "application/octet-stream")
filename := ctx.Params("filename")
if len(filename) > 0 {
decodedFilename, err := base64.RawURLEncoding.DecodeString(filename)
if err == nil {
ctx.Resp.Header().Set("Content-Disposition", "attachment; filename=\""+string(decodedFilename)+"\"")
}
}
ctx.Resp.WriteHeader(statusCode)
io.Copy(ctx.Resp, content)
content.Close()
logRequest(ctx.Req, statusCode)
}
// GetMetaHandler retrieves metadata about the object
func GetMetaHandler(ctx *context.Context) {
rv := unpack(ctx)
meta, err := models.GetLFSMetaObjectByOid(rv.Oid)
if err != nil {
writeStatus(ctx, 404)
return
}
repository, err := models.GetRepositoryByID(meta.RepositoryID)
if err != nil {
writeStatus(ctx, 404)
return
}
if !authenticate(ctx, repository, rv.Authorization, false) {
requireAuth(ctx)
return
}
ctx.Resp.Header().Set("Content-Type", metaMediaType)
if ctx.Req.Method == "GET" {
enc := json.NewEncoder(ctx.Resp)
enc.Encode(Represent(rv, meta, true, false))
}
logRequest(ctx.Req, 200)
}
// PostHandler instructs the client how to upload data
func PostHandler(ctx *context.Context) {
if !setting.LFS.StartServer {
writeStatus(ctx, 404)
return
}
if !MetaMatcher(ctx.Req) {
writeStatus(ctx, 400)
return
}
rv := unpack(ctx)
repositoryString := rv.User + "/" + rv.Repo
repository, err := models.GetRepositoryByRef(repositoryString)
if err != nil {
log.Debug("Could not find repository: %s - %s", repositoryString, err)
writeStatus(ctx, 404)
return
}
if !authenticate(ctx, repository, rv.Authorization, true) {
requireAuth(ctx)
}
meta, err := models.NewLFSMetaObject(&models.LFSMetaObject{Oid: rv.Oid, Size: rv.Size, RepositoryID: repository.ID})
if err != nil {
writeStatus(ctx, 404)
return
}
ctx.Resp.Header().Set("Content-Type", metaMediaType)
sentStatus := 202
contentStore := &ContentStore{BasePath: setting.LFS.ContentPath}
if meta.Existing && contentStore.Exists(meta) {
sentStatus = 200
}
ctx.Resp.WriteHeader(sentStatus)
enc := json.NewEncoder(ctx.Resp)
enc.Encode(Represent(rv, meta, meta.Existing, true))
logRequest(ctx.Req, sentStatus)
}
// BatchHandler provides the batch api
func BatchHandler(ctx *context.Context) {
if !setting.LFS.StartServer {
writeStatus(ctx, 404)
return
}
if !MetaMatcher(ctx.Req) {
writeStatus(ctx, 400)
return
}
bv := unpackbatch(ctx)
var responseObjects []*Representation
// Create a response object
for _, object := range bv.Objects {
repositoryString := object.User + "/" + object.Repo
repository, err := models.GetRepositoryByRef(repositoryString)
if err != nil {
log.Debug("Could not find repository: %s - %s", repositoryString, err)
writeStatus(ctx, 404)
return
}
requireWrite := false
if bv.Operation == "upload" {
requireWrite = true
}
if !authenticate(ctx, repository, object.Authorization, requireWrite) {
requireAuth(ctx)
return
}
meta, err := models.GetLFSMetaObjectByOid(object.Oid)
contentStore := &ContentStore{BasePath: setting.LFS.ContentPath}
if err == nil && contentStore.Exists(meta) { // Object is found and exists
responseObjects = append(responseObjects, Represent(object, meta, true, false))
continue
}
// Object is not found
meta, err = models.NewLFSMetaObject(&models.LFSMetaObject{Oid: object.Oid, Size: object.Size, RepositoryID: repository.ID})
if err == nil {
responseObjects = append(responseObjects, Represent(object, meta, meta.Existing, true))
}
}
ctx.Resp.Header().Set("Content-Type", metaMediaType)
respobj := &BatchResponse{Objects: responseObjects}
enc := json.NewEncoder(ctx.Resp)
enc.Encode(respobj)
logRequest(ctx.Req, 200)
}
// PutHandler receives data from the client and puts it into the content store
func PutHandler(ctx *context.Context) {
rv := unpack(ctx)
meta, err := models.GetLFSMetaObjectByOid(rv.Oid)
if err != nil {
writeStatus(ctx, 404)
return
}
repository, err := models.GetRepositoryByID(meta.RepositoryID)
if err != nil {
writeStatus(ctx, 404)
return
}
if !authenticate(ctx, repository, rv.Authorization, true) {
requireAuth(ctx)
return
}
contentStore := &ContentStore{BasePath: setting.LFS.ContentPath}
if err := contentStore.Put(meta, ctx.Req.Body().ReadCloser()); err != nil {
models.RemoveLFSMetaObjectByOid(rv.Oid)
ctx.Resp.WriteHeader(500)
fmt.Fprintf(ctx.Resp, `{"message":"%s"}`, err)
return
}
logRequest(ctx.Req, 200)
}
// Represent takes a RequestVars and Meta and turns it into a Representation suitable
// for json encoding
func Represent(rv *RequestVars, meta *models.LFSMetaObject, download, upload bool) *Representation {
rep := &Representation{
Oid: meta.Oid,
Size: meta.Size,
Actions: make(map[string]*link),
}
header := make(map[string]string)
header["Accept"] = contentMediaType
if rv.Authorization == "" {
//https://github.com/github/git-lfs/issues/1088
header["Authorization"] = "Authorization: Basic dummy"
} else {
header["Authorization"] = rv.Authorization
}
if download {
rep.Actions["download"] = &link{Href: rv.ObjectLink(), Header: header}
}
if upload {
rep.Actions["upload"] = &link{Href: rv.ObjectLink(), Header: header}
}
return rep
}
// ContentMatcher provides a mux.MatcherFunc that only allows requests that contain
// an Accept header with the contentMediaType
func ContentMatcher(r macaron.Request) bool {
mediaParts := strings.Split(r.Header.Get("Accept"), ";")
mt := mediaParts[0]
return mt == contentMediaType
}
// MetaMatcher provides a mux.MatcherFunc that only allows requests that contain
// an Accept header with the metaMediaType
func MetaMatcher(r macaron.Request) bool {
mediaParts := strings.Split(r.Header.Get("Accept"), ";")
mt := mediaParts[0]
return mt == metaMediaType
}
func unpack(ctx *context.Context) *RequestVars {
r := ctx.Req
rv := &RequestVars{
User: ctx.Params("username"),
Repo: strings.TrimSuffix(ctx.Params("reponame"), ".git"),
Oid: ctx.Params("oid"),
Authorization: r.Header.Get("Authorization"),
}
if r.Method == "POST" { // Maybe also check if +json
var p RequestVars
dec := json.NewDecoder(r.Body().ReadCloser())
err := dec.Decode(&p)
if err != nil {
return rv
}
rv.Oid = p.Oid
rv.Size = p.Size
}
return rv
}
// TODO cheap hack, unify with unpack
func unpackbatch(ctx *context.Context) *BatchVars {
r := ctx.Req
var bv BatchVars
dec := json.NewDecoder(r.Body().ReadCloser())
err := dec.Decode(&bv)
if err != nil {
return &bv
}
for i := 0; i < len(bv.Objects); i++ {
bv.Objects[i].User = ctx.Params("username")
bv.Objects[i].Repo = strings.TrimSuffix(ctx.Params("reponame"), ".git")
bv.Objects[i].Authorization = r.Header.Get("Authorization")
}
return &bv
}
func writeStatus(ctx *context.Context, status int) {
message := http.StatusText(status)
mediaParts := strings.Split(ctx.Req.Header.Get("Accept"), ";")
mt := mediaParts[0]
if strings.HasSuffix(mt, "+json") {
message = `{"message":"` + message + `"}`
}
ctx.Resp.WriteHeader(status)
fmt.Fprint(ctx.Resp, message)
logRequest(ctx.Req, status)
}
func logRequest(r macaron.Request, status int) {
log.Debug("LFS request - Method: %s, URL: %s, Status %d", r.Method, r.URL, status)
}
// authenticate uses the authorization string to determine whether
// or not to proceed. This server assumes an HTTP Basic auth format.
func authenticate(ctx *context.Context, repository *models.Repository, authorization string, requireWrite bool) bool {
accessMode := models.AccessModeRead
if requireWrite {
accessMode = models.AccessModeWrite
}
if !repository.IsPrivate && !requireWrite {
return true
}
if ctx.IsSigned {
accessCheck, _ := models.HasAccess(ctx.User, repository, accessMode)
return accessCheck
}
if authorization == "" {
return false
}
if authenticateToken(repository, authorization, requireWrite) {
return true
}
if !strings.HasPrefix(authorization, "Basic ") {
return false
}
c, err := base64.StdEncoding.DecodeString(strings.TrimPrefix(authorization, "Basic "))
if err != nil {
return false
}
cs := string(c)
i := strings.IndexByte(cs, ':')
if i < 0 {
return false
}
user, password := cs[:i], cs[i+1:]
userModel, err := models.GetUserByName(user)
if err != nil {
return false
}
if !userModel.ValidatePassword(password) {
return false
}
accessCheck, _ := models.HasAccess(userModel, repository, accessMode)
return accessCheck
}
func authenticateToken(repository *models.Repository, authorization string, requireWrite bool) bool {
if !strings.HasPrefix(authorization, "Bearer ") {
return false
}
token, err := jwt.Parse(authorization[7:], func(t *jwt.Token) (interface{}, error) {
if _, ok := t.Method.(*jwt.SigningMethodHMAC); !ok {
return nil, fmt.Errorf("unexpected signing method: %v", t.Header["alg"])
}
return setting.LFS.JWTSecretBytes, nil
})
if err != nil {
return false
}
claims, claimsOk := token.Claims.(jwt.MapClaims)
if !token.Valid || !claimsOk {
return false
}
opStr, ok := claims["op"].(string)
if !ok {
return false
}
if requireWrite && opStr != "upload" {
return false
}
repoID, ok := claims["repo"].(float64)
if !ok {
return false
}
if repository.ID != int64(repoID) {
return false
}
return true
}
func requireAuth(ctx *context.Context) {
ctx.Resp.Header().Set("WWW-Authenticate", "Basic realm=gitea-lfs")
writeStatus(ctx, 401)
}

@ -5,7 +5,10 @@
package setting
import (
"crypto/rand"
"encoding/base64"
"fmt"
"io"
"net/mail"
"net/url"
"os"
@ -17,6 +20,7 @@ import (
"strings"
"time"
"code.gitea.io/git"
"code.gitea.io/gitea/modules/log"
"code.gitea.io/gitea/modules/user"
"github.com/Unknwon/com"
@ -89,6 +93,13 @@ var (
MinimumKeySizes map[string]int `ini:"-"`
}
LFS struct {
StartServer bool `ini:"LFS_START_SERVER"`
ContentPath string `ini:"LFS_CONTENT_PATH"`
JWTSecretBase64 string `ini:"LFS_JWT_SECRET"`
JWTSecretBytes []byte `ini:"-"`
}
// Security settings
InstallLock bool
SecretKey string
@ -583,6 +594,85 @@ please consider changing to GITEA_CUSTOM`)
}
}
if err = Cfg.Section("server").MapTo(&LFS); err != nil {
log.Fatal(4, "Fail to map LFS settings: %v", err)
}
if LFS.StartServer {
if err := os.MkdirAll(LFS.ContentPath, 0700); err != nil {
log.Fatal(4, "Fail to create '%s': %v", LFS.ContentPath, err)
}
LFS.JWTSecretBytes = make([]byte, 32)
n, err := base64.RawURLEncoding.Decode(LFS.JWTSecretBytes, []byte(LFS.JWTSecretBase64))
if err != nil || n != 32 {
//Generate new secret and save to config
_, err := io.ReadFull(rand.Reader, LFS.JWTSecretBytes)
if err != nil {
log.Fatal(4, "Error reading random bytes: %s", err)
}
LFS.JWTSecretBase64 = base64.RawURLEncoding.EncodeToString(LFS.JWTSecretBytes)
// Save secret
cfg := ini.Empty()
if com.IsFile(CustomConf) {
// Keeps custom settings if there is already something.
if err := cfg.Append(CustomConf); err != nil {
log.Error(4, "Fail to load custom conf '%s': %v", CustomConf, err)
}
}
cfg.Section("server").Key("LFS_JWT_SECRET").SetValue(LFS.JWTSecretBase64)
os.MkdirAll(filepath.Dir(CustomConf), os.ModePerm)
if err := cfg.SaveTo(CustomConf); err != nil {
log.Fatal(4, "Error saving generated JWT Secret to custom config: %v", err)
return
}
}
//Disable LFS client hooks if installed for the current OS user
//Needs at least git v2.1.2
binVersion, err := git.BinVersion()
if err != nil {
log.Fatal(4, "Error retrieving git version: %s", err)
}
splitVersion := strings.SplitN(binVersion, ".", 3)
majorVersion, err := strconv.ParseUint(splitVersion[0], 10, 64)
if err != nil {
log.Fatal(4, "Error parsing git major version: %s", err)
}
minorVersion, err := strconv.ParseUint(splitVersion[1], 10, 64)
if err != nil {
log.Fatal(4, "Error parsing git minor version: %s", err)
}
revisionVersion, err := strconv.ParseUint(splitVersion[2], 10, 64)
if err != nil {
log.Fatal(4, "Error parsing git revision version: %s", err)
}
if !((majorVersion > 2) || (majorVersion == 2 && minorVersion > 1) ||
(majorVersion == 2 && minorVersion == 1 && revisionVersion >= 2)) {
LFS.StartServer = false
log.Error(4, "LFS server support needs at least Git v2.1.2")
} else {
git.GlobalCommandArgs = append(git.GlobalCommandArgs, "-c", "filter.lfs.required=",
"-c", "filter.lfs.smudge=", "-c", "filter.lfs.clean=")
}
}
sec = Cfg.Section("security")
InstallLock = sec.Key("INSTALL_LOCK").MustBool(false)
SecretKey = sec.Key("SECRET_KEY").MustString("!#@FDEWREWR&*(")

@ -69,6 +69,8 @@ app_name = Application Name
app_name_helper = Put your organization name here huge and loud!
repo_path = Repository Root Path
repo_path_helper = All Git remote repositories will be saved to this directory.
lfs_path = LFS Root Path
lfs_path_helper = Files stored with Git LFS will be stored in this directory. Leave empty to disable LFS.
run_user = Run User
run_user_helper = The user must have access to Repository Root Path and run Gitea.
domain = Domain
@ -432,6 +434,7 @@ file_view_raw = View Raw
file_permalink = Permalink
file_too_large = This file is too large to be shown
video_not_supported_in_browser = Your browser doesn't support HTML5 video tag.
stored_lfs = Stored with Git LFS
editor.new_file = New file
editor.upload_file = Upload file
@ -1060,6 +1063,7 @@ config.disable_router_log = Disable Router Log
config.run_user = Run User
config.run_mode = Run Mode
config.repo_root_path = Repository Root Path
config.lfs_root_path = LFS Root Path
config.static_file_root_path = Static File Root Path
config.log_file_root_path = Log File Root Path
config.script_type = Script Type

@ -79,6 +79,7 @@ func Install(ctx *context.Context) {
// Application general settings
form.AppName = setting.AppName
form.RepoRootPath = setting.RepoRootPath
form.LFSRootPath = setting.LFS.ContentPath
// Note(unknwon): it's hard for Windows users change a running user,
// so just use current one if config says default.
@ -183,6 +184,16 @@ func InstallPost(ctx *context.Context, form auth.InstallForm) {
return
}
// Test LFS root path if not empty, empty meaning disable LFS
if form.LFSRootPath != "" {
form.LFSRootPath = strings.Replace(form.LFSRootPath, "\\", "/", -1)
if err := os.MkdirAll(form.LFSRootPath, os.ModePerm); err != nil {
ctx.Data["Err_LFSRootPath"] = true
ctx.RenderWithErr(ctx.Tr("install.invalid_lfs_path", err), tplInstall, &form)
return
}
}
// Test log root path.
form.LogRootPath = strings.Replace(form.LogRootPath, "\\", "/", -1)
if err = os.MkdirAll(form.LogRootPath, os.ModePerm); err != nil {
@ -254,6 +265,14 @@ func InstallPost(ctx *context.Context, form auth.InstallForm) {
cfg.Section("server").Key("SSH_PORT").SetValue(com.ToStr(form.SSHPort))
}
if form.LFSRootPath != "" {
cfg.Section("server").Key("LFS_START_SERVER").SetValue("true")
cfg.Section("server").Key("LFS_CONTENT_PATH").SetValue(form.LFSRootPath)
cfg.Section("server").Key("LFS_JWT_SECRET").SetValue(base.GetRandomBytesAsBase64(32))
} else {
cfg.Section("server").Key("LFS_START_SERVER").SetValue("false")
}
if len(strings.TrimSpace(form.SMTPHost)) > 0 {
cfg.Section("mailer").Key("ENABLED").SetValue("true")
cfg.Section("mailer").Key("HOST").SetValue(form.SMTPHost)

@ -17,11 +17,14 @@ import (
"code.gitea.io/gitea/modules/base"
"code.gitea.io/gitea/modules/context"
"code.gitea.io/gitea/modules/highlight"
"code.gitea.io/gitea/modules/lfs"
"code.gitea.io/gitea/modules/log"
"code.gitea.io/gitea/modules/markdown"
"code.gitea.io/gitea/modules/setting"
"code.gitea.io/gitea/modules/templates"
"encoding/base64"
"github.com/Unknwon/paginater"
"strconv"
)
const (
@ -139,6 +142,30 @@ func renderFile(ctx *context.Context, entry *git.TreeEntry, treeLink, rawLink st
isTextFile := base.IsTextFile(buf)
ctx.Data["IsTextFile"] = isTextFile
//Check for LFS meta file
if isTextFile && setting.LFS.StartServer {
headString := string(buf)
if strings.HasPrefix(headString, models.LFSMetaFileIdentifier) {
splitLines := strings.Split(headString, "\n")
if len(splitLines) >= 3 {
oid := strings.TrimPrefix(splitLines[1], models.LFSMetaFileOidPrefix)
size, err := strconv.ParseInt(strings.TrimPrefix(splitLines[2], "size "), 10, 64)
if len(oid) == 64 && err == nil {
contentStore := &lfs.ContentStore{BasePath: setting.LFS.ContentPath}
meta := &models.LFSMetaObject{Oid: oid}
if contentStore.Exists(meta) {
ctx.Data["IsTextFile"] = false
isTextFile = false
ctx.Data["IsLFSFile"] = true
ctx.Data["FileSize"] = size
filenameBase64 := base64.RawURLEncoding.EncodeToString([]byte(blob.Name()))
ctx.Data["RawFileLink"] = fmt.Sprintf("%s%s/info/lfs/objects/%s/%s", setting.AppURL, ctx.Repo.Repository.FullName(), oid, filenameBase64)
}
}
}
}
}
// Assume file is not editable first.
if !isTextFile {
ctx.Data["EditFileTooltip"] = ctx.Tr("repo.editor.cannot_edit_non_text_files")

@ -84,6 +84,11 @@
<input id="repo_root_path" name="repo_root_path" value="{{.repo_root_path}}" required>
<span class="help">{{.i18n.Tr "install.repo_path_helper"}}</span>
</div>
<div class="inline field {{if .Err_LFSRootPath}}error{{end}}">
<label for="lfs_root_path">{{.i18n.Tr "install.lfs_path"}}</label>
<input id="lfs_root_path" name="lfs_root_path" value="{{.lfs_root_path}}">
<span class="help">{{.i18n.Tr "install.lfs_path_helper"}}</span>
</div>
<div class="inline required field {{if .Err_RunUser}}error{{end}}">
<label for="run_user">{{.i18n.Tr "install.run_user"}}</label>
<input id="run_user" name="run_user" value="{{.run_user}}" required>

@ -66,7 +66,7 @@
<span class="del" data-line="{{.Deletion}}">- {{.Deletion}}</span>
{{end}}
</div>
<span class="file">{{if $file.IsRenamed}}{{$file.OldName}} &rarr; {{end}}{{$file.Name}}</span>
<span class="file">{{if $file.IsRenamed}}{{$file.OldName}} &rarr; {{end}}{{$file.Name}}{{if .IsLFSFile}} ({{$.i18n.Tr "repo.stored_lfs"}}){{end}}</span>
{{if not $file.IsSubmodule}}
<div class="ui right">
{{if $file.IsDeleted}}

@ -5,11 +5,11 @@
{{if .ReadmeInList}}
<strong>{{.FileName}}</strong>
{{else}}
<strong>{{.FileName}}</strong> <span class="text grey normal">{{FileSize .FileSize}}</span>
<strong>{{.FileName}}</strong> <span class="text grey normal">{{FileSize .FileSize}}{{if .IsLFSFile}} ({{.i18n.Tr "repo.stored_lfs"}}){{end}}</span>
{{end}}
{{else}}
<i class="file text outline icon ui left"></i>
<strong>{{.FileName}}</strong> <span class="text grey normal">{{FileSize .FileSize}}</span>
<strong>{{.FileName}}</strong> <span class="text grey normal">{{FileSize .FileSize}}{{if .IsLFSFile}} ({{.i18n.Tr "repo.stored_lfs"}}){{end}}</span>
{{end}}
{{if not .ReadmeInList}}
<div class="ui right file-actions">

@ -0,0 +1,8 @@
Copyright (c) 2012 Dave Grijalva
Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.