Skip to content

Commit

Permalink
Merge #30859
Browse files Browse the repository at this point in the history
30859: server,lint: fix misuse of hash.Hash.Sum  r=bdarnell a=benesch

The hash.Hash interface is poorly designed and easily misused. Several
locations in our codebase were assuming that the Sum function returns
the hash of its input, when in fact it returns the concatenation of its
input with the hash of the empty string. See the comments within for the
full detail.s

This commit:

  1. Corrects the code in our web UI login to avoid leaking session
     secrets into the audit log. Since web UI login has only been
     released in betas, there are no backwards compatibility concerns
     here. Existing web sessions will simply be invalidated when the
     cluster is upgraded and users will need to re-login.

  2. Files a TODO about correcting the code in our password authentication. It
     is not a security risk there as the SHA-256 application was only
     intended to provided a fixed-length input to bcrypt. Fixing this
     code path is more difficult as it requires a migration for existing
     clusters, since this bug has been present since pre-1.0.

  3. Adds a linter to prevent future misuse.

Release note: None

/cc @couchand @vilterp

Co-authored-by: Nikhil Benesch <[email protected]>
  • Loading branch information
craig[bot] and benesch committed Oct 2, 2018
2 parents 64c9286 + 2a127fc commit ca30eaf
Show file tree
Hide file tree
Showing 9 changed files with 642 additions and 385 deletions.
10 changes: 10 additions & 0 deletions pkg/security/password.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,12 +42,22 @@ var ErrEmptyPassword = errors.New("empty passwords are not permitted")
// error.
func CompareHashAndPassword(hashedPassword []byte, password string) error {
h := sha256.New()
// TODO(benesch): properly apply SHA-256 to the password. The current code
// erroneously appends the SHA-256 of the empty hash to the unhashed password
// instead of actually hashing the password. Fixing this requires a somewhat
// complicated backwards compatibility dance. This is not a security issue
// because the round of SHA-256 was only intended to achieve a fixed-length
// input to bcrypt; it is bcrypt that provides the cryptographic security, and
// bcrypt is correctly applied.
//
//lint:ignore HC1000 backwards compatibility
return bcrypt.CompareHashAndPassword(hashedPassword, h.Sum([]byte(password)))
}

// HashPassword takes a raw password and returns a bcrypt hashed password.
func HashPassword(password string) ([]byte, error) {
h := sha256.New()
//lint:ignore HC1000 backwards compatibility (see CompareHashAndPassword)
return bcrypt.GenerateFromPassword(h.Sum([]byte(password)), BcryptCost)
}

Expand Down
6 changes: 4 additions & 2 deletions pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,8 @@ WHERE id = $1`
}

hasher := sha256.New()
hashedCookieSecret := hasher.Sum(cookie.Secret)
_, _ = hasher.Write(cookie.Secret)
hashedCookieSecret := hasher.Sum(nil)
if !bytes.Equal(hashedSecret, hashedCookieSecret) {
return false, "", nil
}
Expand Down Expand Up @@ -286,7 +287,8 @@ func (s *authenticationServer) newAuthSession(
}

hasher := sha256.New()
hashedSecret := hasher.Sum(secret)
_, _ = hasher.Write(secret)
hashedSecret := hasher.Sum(nil)
expiration := s.server.clock.PhysicalTime().Add(webSessionTimeout.Get(&s.server.st.SV))

insertSessionStmt := `
Expand Down
6 changes: 4 additions & 2 deletions pkg/server/authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -292,7 +292,8 @@ WHERE id = $1`

// Verify hashed secret matches original secret
hasher := sha256.New()
hashedSecret := hasher.Sum(origSecret)
_, _ = hasher.Write(origSecret)
hashedSecret := hasher.Sum(nil)
if !bytes.Equal(sessHashedSecret, hashedSecret) {
t.Fatalf("hashed value of secret: \n%#v\ncomputed as: \n%#v\nwanted: \n%#v", origSecret, hashedSecret, sessHashedSecret)
}
Expand Down Expand Up @@ -492,7 +493,8 @@ func TestAuthenticationAPIUserLogin(t *testing.T) {
}

hasher := sha256.New()
hashedSecret := hasher.Sum(sessionCookie.Secret)
_, _ = hasher.Write(sessionCookie.Secret)
hashedSecret := hasher.Sum(nil)
if a, e := sessHashedSecret, hashedSecret; !bytes.Equal(a, e) {
t.Fatalf(
"session secret hash was %v, wanted %v (derived from original secret %v)",
Expand Down
72 changes: 72 additions & 0 deletions pkg/testutils/lint/checker_float.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
// Copyright 2016 The Cockroach Authors.
//
// Licensed 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.

// +build lint

package lint

import (
"go/ast"
"go/types"

"honnef.co/go/tools/lint"
)

// @ianlancetaylor via golang-nuts[0]:
//
// For the record, the spec says, in https://golang.org/ref/spec#Conversions:
// "In all non-constant conversions involving floating-point or complex
// values, if the result type cannot represent the value the conversion
// succeeds but the result value is implementation-dependent." That is the
// case that applies here: you are converting a negative floating point number
// to uint64, which can not represent a negative value, so the result is
// implementation-dependent. The conversion to int64 works, of course. And
// the conversion to int64 and then to uint64 succeeds in converting to int64,
// and when converting to uint64 follows a different rule: "When converting
// between integer types, if the value is a signed integer, it is sign
// extended to implicit infinite precision; otherwise it is zero extended. It
// is then truncated to fit in the result type's size."
//
// So, basically, don't convert a negative floating point number to an
// unsigned integer type.
//
// [0] https://groups.google.com/d/msg/golang-nuts/LH2AO1GAIZE/PyygYRwLAwAJ
//
// TODO(tamird): upstream this.
func checkConvertFloatToUnsigned(j *lint.Job) {
forAllFiles(j, func(n ast.Node) bool {
call, ok := n.(*ast.CallExpr)
if !ok {
return true
}
castType, ok := j.Program.Info.TypeOf(call.Fun).(*types.Basic)
if !ok {
return true
}
if castType.Info()&types.IsUnsigned == 0 {
return true
}
for _, arg := range call.Args {
argType, ok := j.Program.Info.TypeOf(arg).(*types.Basic)
if !ok {
continue
}
if argType.Info()&types.IsFloat == 0 {
continue
}
j.Errorf(arg, "do not convert a floating point number to an unsigned integer type")
}
return true
})
}
147 changes: 147 additions & 0 deletions pkg/testutils/lint/checker_hash.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
// Copyright 2016 The Cockroach Authors.
//
// Licensed 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.

// +build lint

package lint

import (
"go/ast"
"go/types"
"log"

"honnef.co/go/tools/lint"
)

// hashChecker assures that the hash.Hash interface is not misused. A common
// mistake is to assume that the Sum function returns the hash of its input,
// like so:
//
// hashedBytes := sha256.New().Sum(inputBytes)
//
// In fact, the parameter to Sum is not the bytes to be hashed, but a slice that
// will be used as output in case the caller wants to avoid an allocation. In
// the example above, hashedBytes is not the SHA-256 hash of inputBytes, but a
// the concatenation of inputBytes with the hash of the empty string.
//
// Correct uses of the hash.Hash interface are as follows:
//
// h := sha256.New()
// h.Write(inputBytes)
// hashedBytes := h.Sum(nil)
//
// h := sha256.New()
// h.Write(inputBytes)
// var hashedBytes [sha256.Size]byte
// h.Sum(hashedBytes[:0])
//
// To differentiate between correct and incorrect usages, hashChecker applies a
// simple heuristic: it flags calls to Sum where a) the parameter is non-nil and
// b) the return value is used.
//
// The hash.Hash interface may be remedied in Go 2. See golang/go#21070.
type hashChecker struct {
hashType *types.Interface
}

func (*hashChecker) Name() string {
return "hashcheck"
}

func (*hashChecker) Prefix() string {
return "HC"
}

func (c *hashChecker) Init(program *lint.Program) {
hashPkg := program.Prog.Package("hash")
if hashPkg == nil {
log.Fatal("hash package not found")
}
hashIface := hashPkg.Pkg.Scope().Lookup("Hash")
if hashIface == nil {
log.Fatal("hash.Hash type not found")
}
c.hashType = hashIface.Type().Underlying().(*types.Interface)
}

func (c *hashChecker) Funcs() map[string]lint.Func {
return map[string]lint.Func{
"HC1000": c.checkHashWritten,
}
}

func (c *hashChecker) checkHashWritten(j *lint.Job) {
stack := make([]ast.Node, 0, 32)
forAllFiles(j, func(n ast.Node) bool {
if n == nil {
stack = stack[:len(stack)-1] // pop
return true
}
stack = append(stack, n) // push

// Find a call to hash.Hash.Sum.
selExpr, ok := n.(*ast.SelectorExpr)
if !ok {
return true
}
if selExpr.Sel.Name != "Sum" {
return true
}
if !types.Implements(j.Program.Info.TypeOf(selExpr.X), c.hashType) {
return true
}
callExpr, ok := stack[len(stack)-2].(*ast.CallExpr)
if !ok {
return true
}
if len(callExpr.Args) != 1 {
return true
}
// We have a valid call to hash.Hash.Sum.

// Is the argument nil?
var nilArg bool
if id, ok := callExpr.Args[0].(*ast.Ident); ok && id.Name == "nil" {
nilArg = true
}

// Is the return value unused?
var retUnused bool
Switch:
switch t := stack[len(stack)-3].(type) {
case *ast.AssignStmt:
for i := range t.Rhs {
if t.Rhs[i] == stack[len(stack)-2] {
if id, ok := t.Lhs[i].(*ast.Ident); ok && id.Name == "_" {
// Assigning to the blank identifier does not count as using the
// return value.
retUnused = true
}
break Switch
}
}
panic("unreachable")
case *ast.ExprStmt:
// An expression statement means the return value is unused.
retUnused = true
default:
}

if !nilArg && !retUnused {
j.Errorf(callExpr, "probable misuse of hash.Hash.Sum: "+
"provide parameter or use return value, but not both")
}
return true
})
}
75 changes: 75 additions & 0 deletions pkg/testutils/lint/checker_misc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
// Copyright 2016 The Cockroach Authors.
//
// Licensed 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.

// +build lint

package lint

import (
"go/ast"

"honnef.co/go/tools/lint"
)

type miscChecker struct{}

func (*miscChecker) Name() string {
return "misccheck"
}

func (*miscChecker) Prefix() string {
return "CR"
}

func (*miscChecker) Init(*lint.Program) {}

func (*miscChecker) Funcs() map[string]lint.Func {
return map[string]lint.Func{
"FloatToUnsigned": checkConvertFloatToUnsigned,
"Unconvert": checkUnconvert,
}
}

func forAllFiles(j *lint.Job, fn func(node ast.Node) bool) {
for _, f := range j.Program.Files {
if !lint.IsGenerated(f) {
ast.Inspect(f, fn)
}
}
}

func walkForStmts(n ast.Node, fn func(ast.Stmt) bool) bool {
fr, ok := n.(*ast.ForStmt)
if !ok {
return true
}
return walkStmts(fr.Body.List, fn)
}

func walkSelectStmts(n ast.Node, fn func(ast.Stmt) bool) bool {
sel, ok := n.(*ast.SelectStmt)
if !ok {
return true
}
return walkStmts(sel.Body.List, fn)
}

func walkStmts(stmts []ast.Stmt, fn func(ast.Stmt) bool) bool {
for _, stmt := range stmts {
if !fn(stmt) {
return false
}
}
return true
}
Loading

0 comments on commit ca30eaf

Please sign in to comment.