Moves out node pool manager behind an extension using runner pool abstraction (Part 2) (#862)

* Move out node-pool manager and replace it with RunnerPool extension

* adds extension points for runner pools in load-balanced mode

* adds error to return values in RunnerPool and Runner interfaces

* Implements runner pool contract with context-aware shutdown

* fixes issue with range

* fixes tests to use runner abstraction

* adds empty test file as a workaround for build requiring go source files in top-level package

* removes flappy timeout test

* update docs to reflect runner pool setup

* refactors system tests to use runner abstraction

* removes poolmanager

* moves runner interfaces from models to api/runnerpool package

* Adds a second runner to pool docs example

* explicitly check for request spillover to second runner in test

* moves runner pool package name for system tests

* renames runner pool pointer variable for consistency

* pass model json to runner

* automatically cast to http.ResponseWriter in load-balanced call case

* allow overriding of server RunnerPool via a programmatic ServerOption

* fixes return type of ResponseWriter in test

* move Placer interface to runnerpool package

* moves hash-based placer out of open source project

* removes siphash from Gopkg.lock
This commit is contained in:
Gerardo Viedma
2018-03-16 13:46:21 +00:00
committed by GitHub
parent 1a390dc067
commit 73ae77614c
45 changed files with 877 additions and 5870 deletions

8
Gopkg.lock generated
View File

@@ -104,12 +104,6 @@
revision = "8ab6407b697782a06568d4b7f1db25550ec2e4c6"
version = "v0.2.0"
[[projects]]
name = "github.com/dchest/siphash"
packages = ["."]
revision = "4ebf1de738443ea7f45f02dc394c4df1942a126d"
version = "v1.1.0"
[[projects]]
name = "github.com/docker/distribution"
packages = [
@@ -659,6 +653,6 @@
[solve-meta]
analyzer-name = "dep"
analyzer-version = 1
inputs-digest = "98644a27c50160c1daa8110197722573a586ea897d713e19525deccafffa44b3"
inputs-digest = "d8661bd78eda741b449a3675c52207af1bd738c63cb4f74d018edbaf0e9ef77d"
solver-name = "gps-cdcl"
solver-version = 1

View File

@@ -274,6 +274,22 @@ type call struct {
disablePreemptiveCapacityCheck bool
}
func (c *call) SlotDeadline() time.Time {
return c.slotDeadline
}
func (c *call) Request() *http.Request {
return c.req
}
func (c *call) ResponseWriter() http.ResponseWriter {
return c.w.(http.ResponseWriter)
}
func (c *call) StdErr() io.ReadWriteCloser {
return c.stderr
}
func (c *call) Model() *models.Call { return c.Call }
func (c *call) Start(ctx context.Context) error {

View File

@@ -2,9 +2,6 @@ package agent
import (
"context"
"errors"
"io"
"net/http"
"sync"
"time"
@@ -13,55 +10,18 @@ import (
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models"
pool "github.com/fnproject/fn/api/runnerpool"
"github.com/fnproject/fn/fnext"
"github.com/fnproject/fn/poolmanager"
)
// RequestReader takes an agent.Call and return a ReadCloser for the request body inside it
func RequestReader(c *Call) (io.ReadCloser, error) {
// Get the call :(((((
cc, ok := (*c).(*call)
if !ok {
return nil, errors.New("Can't cast agent.Call to agent.call")
}
if cc.req == nil {
return nil, errors.New("Call doesn't contain a request")
}
return cc.req.Body, nil
}
func ResponseWriter(c *Call) (*http.ResponseWriter, error) {
cc, ok := (*c).(*call)
if !ok {
return nil, errors.New("Can't cast agent.Call to agent.call")
}
if rw, ok := cc.w.(http.ResponseWriter); ok {
return &rw, nil
}
return nil, errors.New("Unable to get HTTP response writer from the call")
}
type remoteSlot struct {
lbAgent *lbAgent
}
func (s *remoteSlot) exec(ctx context.Context, call *call) error {
func (s *remoteSlot) exec(ctx context.Context, call pool.RunnerCall) error {
a := s.lbAgent
memMb := call.Model().Memory
lbGroupID := GetGroupID(call.Model())
capacityRequest := &poolmanager.CapacityRequest{TotalMemoryMb: memMb, LBGroupID: lbGroupID}
a.np.AssignCapacity(capacityRequest)
defer a.np.ReleaseCapacity(capacityRequest)
err := a.placer.PlaceCall(a.np, ctx, call, lbGroupID)
err := a.placer.PlaceCall(a.rp, ctx, call)
if err != nil {
logrus.WithError(err).Error("Failed to place call")
}
@@ -76,14 +36,10 @@ func (s *remoteSlot) Error() error {
return nil
}
type Placer interface {
PlaceCall(np NodePool, ctx context.Context, call *call, lbGroupID string) error
}
type naivePlacer struct {
}
func NewNaivePlacer() Placer {
func NewNaivePlacer() pool.Placer {
return &naivePlacer{}
}
@@ -94,8 +50,8 @@ func minDuration(f, s time.Duration) time.Duration {
return s
}
func (sp *naivePlacer) PlaceCall(np NodePool, ctx context.Context, call *call, lbGroupID string) error {
timeout := time.After(call.slotDeadline.Sub(time.Now()))
func (sp *naivePlacer) PlaceCall(rp pool.RunnerPool, ctx context.Context, call pool.RunnerCall) error {
timeout := time.After(call.SlotDeadline().Sub(time.Now()))
for {
select {
@@ -104,19 +60,23 @@ func (sp *naivePlacer) PlaceCall(np NodePool, ctx context.Context, call *call, l
case <-timeout:
return models.ErrCallTimeoutServerBusy
default:
for _, r := range np.Runners(lbGroupID) {
placed, err := r.TryExec(ctx, call)
if err != nil {
logrus.WithError(err).Error("Failed during call placement")
}
if placed {
return err
runners, err := rp.Runners(call)
if err != nil {
logrus.WithError(err).Error("Failed to find runners for call")
} else {
for _, r := range runners {
placed, err := r.TryExec(ctx, call)
if placed {
return err
}
}
}
remaining := call.slotDeadline.Sub(time.Now())
remaining := call.SlotDeadline().Sub(time.Now())
if remaining <= 0 {
return models.ErrCallTimeoutServerBusy
}
// backoff
time.Sleep(minDuration(retryWaitInterval, remaining))
}
}
@@ -129,22 +89,23 @@ const (
// sleep time when scaling from 0 to 1 runners
noCapacityWaitInterval = 1 * time.Second
// amount of time to wait to place a request on a runner
placementTimeout = 15 * time.Second
placementTimeout = 15 * time.Second
runnerPoolShutdownTimeout = 5 * time.Second
)
type lbAgent struct {
delegatedAgent Agent
np NodePool
placer Placer
rp pool.RunnerPool
placer pool.Placer
wg sync.WaitGroup // Needs a good name
shutdown chan struct{}
}
func NewLBAgent(agent Agent, np NodePool, p Placer) (Agent, error) {
func NewLBAgent(agent Agent, rp pool.RunnerPool, p pool.Placer) (Agent, error) {
a := &lbAgent{
delegatedAgent: agent,
np: np,
rp: rp,
placer: p,
}
return a, nil
@@ -158,7 +119,11 @@ func (a *lbAgent) GetCall(opts ...CallOpt) (Call, error) {
}
func (a *lbAgent) Close() error {
a.np.Shutdown()
// we should really be passing the server's context here
ctx, cancel := context.WithTimeout(context.Background(), runnerPoolShutdownTimeout)
defer cancel()
a.rp.Shutdown(ctx)
err := a.delegatedAgent.Close()
if err != nil {
return err

225
api/agent/lb_agent_test.go Normal file
View File

@@ -0,0 +1,225 @@
package agent
import (
"context"
"errors"
"fmt"
"io"
"net/http"
"sync"
"testing"
"time"
"github.com/fnproject/fn/api/models"
pool "github.com/fnproject/fn/api/runnerpool"
)
type mockRunner struct {
wg sync.WaitGroup
sleep time.Duration
mtx sync.Mutex
maxCalls int32 // Max concurrent calls
curCalls int32 // Current calls
procCalls int32 // Processed calls
addr string
}
type mockRunnerPool struct {
runners []pool.Runner
generator insecureRunnerFactory
pki *pkiData
}
func newMockRunnerPool(rf insecureRunnerFactory, runnerAddrs []string) *mockRunnerPool {
var runners []pool.Runner
for _, addr := range runnerAddrs {
r, err := rf(addr)
if err != nil {
continue
}
runners = append(runners, r)
}
return &mockRunnerPool{
runners: runners,
generator: rf,
pki: &pkiData{},
}
}
func (npm *mockRunnerPool) Runners(call pool.RunnerCall) ([]pool.Runner, error) {
return npm.runners, nil
}
func (npm *mockRunnerPool) Shutdown(context.Context) error {
return nil
}
func NewMockRunnerFactory(sleep time.Duration, maxCalls int32) insecureRunnerFactory {
return func(addr string) (pool.Runner, error) {
return &mockRunner{
sleep: sleep,
maxCalls: maxCalls,
addr: addr,
}, nil
}
}
func FaultyRunnerFactory() insecureRunnerFactory {
return func(addr string) (pool.Runner, error) {
return &mockRunner{
addr: addr,
}, errors.New("Creation of new runner failed")
}
}
func (r *mockRunner) checkAndIncrCalls() error {
r.mtx.Lock()
defer r.mtx.Unlock()
if r.curCalls >= r.maxCalls {
return models.ErrCallTimeoutServerBusy //TODO is that the correct error?
}
r.curCalls++
return nil
}
func (r *mockRunner) decrCalls() {
r.mtx.Lock()
defer r.mtx.Unlock()
r.curCalls--
}
func (r *mockRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, error) {
err := r.checkAndIncrCalls()
if err != nil {
return false, err
}
defer r.decrCalls()
r.wg.Add(1)
defer r.wg.Done()
time.Sleep(r.sleep)
r.procCalls++
return true, nil
}
func (r *mockRunner) Close(ctx context.Context) error {
go func() {
r.wg.Wait()
}()
return nil
}
func (r *mockRunner) Address() string {
return r.addr
}
type mockRunnerCall struct {
slotDeadline time.Time
r *http.Request
rw http.ResponseWriter
stdErr io.ReadWriteCloser
model *models.Call
}
func (c *mockRunnerCall) SlotDeadline() time.Time {
return c.slotDeadline
}
func (c *mockRunnerCall) Request() *http.Request {
return c.r
}
func (c *mockRunnerCall) ResponseWriter() http.ResponseWriter {
return c.rw
}
func (c *mockRunnerCall) StdErr() io.ReadWriteCloser {
return c.stdErr
}
func (c *mockRunnerCall) Model() *models.Call {
return c.model
}
func setupMockRunnerPool(expectedRunners []string, execSleep time.Duration, maxCalls int32) *mockRunnerPool {
rf := NewMockRunnerFactory(execSleep, maxCalls)
return newMockRunnerPool(rf, expectedRunners)
}
func TestOneRunner(t *testing.T) {
placer := NewNaivePlacer()
rp := setupMockRunnerPool([]string{"171.19.0.1"}, 10*time.Millisecond, 5)
call := &mockRunnerCall{slotDeadline: time.Now().Add(1 * time.Second)}
err := placer.PlaceCall(rp, context.Background(), call)
if err != nil {
t.Fatalf("Failed to place call on runner %v", err)
}
}
func TestEnforceTimeoutFromContext(t *testing.T) {
placer := NewNaivePlacer()
rp := setupMockRunnerPool([]string{"171.19.0.1"}, 10*time.Millisecond, 5)
call := &mockRunnerCall{slotDeadline: time.Now().Add(1 * time.Second)}
ctx, cancel := context.WithDeadline(context.Background(), time.Now())
defer cancel()
err := placer.PlaceCall(rp, ctx, call)
if err == nil {
t.Fatal("Call should have timed out")
}
}
func TestSpilloverToSecondRunner(t *testing.T) {
placer := NewNaivePlacer()
rp := setupMockRunnerPool([]string{"171.19.0.1", "171.19.0.2"}, 10*time.Millisecond, 2)
parallelCalls := 3
var wg sync.WaitGroup
failures := make(chan error, parallelCalls)
for i := 0; i < parallelCalls; i++ {
wg.Add(1)
go func(i int) {
defer wg.Done()
call := &mockRunnerCall{slotDeadline: time.Now().Add(10 * time.Millisecond)}
err := placer.PlaceCall(rp, context.Background(), call)
if err != nil {
failures <- fmt.Errorf("Timed out call %d", i)
}
}(i)
}
wg.Wait()
close(failures)
err := <-failures
if err != nil || rp.runners[1].(*mockRunner).procCalls != 1 {
t.Fatal("Expected spillover to second runner")
}
}
func TestEnforceSlotTimeout(t *testing.T) {
placer := NewNaivePlacer()
rp := setupMockRunnerPool([]string{"171.19.0.1", "171.19.0.2"}, 10*time.Millisecond, 2)
parallelCalls := 5
var wg sync.WaitGroup
failures := make(chan error, parallelCalls)
for i := 0; i < parallelCalls; i++ {
wg.Add(1)
go func(i int) {
defer wg.Done()
call := &mockRunnerCall{slotDeadline: time.Now().Add(10 * time.Millisecond)}
err := placer.PlaceCall(rp, context.Background(), call)
if err != nil {
failures <- fmt.Errorf("Timed out call %d", i)
}
}(i)
}
wg.Wait()
close(failures)
err := <-failures
if err == nil {
t.Fatal("Expected a call failure")
}
}

View File

@@ -1,74 +0,0 @@
/* The consistent hash ring from the original fnlb.
The behaviour of this depends on changes to the runner list leaving it relatively stable.
*/
package agent
import (
"context"
"time"
"github.com/dchest/siphash"
"github.com/fnproject/fn/api/models"
"github.com/sirupsen/logrus"
)
type chPlacer struct {
}
func NewCHPlacer() Placer {
return &chPlacer{}
}
// This borrows the CH placement algorithm from the original FNLB.
// Because we ask a runner to accept load (queuing on the LB rather than on the nodes), we don't use
// the LB_WAIT to drive placement decisions: runners only accept work if they have the capacity for it.
func (p *chPlacer) PlaceCall(np NodePool, ctx context.Context, call *call, lbGroupID string) error {
// The key is just the path in this case
key := call.Path
sum64 := siphash.Hash(0, 0x4c617279426f6174, []byte(key))
timeout := time.After(call.slotDeadline.Sub(time.Now()))
for {
select {
case <-ctx.Done():
return models.ErrCallTimeoutServerBusy
case <-timeout:
return models.ErrCallTimeoutServerBusy
default:
runners := np.Runners(lbGroupID)
i := int(jumpConsistentHash(sum64, int32(len(runners))))
for j := 0; j < len(runners); j++ {
r := runners[i]
placed, err := r.TryExec(ctx, call)
if err != nil {
logrus.WithError(err).Error("Failed during call placement")
}
if placed {
return err
}
i = (i + 1) % len(runners)
}
remaining := call.slotDeadline.Sub(time.Now())
if remaining <= 0 {
return models.ErrCallTimeoutServerBusy
}
time.Sleep(minDuration(retryWaitInterval, remaining))
}
}
}
// A Fast, Minimal Memory, Consistent Hash Algorithm:
// https://arxiv.org/ftp/arxiv/papers/1406/1406.2294.pdf
func jumpConsistentHash(key uint64, num_buckets int32) int32 {
var b, j int64 = -1, 0
for j < int64(num_buckets) {
b = j
key = key*2862933555777941757 + 1
j = (b + 1) * int64((1<<31)/(key>>33)+1)
}
return int32(b)
}

View File

@@ -1,22 +0,0 @@
package agent
import (
"context"
"github.com/fnproject/fn/poolmanager"
)
// NodePool provides information about pools of runners and receives capacity demands
type NodePool interface {
Runners(lbgID string) []Runner
AssignCapacity(r *poolmanager.CapacityRequest)
ReleaseCapacity(r *poolmanager.CapacityRequest)
Shutdown()
}
// Runner is the interface to invoke the execution of a function call on a specific runner
type Runner interface {
TryExec(ctx context.Context, call Call) (bool, error)
Close()
Address() string
}

View File

@@ -1,415 +0,0 @@
package grpc
import (
"context"
"encoding/json"
"io"
"sync"
"sync/atomic"
"time"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"github.com/fnproject/fn/api/agent"
pb "github.com/fnproject/fn/api/agent/grpc"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/grpcutil"
"github.com/fnproject/fn/poolmanager"
"github.com/sirupsen/logrus"
)
const (
// CapacityUpdatePeriod defines how often the capacity updates are sent
CapacityUpdatePeriod = 1 * time.Second
)
type gRPCNodePool struct {
npm poolmanager.NodePoolManager
advertiser poolmanager.CapacityAdvertiser
mx sync.RWMutex
lbg map[string]*lbg // {lbgid -> *lbg}
generator secureRunnerFactory
//TODO find a better place for this
pki *pkiData
shutdown chan struct{}
}
// TODO need to go in a better place
type pkiData struct {
ca string
key string
cert string
}
type lbg struct {
mx sync.RWMutex
id string
runners map[string]agent.Runner
r_list atomic.Value // We attempt to maintain the same order of runners as advertised by the NPM.
// This is to preserve as reasonable behaviour as possible for the CH algorithm
generator secureRunnerFactory
}
type nullRunner struct{}
func (n *nullRunner) TryExec(ctx context.Context, call agent.Call) (bool, error) {
return false, nil
}
func (n *nullRunner) Close() {}
func (n *nullRunner) Address() string {
return ""
}
var nullRunnerSingleton = new(nullRunner)
type gRPCRunner struct {
// Need a WaitGroup of TryExec in flight
wg sync.WaitGroup
address string
conn *grpc.ClientConn
client pb.RunnerProtocolClient
}
// allow factory to be overridden in tests
type secureRunnerFactory func(addr string, cert string, key string, ca string) (agent.Runner, error)
func secureGRPCRunnerFactory(addr string, cert string, key string, ca string) (agent.Runner, error) {
p := &pkiData{
cert: cert,
key: key,
ca: ca,
}
conn, client, err := runnerConnection(addr, p)
if err != nil {
return nil, err
}
return &gRPCRunner{
address: addr,
conn: conn,
client: client,
}, nil
}
func DefaultgRPCNodePool(npmAddress string, cert string, key string, ca string) agent.NodePool {
npm := poolmanager.NewNodePoolManager(npmAddress, cert, key, ca)
// TODO do we need to persistent this ID in order to survive restart?
lbID := id.New().String()
advertiser := poolmanager.NewCapacityAdvertiser(npm, lbID, CapacityUpdatePeriod)
return newgRPCNodePool(cert, key, ca, npm, advertiser, secureGRPCRunnerFactory)
}
func newgRPCNodePool(cert string, key string, ca string, npm poolmanager.NodePoolManager, advertiser poolmanager.CapacityAdvertiser, rf secureRunnerFactory) agent.NodePool {
logrus.Info("Starting dynamic runner pool")
p := &pkiData{
ca: ca,
cert: cert,
key: key,
}
np := &gRPCNodePool{
npm: npm,
advertiser: advertiser,
lbg: make(map[string]*lbg),
generator: rf,
shutdown: make(chan struct{}),
pki: p,
}
go np.maintenance()
return np
}
func (np *gRPCNodePool) Runners(lbgID string) []agent.Runner {
np.mx.RLock()
lbg, ok := np.lbg[lbgID]
np.mx.RUnlock()
if !ok {
np.mx.Lock()
lbg, ok = np.lbg[lbgID]
if !ok {
lbg = newLBG(lbgID, np.generator)
np.lbg[lbgID] = lbg
}
np.mx.Unlock()
}
return lbg.runnerList()
}
func (np *gRPCNodePool) Shutdown() {
np.advertiser.Shutdown()
np.npm.Shutdown()
}
func (np *gRPCNodePool) AssignCapacity(r *poolmanager.CapacityRequest) {
np.advertiser.AssignCapacity(r)
}
func (np *gRPCNodePool) ReleaseCapacity(r *poolmanager.CapacityRequest) {
np.advertiser.ReleaseCapacity(r)
}
func (np *gRPCNodePool) maintenance() {
ticker := time.NewTicker(500 * time.Millisecond)
for {
select {
case <-np.shutdown:
return
case <-ticker.C:
// Reload any LBGroup information from NPM (pull for the moment, shift to listening to a stream later)
np.reloadLBGmembership()
}
}
}
func newLBG(lbgID string, generator secureRunnerFactory) *lbg {
lbg := &lbg{
id: lbgID,
runners: make(map[string]agent.Runner),
r_list: atomic.Value{},
generator: generator,
}
lbg.r_list.Store([]agent.Runner{})
return lbg
}
func (np *gRPCNodePool) reloadLBGmembership() {
np.mx.RLock()
lbgroups := np.lbg
np.mx.RUnlock()
for lbgID, lbg := range lbgroups {
lbg.reloadMembers(lbgID, np.npm, np.pki)
}
}
func (lbg *lbg) runnerList() []agent.Runner {
orig_runners := lbg.r_list.Load().([]agent.Runner)
// XXX: Return a copy. If we required this to be immutably read by the caller, we could return the structure directly
runners := make([]agent.Runner, len(orig_runners))
copy(runners, orig_runners)
return runners
}
func (lbg *lbg) reloadMembers(lbgID string, npm poolmanager.NodePoolManager, p *pkiData) {
runners, err := npm.GetRunners(lbgID)
if err != nil {
logrus.Debug("Failed to get the list of runners from node pool manager")
}
lbg.mx.Lock()
defer lbg.mx.Unlock()
r_list := make([]agent.Runner, len(runners))
seen := map[string]bool{} // If we've seen a particular runner or not
var errGenerator error
for i, addr := range runners {
r, ok := lbg.runners[addr]
if !ok {
logrus.WithField("runner_addr", addr).Debug("New Runner to be added")
r, errGenerator = lbg.generator(addr, p.cert, p.key, p.ca)
if errGenerator != nil {
logrus.WithField("runner_addr", addr).Debug("Creation of the new runner failed")
} else {
lbg.runners[addr] = r
}
}
if errGenerator == nil {
r_list[i] = r // Maintain the delivered order
} else {
// some algorithms (like consistent hash) work better if the i'th element
// of r_list points to the same node on all LBs, so insert a placeholder
// if we can't create the runner for some reason"
r_list[i] = nullRunnerSingleton
}
seen[addr] = true
}
lbg.r_list.Store(r_list)
// Remove any runners that we have not encountered
for addr, r := range lbg.runners {
if _, ok := seen[addr]; !ok {
logrus.WithField("runner_addr", addr).Debug("Removing drained runner")
delete(lbg.runners, addr)
r.Close()
}
}
}
func (r *gRPCRunner) Close() {
go func() {
r.wg.Wait()
r.conn.Close()
}()
}
func runnerConnection(address string, pki *pkiData) (*grpc.ClientConn, pb.RunnerProtocolClient, error) {
ctx := context.Background()
var creds credentials.TransportCredentials
if pki != nil {
var err error
creds, err = grpcutil.CreateCredentials(pki.cert, pki.key, pki.ca)
if err != nil {
logrus.WithError(err).Error("Unable to create credentials to connect to runner node")
return nil, nil, err
}
}
// we want to set a very short timeout to fail-fast if something goes wrong
conn, err := grpcutil.DialWithBackoff(ctx, address, creds, 100*time.Millisecond, grpc.DefaultBackoffConfig)
if err != nil {
logrus.WithError(err).Error("Unable to connect to runner node")
}
protocolClient := pb.NewRunnerProtocolClient(conn)
logrus.WithField("runner_addr", address).Info("Connected to runner")
return conn, protocolClient, nil
}
func (r *gRPCRunner) Address() string {
return r.address
}
func (r *gRPCRunner) TryExec(ctx context.Context, call agent.Call) (bool, error) {
logrus.WithField("runner_addr", r.address).Debug("Attempting to place call")
r.wg.Add(1)
defer r.wg.Done()
// Get app and route information
// Construct model.Call with CONFIG in it already
modelJSON, err := json.Marshal(call.Model())
if err != nil {
logrus.WithError(err).Error("Failed to encode model as JSON")
// If we can't encode the model, no runner will ever be able to run this. Give up.
return true, err
}
runnerConnection, err := r.client.Engage(ctx)
if err != nil {
logrus.WithError(err).Error("Unable to create client to runner node")
// Try on next runner
return false, err
}
err = runnerConnection.Send(&pb.ClientMsg{Body: &pb.ClientMsg_Try{Try: &pb.TryCall{ModelsCallJson: string(modelJSON)}}})
if err != nil {
logrus.WithError(err).Error("Failed to send message to runner node")
return false, err
}
msg, err := runnerConnection.Recv()
if err != nil {
logrus.WithError(err).Error("Failed to receive first message from runner node")
return false, err
}
switch body := msg.Body.(type) {
case *pb.RunnerMsg_Acknowledged:
if !body.Acknowledged.Committed {
logrus.Debugf("Runner didn't commit invocation request: %v", body.Acknowledged.Details)
return false, nil
// Try the next runner
}
logrus.Debug("Runner committed invocation request, sending data frames")
done := make(chan error)
go receiveFromRunner(runnerConnection, call, done)
sendToRunner(call, runnerConnection)
return true, <-done
default:
logrus.Errorf("Unhandled message type received from runner: %v\n", msg)
return true, nil
}
}
func sendToRunner(call agent.Call, protocolClient pb.RunnerProtocol_EngageClient) error {
bodyReader, err := agent.RequestReader(&call)
if err != nil {
logrus.WithError(err).Error("Unable to get reader for request body")
return err
}
writeBufferSize := 10 * 1024 // 10KB
writeBuffer := make([]byte, writeBufferSize)
for {
n, err := bodyReader.Read(writeBuffer)
logrus.Debugf("Wrote %v bytes to the runner", n)
if err == io.EOF {
err = protocolClient.Send(&pb.ClientMsg{
Body: &pb.ClientMsg_Data{
Data: &pb.DataFrame{
Data: writeBuffer,
Eof: true,
},
},
})
if err != nil {
logrus.WithError(err).Error("Failed to send data frame with EOF to runner")
}
break
}
err = protocolClient.Send(&pb.ClientMsg{
Body: &pb.ClientMsg_Data{
Data: &pb.DataFrame{
Data: writeBuffer,
Eof: false,
},
},
})
if err != nil {
logrus.WithError(err).Error("Failed to send data frame")
return err
}
}
return nil
}
func receiveFromRunner(protocolClient pb.RunnerProtocol_EngageClient, call agent.Call, done chan error) {
w, err := agent.ResponseWriter(&call)
if err != nil {
logrus.WithError(err).Error("Unable to get response writer from call")
done <- err
return
}
for {
msg, err := protocolClient.Recv()
if err != nil {
logrus.WithError(err).Error("Failed to receive message from runner")
done <- err
return
}
switch body := msg.Body.(type) {
case *pb.RunnerMsg_ResultStart:
switch meta := body.ResultStart.Meta.(type) {
case *pb.CallResultStart_Http:
for _, header := range meta.Http.Headers {
(*w).Header().Set(header.Key, header.Value)
}
default:
logrus.Errorf("Unhandled meta type in start message: %v", meta)
}
case *pb.RunnerMsg_Data:
(*w).Write(body.Data.Data)
case *pb.RunnerMsg_Finished:
if body.Finished.Success {
logrus.Infof("Call finished successfully: %v", body.Finished.Details)
} else {
logrus.Infof("Call finish unsuccessfully:: %v", body.Finished.Details)
}
close(done)
return
default:
logrus.Errorf("Unhandled message type from runner: %v", body)
}
}
}

View File

@@ -1,214 +0,0 @@
package grpc
import (
"context"
"errors"
"strconv"
"sync"
"testing"
"time"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/poolmanager"
model "github.com/fnproject/fn/poolmanager/grpc"
)
type mockRunner struct {
wg sync.WaitGroup
sleep time.Duration
mtx sync.Mutex
maxCalls int32 // Max concurrent calls
curCalls int32 // Current calls
addr string
}
type mockNodePoolManager struct {
runners []string
}
type mockgRPCNodePool struct {
npm poolmanager.NodePoolManager
lbg map[string]*lbg
generator secureRunnerFactory
pki *pkiData
}
func newMockgRPCNodePool(rf secureRunnerFactory, runners []string) *mockgRPCNodePool {
npm := &mockNodePoolManager{runners: runners}
return &mockgRPCNodePool{
npm: npm,
lbg: make(map[string]*lbg),
generator: rf,
pki: &pkiData{},
}
}
func (npm *mockNodePoolManager) AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error {
return nil
}
func (npm *mockNodePoolManager) GetRunners(lbgID string) ([]string, error) {
return npm.runners, nil
}
func (npm *mockNodePoolManager) Shutdown() error {
return nil
}
func NewMockRunnerFactory(sleep time.Duration, maxCalls int32) secureRunnerFactory {
return func(addr string, cert string, key string, ca string) (agent.Runner, error) {
return &mockRunner{
sleep: sleep,
maxCalls: maxCalls,
addr: addr,
}, nil
}
}
func FaultyRunnerFactory() secureRunnerFactory {
return func(addr string, cert string, key string, ca string) (agent.Runner, error) {
return &mockRunner{
addr: addr,
}, errors.New("Creation of new runner failed")
}
}
func (r *mockRunner) checkAndIncrCalls() error {
r.mtx.Lock()
defer r.mtx.Unlock()
if r.curCalls >= r.maxCalls {
return models.ErrCallTimeoutServerBusy //TODO is that the correct error?
}
r.curCalls++
return nil
}
func (r *mockRunner) decrCalls() {
r.mtx.Lock()
defer r.mtx.Unlock()
r.curCalls--
}
func (r *mockRunner) TryExec(ctx context.Context, call agent.Call) (bool, error) {
err := r.checkAndIncrCalls()
if err != nil {
return false, err
}
defer r.decrCalls()
r.wg.Add(1)
defer r.wg.Done()
time.Sleep(r.sleep)
w, err := agent.ResponseWriter(&call)
if err != nil {
return true, err
}
buf := []byte("OK")
(*w).Header().Set("Content-Type", "text/plain")
(*w).Header().Set("Content-Length", strconv.Itoa(len(buf)))
(*w).Write(buf)
return true, nil
}
func (r *mockRunner) Close() {
go func() {
r.wg.Wait()
}()
}
func (r *mockRunner) Address() string {
return ""
}
func setupMockNodePool(lbgID string, expectedRunners []string) (*mockgRPCNodePool, *lbg) {
rf := NewMockRunnerFactory(1*time.Millisecond, 1)
lb := newLBG(lbgID, rf)
np := newMockgRPCNodePool(rf, expectedRunners)
np.lbg[lbgID] = lb
return np, lb
}
func checkRunners(t *testing.T, expectedRunners []string, actualRunners map[string]agent.Runner, ordList []agent.Runner) {
if len(expectedRunners) != len(actualRunners) {
t.Errorf("List of runners is wrong, expected: %d got: %d", len(expectedRunners), len(actualRunners))
}
for i, r := range expectedRunners {
_, ok := actualRunners[r]
if !ok {
t.Errorf("Advertised runner %s not found in the list of runners", r)
}
ordR := ordList[i].(*mockRunner)
if ordR.addr != r {
t.Error("Ordered list is not in sync with the advertised list")
}
}
}
func TestReloadMembersNoRunners(t *testing.T) {
lbgID := "lb-test"
// // Empty list, no runners available
np, lb := setupMockNodePool(lbgID, make([]string, 0))
np.lbg[lbgID].reloadMembers(lbgID, np.npm, np.pki)
expectedRunners := []string{}
ordList := lb.r_list.Load().([]agent.Runner)
checkRunners(t, expectedRunners, lb.runners, ordList)
}
func TestReloadMembersNewRunners(t *testing.T) {
lbgID := "lb-test"
expectedRunners := []string{"171.16.0.1", "171.16.0.2"}
np, lb := setupMockNodePool(lbgID, expectedRunners)
np.lbg[lbgID].reloadMembers(lbgID, np.npm, np.pki)
ordList := lb.r_list.Load().([]agent.Runner)
checkRunners(t, expectedRunners, lb.runners, ordList)
}
func TestReloadMembersRemoveRunners(t *testing.T) {
lbgID := "lb-test"
expectedRunners := []string{"171.16.0.1", "171.16.0.3"}
np, lb := setupMockNodePool(lbgID, expectedRunners)
// actual runners before the update
actualRunners := []string{"171.16.0.1", "171.16.0.2", "171.16.0.19"}
for _, v := range actualRunners {
r, err := lb.generator(v, np.pki.cert, np.pki.key, np.pki.ca)
if err != nil {
t.Error("Failed to create new runner")
}
lb.runners[v] = r
}
if len(lb.runners) != len(actualRunners) {
t.Errorf("Failed to load list of runners")
}
np.lbg[lbgID].reloadMembers(lbgID, np.npm, np.pki)
ordList := lb.r_list.Load().([]agent.Runner)
checkRunners(t, expectedRunners, lb.runners, ordList)
}
func TestReloadMembersFailToCreateNewRunners(t *testing.T) {
lbgID := "lb-test"
rf := FaultyRunnerFactory()
lb := newLBG(lbgID, rf)
np := newMockgRPCNodePool(rf, []string{"171.19.0.1"})
np.lbg[lbgID] = lb
np.lbg[lbgID].reloadMembers(lbgID, np.npm, np.pki)
actualRunners := lb.runners
if len(actualRunners) != 0 {
t.Errorf("List of runners should be empty")
}
ordList := lb.r_list.Load().([]agent.Runner)
if ordList[0] != nullRunnerSingleton {
t.Errorf("Ordered list should have a nullRunner")
}
}

View File

@@ -1,105 +0,0 @@
package grpc
import (
"sync"
"github.com/sirupsen/logrus"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/poolmanager"
)
// allow factory to be overridden in tests
type insecureRunnerFactory func(addr string) (agent.Runner, error)
func insecureGRPCRunnerFactory(addr string) (agent.Runner, error) {
conn, client, err := runnerConnection(addr, nil)
if err != nil {
return nil, err
}
return &gRPCRunner{
address: addr,
conn: conn,
client: client,
}, nil
}
// manages a single set of runners ignoring lb groups
type staticNodePool struct {
generator insecureRunnerFactory
rMtx *sync.RWMutex
runners []agent.Runner
}
// NewStaticNodePool returns a NodePool consisting of a static set of runners
func DefaultStaticNodePool(runnerAddresses []string) agent.NodePool {
return newStaticNodePool(runnerAddresses, insecureGRPCRunnerFactory)
}
// NewStaticNodePool returns a NodePool consisting of a static set of runners
func newStaticNodePool(runnerAddresses []string, runnerFactory insecureRunnerFactory) agent.NodePool {
logrus.WithField("runners", runnerAddresses).Info("Starting static runner pool")
var runners []agent.Runner
for _, addr := range runnerAddresses {
r, err := runnerFactory(addr)
if err != nil {
logrus.WithField("runner_addr", addr).Warn("Invalid runner")
continue
}
logrus.WithField("runner_addr", addr).Debug("Adding runner to pool")
runners = append(runners, r)
}
return &staticNodePool{
rMtx: &sync.RWMutex{},
runners: runners,
generator: runnerFactory,
}
}
func (np *staticNodePool) Runners(lbgID string) []agent.Runner {
np.rMtx.RLock()
defer np.rMtx.RUnlock()
r := make([]agent.Runner, len(np.runners))
copy(r, np.runners)
return r
}
func (np *staticNodePool) AddRunner(address string) error {
np.rMtx.Lock()
defer np.rMtx.Unlock()
r, err := np.generator(address)
if err != nil {
logrus.WithField("runner_addr", address).Warn("Failed to add runner")
return err
}
np.runners = append(np.runners, r)
return nil
}
func (np *staticNodePool) RemoveRunner(address string) {
np.rMtx.Lock()
defer np.rMtx.Unlock()
for i, r := range np.runners {
if r.Address() == address {
// delete runner from list
np.runners = append(np.runners[:i], np.runners[i+1:]...)
return
}
}
}
func (np *staticNodePool) AssignCapacity(r *poolmanager.CapacityRequest) {
// NO-OP
}
func (np *staticNodePool) ReleaseCapacity(r *poolmanager.CapacityRequest) {
// NO-OP
}
func (np *staticNodePool) Shutdown() {
// NO-OP
}

View File

@@ -1,79 +0,0 @@
package grpc
import (
"context"
"testing"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/poolmanager"
)
func setupStaticPool(runners []string) agent.NodePool {
return newStaticNodePool(runners, mockRunnerFactory)
}
type mockStaticRunner struct {
address string
}
func (r *mockStaticRunner) TryExec(ctx context.Context, call agent.Call) (bool, error) {
return true, nil
}
func (r *mockStaticRunner) Close() {
}
func (r *mockStaticRunner) Address() string {
return r.address
}
func mockRunnerFactory(addr string) (agent.Runner, error) {
return &mockStaticRunner{address: addr}, nil
}
func TestNewStaticPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs)
if len(np.Runners("foo")) != len(addrs) {
t.Fatalf("Invalid number of runners %v", len(np.Runners("foo")))
}
}
func TestCapacityForStaticPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs)
cr := &poolmanager.CapacityRequest{TotalMemoryMb: 100, LBGroupID: "foo"}
np.AssignCapacity(cr)
np.ReleaseCapacity(cr)
if len(np.Runners("foo")) != len(addrs) {
t.Fatalf("Invalid number of runners %v", len(np.Runners("foo")))
}
}
func TestAddNodeToPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs).(*staticNodePool)
np.AddRunner("127.0.0.1:8082")
if len(np.Runners("foo")) != len(addrs)+1 {
t.Fatalf("Invalid number of runners %v", len(np.Runners("foo")))
}
}
func TestRemoveNodeFromPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs).(*staticNodePool)
np.RemoveRunner("127.0.0.1:8081")
if len(np.Runners("foo")) != len(addrs)-1 {
t.Fatalf("Invalid number of runners %v", len(np.Runners("foo")))
}
np.RemoveRunner("127.0.0.1:8081")
if len(np.Runners("foo")) != len(addrs)-1 {
t.Fatalf("Invalid number of runners %v", len(np.Runners("foo")))
}
}

240
api/agent/runner_client.go Normal file
View File

@@ -0,0 +1,240 @@
package agent
import (
"context"
"encoding/json"
"io"
"sync"
"time"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
pb "github.com/fnproject/fn/api/agent/grpc"
pool "github.com/fnproject/fn/api/runnerpool"
"github.com/fnproject/fn/grpcutil"
"github.com/sirupsen/logrus"
)
type pkiData struct {
ca string
key string
cert string
}
type nullRunner struct{}
func (n *nullRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, error) {
return false, nil
}
func (n *nullRunner) Close() error {
return nil
}
func (n *nullRunner) Address() string {
return ""
}
var nullRunnerSingleton = new(nullRunner)
type gRPCRunner struct {
// Need a WaitGroup of TryExec in flight
wg sync.WaitGroup
address string
conn *grpc.ClientConn
client pb.RunnerProtocolClient
}
// allow factory to be overridden in tests
type secureRunnerFactory func(addr string, cert string, key string, ca string) (pool.Runner, error)
func secureGRPCRunnerFactory(addr string, cert string, key string, ca string) (pool.Runner, error) {
p := &pkiData{
cert: cert,
key: key,
ca: ca,
}
conn, client, err := runnerConnection(addr, p)
if err != nil {
return nil, err
}
return &gRPCRunner{
address: addr,
conn: conn,
client: client,
}, nil
}
// Close waits until the context is closed for all inflight requests
// to complete prior to terminating the underlying grpc connection
func (r *gRPCRunner) Close(ctx context.Context) error {
err := make(chan error)
go func() {
defer close(err)
r.wg.Wait()
err <- r.conn.Close()
}()
select {
case e := <-err:
return e
case <-ctx.Done():
return ctx.Err() // context timed out while waiting
}
}
func runnerConnection(address string, pki *pkiData) (*grpc.ClientConn, pb.RunnerProtocolClient, error) {
ctx := context.Background()
var creds credentials.TransportCredentials
if pki != nil {
var err error
creds, err = grpcutil.CreateCredentials(pki.cert, pki.key, pki.ca)
if err != nil {
logrus.WithError(err).Error("Unable to create credentials to connect to runner node")
return nil, nil, err
}
}
// we want to set a very short timeout to fail-fast if something goes wrong
conn, err := grpcutil.DialWithBackoff(ctx, address, creds, 100*time.Millisecond, grpc.DefaultBackoffConfig)
if err != nil {
logrus.WithError(err).Error("Unable to connect to runner node")
}
protocolClient := pb.NewRunnerProtocolClient(conn)
logrus.WithField("runner_addr", address).Info("Connected to runner")
return conn, protocolClient, nil
}
func (r *gRPCRunner) Address() string {
return r.address
}
func (r *gRPCRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, error) {
logrus.WithField("runner_addr", r.address).Debug("Attempting to place call")
r.wg.Add(1)
defer r.wg.Done()
// extract the call's model data to pass on to the pure runner
modelJSON, err := json.Marshal(call.Model())
if err != nil {
logrus.WithError(err).Error("Failed to encode model as JSON")
// If we can't encode the model, no runner will ever be able to run this. Give up.
return true, err
}
runnerConnection, err := r.client.Engage(ctx)
if err != nil {
logrus.WithError(err).Error("Unable to create client to runner node")
// Try on next runner
return false, err
}
err = runnerConnection.Send(&pb.ClientMsg{Body: &pb.ClientMsg_Try{Try: &pb.TryCall{ModelsCallJson: string(modelJSON)}}})
if err != nil {
logrus.WithError(err).Error("Failed to send message to runner node")
return false, err
}
msg, err := runnerConnection.Recv()
if err != nil {
logrus.WithError(err).Error("Failed to receive first message from runner node")
return false, err
}
switch body := msg.Body.(type) {
case *pb.RunnerMsg_Acknowledged:
if !body.Acknowledged.Committed {
logrus.Debugf("Runner didn't commit invocation request: %v", body.Acknowledged.Details)
return false, nil
// Try the next runner
}
logrus.Debug("Runner committed invocation request, sending data frames")
done := make(chan error)
go receiveFromRunner(runnerConnection, call, done)
sendToRunner(call, runnerConnection)
return true, <-done
default:
logrus.Errorf("Unhandled message type received from runner: %v\n", msg)
return true, nil
}
}
func sendToRunner(call pool.RunnerCall, protocolClient pb.RunnerProtocol_EngageClient) error {
bodyReader := call.Request().Body
writeBufferSize := 10 * 1024 // 10KB
writeBuffer := make([]byte, writeBufferSize)
for {
n, err := bodyReader.Read(writeBuffer)
logrus.Debugf("Wrote %v bytes to the runner", n)
if err == io.EOF {
err = protocolClient.Send(&pb.ClientMsg{
Body: &pb.ClientMsg_Data{
Data: &pb.DataFrame{
Data: writeBuffer,
Eof: true,
},
},
})
if err != nil {
logrus.WithError(err).Error("Failed to send data frame with EOF to runner")
}
break
}
err = protocolClient.Send(&pb.ClientMsg{
Body: &pb.ClientMsg_Data{
Data: &pb.DataFrame{
Data: writeBuffer,
Eof: false,
},
},
})
if err != nil {
logrus.WithError(err).Error("Failed to send data frame")
return err
}
}
return nil
}
func receiveFromRunner(protocolClient pb.RunnerProtocol_EngageClient, c pool.RunnerCall, done chan error) {
w := c.ResponseWriter()
for {
msg, err := protocolClient.Recv()
if err != nil {
logrus.WithError(err).Error("Failed to receive message from runner")
done <- err
return
}
switch body := msg.Body.(type) {
case *pb.RunnerMsg_ResultStart:
switch meta := body.ResultStart.Meta.(type) {
case *pb.CallResultStart_Http:
for _, header := range meta.Http.Headers {
w.Header().Set(header.Key, header.Value)
}
default:
logrus.Errorf("Unhandled meta type in start message: %v", meta)
}
case *pb.RunnerMsg_Data:
w.Write(body.Data.Data)
case *pb.RunnerMsg_Finished:
if body.Finished.Success {
logrus.Infof("Call finished successfully: %v", body.Finished.Details)
} else {
logrus.Infof("Call finish unsuccessfully:: %v", body.Finished.Details)
}
close(done)
return
default:
logrus.Errorf("Unhandled message type from runner: %v", body)
}
}
}

146
api/agent/static_pool.go Normal file
View File

@@ -0,0 +1,146 @@
package agent
import (
"context"
"sync"
"time"
pool "github.com/fnproject/fn/api/runnerpool"
"github.com/sirupsen/logrus"
)
const (
staticPoolShutdownTimeout = 5 * time.Second
)
// allow factory to be overridden in tests
type insecureRunnerFactory func(addr string) (pool.Runner, error)
func insecureGRPCRunnerFactory(addr string) (pool.Runner, error) {
conn, client, err := runnerConnection(addr, nil)
if err != nil {
return nil, err
}
return &gRPCRunner{
address: addr,
conn: conn,
client: client,
}, nil
}
// manages a single set of runners ignoring lb groups
type staticRunnerPool struct {
generator insecureRunnerFactory
rMtx *sync.RWMutex
runners []pool.Runner
}
// DefaultStaticRunnerPool returns a RunnerPool consisting of a static set of runners
func DefaultStaticRunnerPool(runnerAddresses []string) pool.RunnerPool {
return newStaticRunnerPool(runnerAddresses, insecureGRPCRunnerFactory)
}
func newStaticRunnerPool(runnerAddresses []string, runnerFactory insecureRunnerFactory) pool.RunnerPool {
logrus.WithField("runners", runnerAddresses).Info("Starting static runner pool")
var runners []pool.Runner
for _, addr := range runnerAddresses {
r, err := runnerFactory(addr)
if err != nil {
logrus.WithField("runner_addr", addr).Warn("Invalid runner")
continue
}
logrus.WithField("runner_addr", addr).Debug("Adding runner to pool")
runners = append(runners, r)
}
return &staticRunnerPool{
rMtx: &sync.RWMutex{},
runners: runners,
generator: runnerFactory,
}
}
func (rp *staticRunnerPool) Runners(call pool.RunnerCall) ([]pool.Runner, error) {
rp.rMtx.RLock()
defer rp.rMtx.RUnlock()
r := make([]pool.Runner, len(rp.runners))
copy(r, rp.runners)
return r, nil
}
func (rp *staticRunnerPool) AddRunner(address string) error {
rp.rMtx.Lock()
defer rp.rMtx.Unlock()
r, err := rp.generator(address)
if err != nil {
logrus.WithField("runner_addr", address).Warn("Failed to add runner")
return err
}
rp.runners = append(rp.runners, r)
return nil
}
func (rp *staticRunnerPool) RemoveRunner(address string) {
rp.rMtx.Lock()
defer rp.rMtx.Unlock()
ctx, cancel := context.WithTimeout(context.Background(), staticPoolShutdownTimeout)
defer cancel()
for i, r := range rp.runners {
if r.Address() == address {
err := r.Close(ctx)
if err != nil {
logrus.WithError(err).WithField("runner_addr", r.Address()).Error("Failed to close runner")
}
// delete runner from list
rp.runners = append(rp.runners[:i], rp.runners[i+1:]...)
return
}
}
}
// Shutdown blocks waiting for all runners to close, or until ctx is done
func (rp *staticRunnerPool) Shutdown(ctx context.Context) (e error) {
rp.rMtx.Lock()
defer rp.rMtx.Unlock()
ctx, cancel := context.WithTimeout(context.Background(), staticPoolShutdownTimeout)
defer cancel()
errors := make(chan error, len(rp.runners))
var wg sync.WaitGroup
for _, r := range rp.runners {
wg.Add(1)
go func(runner pool.Runner) {
defer wg.Done()
err := runner.Close(ctx)
if err != nil {
logrus.WithError(err).WithField("runner_addr", runner.Address()).Error("Failed to close runner")
errors <- err
}
}(r)
}
done := make(chan interface{})
go func() {
defer close(done)
wg.Wait()
}()
select {
case <-done:
close(errors)
for e := range errors {
// return the first error
if e != nil {
return e
}
}
return nil
case <-ctx.Done():
return ctx.Err() // context timed out while waiting
}
}

View File

@@ -0,0 +1,84 @@
package agent
import (
"context"
"testing"
pool "github.com/fnproject/fn/api/runnerpool"
)
func setupStaticPool(runners []string) pool.RunnerPool {
return newStaticRunnerPool(runners, mockRunnerFactory)
}
type mockStaticRunner struct {
address string
}
func (r *mockStaticRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, error) {
return true, nil
}
func (r *mockStaticRunner) Close(ctx context.Context) error {
return nil
}
func (r *mockStaticRunner) Address() string {
return r.address
}
func mockRunnerFactory(addr string) (pool.Runner, error) {
return &mockStaticRunner{address: addr}, nil
}
func TestNewStaticPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs)
runners, err := np.Runners(nil)
if err != nil {
t.Fatalf("Failed to list runners %v", err)
}
if len(runners) != len(addrs) {
t.Fatalf("Invalid number of runners %v", len(runners))
}
}
func TestAddNodeToPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs).(*staticRunnerPool)
np.AddRunner("127.0.0.1:8082")
runners, err := np.Runners(nil)
if err != nil {
t.Fatalf("Failed to list runners %v", err)
}
if len(runners) != 3 {
t.Fatalf("Invalid number of runners %v", len(runners))
}
}
func TestRemoveNodeFromPool(t *testing.T) {
addrs := []string{"127.0.0.1:8080", "127.0.0.1:8081"}
np := setupStaticPool(addrs).(*staticRunnerPool)
np.RemoveRunner("127.0.0.1:8081")
runners, err := np.Runners(nil)
if err != nil {
t.Fatalf("Failed to list runners %v", err)
}
if len(runners) != 1 {
t.Fatalf("Invalid number of runners %v", len(runners))
}
np.RemoveRunner("127.0.0.1:8081")
runners, err = np.Runners(nil)
if err != nil {
t.Fatalf("Failed to list runners %v", err)
}
if len(runners) != 1 {
t.Fatalf("Invalid number of runners %v", len(runners))
}
}

View File

@@ -1,22 +1,30 @@
package models
package runnerpool
import (
"context"
"io"
"net/http"
"time"
"github.com/fnproject/fn/api/models"
)
// Placer implements a placement strategy for calls that are load-balanced
// across runners in a pool
type Placer interface {
PlaceCall(rp RunnerPool, ctx context.Context, call RunnerCall) error
}
// RunnerPool is the abstraction for getting an ordered list of runners to try for a call
type RunnerPool interface {
Runners(call RunnerCall) ([]Runner, error)
Shutdown() error
Shutdown(context.Context) error
}
// Runner is the interface to invoke the execution of a function call on a specific runner
type Runner interface {
TryExec(ctx context.Context, call RunnerCall) (bool, error)
Close() error
Close(ctx context.Context) error
Address() string
}
@@ -25,7 +33,7 @@ type Runner interface {
type RunnerCall interface {
SlotDeadline() time.Time
Request() *http.Request
ResponseWriter() io.Writer
ResponseWriter() http.ResponseWriter
StdErr() io.ReadWriteCloser
Model() *Call
Model() *models.Call
}

View File

@@ -21,13 +21,13 @@ import (
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/agent/hybrid"
agent_grpc "github.com/fnproject/fn/api/agent/nodepool/grpc"
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/datastore"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/api/logs"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/mqs"
pool "github.com/fnproject/fn/api/runnerpool"
"github.com/fnproject/fn/api/version"
"github.com/fnproject/fn/fnext"
"github.com/gin-gonic/gin"
@@ -50,9 +50,7 @@ const (
EnvDBURL = "FN_DB_URL"
EnvLOGDBURL = "FN_LOGSTORE_URL"
EnvRunnerURL = "FN_RUNNER_API_URL"
EnvNPMAddress = "FN_NPM_ADDRESS"
EnvRunnerAddresses = "FN_RUNNER_ADDRESSES"
EnvLBPlacementAlg = "FN_PLACER"
EnvNodeType = "FN_NODE_TYPE"
EnvPort = "FN_PORT" // be careful, Gin expects this variable to be "port"
EnvGRPCPort = "FN_GRPC_PORT"
@@ -116,7 +114,6 @@ type Server struct {
rootMiddlewares []fnext.Middleware
apiMiddlewares []fnext.Middleware
promExporter *prometheus.Exporter
runnerPool models.RunnerPool
// Extensions can append to this list of contexts so that cancellations are properly handled.
extraCtxs []context.Context
}
@@ -351,6 +348,18 @@ func WithAgent(agent agent.Agent) ServerOption {
}
}
func (s *Server) defaultRunnerPool() (pool.RunnerPool, error) {
runnerAddresses := getEnv(EnvRunnerAddresses, "")
if runnerAddresses == "" {
return nil, errors.New("Must provide FN_RUNNER_ADDRESSES when running in default load-balanced mode!")
}
return agent.DefaultStaticRunnerPool(strings.Split(runnerAddresses, ",")), nil
}
func (s *Server) defaultPlacer() pool.Placer {
return agent.NewNaivePlacer()
}
func WithLogstoreFromDatastore() ServerOption {
return func(ctx context.Context, s *Server) error {
if s.datastore == nil {
@@ -434,28 +443,13 @@ func WithAgentFromEnv() ServerOption {
}
delegatedAgent := agent.New(agent.NewCachedDataAccess(cl))
npmAddress := getEnv(EnvNPMAddress, "")
runnerAddresses := getEnv(EnvRunnerAddresses, "")
var nodePool agent.NodePool
if npmAddress != "" {
// TODO refactor DefaultgRPCNodePool as an extension
nodePool = agent_grpc.DefaultgRPCNodePool(npmAddress, s.cert, s.certKey, s.certAuthority)
} else if runnerAddresses != "" {
nodePool = agent_grpc.DefaultStaticNodePool(strings.Split(runnerAddresses, ","))
} else {
return errors.New("Must provide either FN_NPM_ADDRESS or FN_RUNNER_ADDRESSES for an Fn NuLB node.")
runnerPool, err := s.defaultRunnerPool()
if err != nil {
return err
}
placer := s.defaultPlacer()
// Select the placement algorithm
var placer agent.Placer
switch getEnv(EnvLBPlacementAlg, "") {
case "ch":
placer = agent.NewCHPlacer()
default:
placer = agent.NewNaivePlacer()
}
s.agent, err = agent.NewLBAgent(delegatedAgent, nodePool, placer)
s.agent, err = agent.NewLBAgent(delegatedAgent, runnerPool, placer)
if err != nil {
return errors.New("LBAgent creation failed")
}
@@ -892,12 +886,6 @@ func (s *Server) Datastore() models.Datastore {
return s.datastore
}
// WithRunnerPool provides an extension point for overriding
// the default runner pool implementation when running in load-balanced mode
func (s *Server) WithRunnerPool(runnerPool models.RunnerPool) {
s.runnerPool = runnerPool
}
// returns the unescaped ?cursor and ?perPage values
// pageParams clamps 0 < ?perPage <= 100 and defaults to 30 if 0
// ignores parsing errors and falls back to defaults.

View File

@@ -1,252 +0,0 @@
# Running fn in Multitenant Compute Mode
## Motivation
By running Fn in multitenant mode, you can define independent pools of compute resources available to functions in the platform. By associating a function with a particular _load balancing group_, its invocations are guaranteed to execute on the compute resources assigned to that specific group. The pluggable _node pool manager_ abstraction provides a mechanism to scale compute resources dynamically, based on capacity requirements advertised by the load-balancing layer. Together with load balancer groups, it allows you to implement independent capacity and scaling policies for different sets of users or tenants.
## Create certificates
This is a useful article to read for quickly generating mutual TLS certs:
http://www.levigross.com/2015/11/21/mutual-tls-authentication-in-go/
tl;dr: Get this https://github.com/levigross/go-mutual-tls/blob/master/generate\_client\_cert.go
add IP `127.0.0.1` to the cert by adding the line
```golang
template.IPAddresses = append(template.IPAddresses, net.ParseIP("127.0.0.1"))
```
somewhere around line 124,
add the "net" package to the list of import packages and run it with
```bash
go run generate_client_cert.go --email-address a@a.com
```
Tada! Certs.
## Starting the components (as regular processes)
### API server
```bash
FN_NODE_TYPE=api ./fnserver
```
### Runner
```bash
mkdir /tmp/runnerdata
FN_NODE_TYPE=pure-runner FN_PORT=8082 FN_NODE_CERT=cert.pem FN_NODE_CERT_AUTHORITY=cert.pem FN_NODE_CERT_KEY=key.pem ./fnserver
```
### LB
```bash
mkdir /tmp/lbdata
FN_NODE_TYPE=lb FN_PORT=8081 FN_RUNNER_API_URL=http://localhost:8080 FN_NODE_CERT=cert.pem FN_NODE_CERT_AUTHORITY=cert.pem FN_NPM_ADDRESS=localhost:8083 FN_NODE_CERT_KEY=key.pem FN_LOG_LEVEL=DEBUG ./fnserver
```
### Node Pool Manager (NPM)
Currently the NPM uses a fixed, single-node instance of the Runner to simulate its "pool". The runner answers on port 8082 in this example,
but the GRPC port is 9190.
Grap the runner address and put in as value for the `FN_RUNNER_ADDRESSES` env variable.
```bash
go build -buildmode=plugin -o noop.so poolmanager/server/controlplane/plugin/noop.go
go build -o fnnpm poolmanager/server/main.go
FN_LOG_LEVEL=DEBUG \
FN_NODE_CERT=cert.pem \
FN_NODE_CERT_KEY=key.pem \
FN_NODE_CERT_AUTHORITY=cert.pem \
FN_PORT=8083 \
FN_RUNNER_ADDRESSES=<RUNNER_ADDRESS_HERE>:9190 \
CONTROL_PLANE_SO=noop.so \
./fnnpm
```
### Directing a request to a specific LB Group
Until a generic metadata mechanism is available in fn, an application or route can be [configured][docs/developers/configs.md] so that incoming requests are forwarded to runners in the specified LB group. In the absence of this configuration, requests will map to the _default_ LB group.
For example, to set an app's LB group:
```bash
fn apps config set myapp FN_LB_GROUP_ID my-app-pool
```
Note that the value of _FN\_LB\_GROUP\_ID_ above will then be visible to the function as an environment variable.
## Starting the components (in Docker containers)
### Build the images
The images don't yet exist in a registry, so they need building first.
```bash
docker build -f images/fnnpm/Dockerfile -t fnproject/fnnpm:latest .
docker build -f images/lb/Dockerfile -t fnproject/lb:latest .
docker build -f images/api/Dockerfile -t fnproject/api:latest .
docker build -f images/runner/Dockerfile -t fnproject/runner:latest .
```
### Start the containers
### API
This *shouldn't* need to talk to the Docker daemon, but it still tries to *for now*. So mount the socket.
```
docker run -d \
--name api \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8080:8080 \
fnproject/api:latest
```
#### First runner
```bash
docker run -d \
--name runner \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9190:9190 \
-e FN_GRPC_PORT=9190 \
-p 8095:8080 \
-v $(pwd)/cert.pem:/certs/cert.pem \
-v $(pwd)/key.pem:/certs/key.pem \
-e FN_NODE_CERT=/certs/cert.pem \
-e FN_NODE_CERT_KEY=/certs/key.pem \
-e FN_NODE_CERT_AUTHORITY=/certs/cert.pem \
fnproject/runner:latest
```
#### Second runner
```bash
docker run -d \
--name runner-2 \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9191:9191 \
-e FN_GRPC_PORT=9191 \
-p 8096:8080 \
-v $(pwd)/cert.pem:/certs/cert.pem \
-v $(pwd)/key.pem:/certs/key.pem \
-e FN_NODE_CERT=/certs/cert.pem \
-e FN_NODE_CERT_KEY=/certs/key.pem \
-e FN_NODE_CERT_AUTHORITY=/certs/cert.pem \
fnproject/runner:latest
```
### Node Pool Manager (NPM)
Retrieve the IP addresses for the runners:
```bash
export RUNNER1=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner`
export RUNNER2=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner-2`
```
```
docker run -d \
--name fnnpm \
-e FN_RUNNER_ADDRESSES=$RUNNER1:9190,$RUNNER2:9191 \
-p 8083:8080 \
-v $(pwd)/cert.pem:/certs/cert.pem \
-v $(pwd)/key.pem:/certs/key.pem \
-e FN_NODE_CERT=/certs/cert.pem \
-e FN_NODE_CERT_KEY=/certs/key.pem \
-e FN_NODE_CERT_AUTHORITY=/certs/cert.pem \
-e FN_LOG_LEVEL=INFO \
-e FN_PORT=8083 \
fnproject/fnnpm:latest
```
### LB
Again, this *shouldn't* need to talk to the Docker daemon, but it still tries to *for now*. So mount the socket.
Retrieve the IP address for API and NPM:
```bash
export API=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' api`
export NPM=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' fnnpm`
```
```bash
docker run -d \
--name lb \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8081:8080 \
-v $(pwd)/cert.pem:/certs/cert.pem \
-v $(pwd)/key.pem:/certs/key.pem \
-e FN_NODE_TYPE=lb \
-e FN_RUNNER_API_URL=http://$API:8080 \
-e FN_NPM_ADDRESS=$NPM:8083 \
-e FN_NODE_CERT=/certs/cert.pem \
-e FN_NODE_CERT_KEY=/certs/key.pem \
-e FN_NODE_CERT_AUTHORITY=/certs/cert.pem \
fnproject/lb:latest
```
## Running without the Node Pool Manager
This mode assumes that LB is started with a static set of runners in a single global pool. Note that this configuration does not support runner certificates and is that the communication between LB and runners is unencrypted.
### API
```
docker run -d \
--name api \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8080:8080 \
fnproject/api:latest
```
#### First runner
```bash
docker run -d \
--name runner \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9190:9190 \
-e FN_GRPC_PORT=9190 \
-p 8095:8080 \
fnproject/runner:latest
```
#### Second runner
```bash
docker run -d \
--name runner-2 \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9191:9191 \
-e FN_GRPC_PORT=9191 \
-p 8096:8080 \
fnproject/runner:latest
```
### LB
Retrieve the IP addresses for the runners and the API:
```bash
export RUNNER1=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner`
export RUNNER2=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner-2`
export API=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' api`
```
Pass in the static set of runners to _FN\_RUNNER\_ADDRESSES_:
```bash
docker run -d \
--name lb \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8081:8080 \
-e FN_RUNNER_API_URL=http://$API:8080 \
-e FN_RUNNER_ADDRESSES=$RUNNER1:9190,$RUNNER2:9191 \
fnproject/lb:latest
```

View File

@@ -0,0 +1,101 @@
# Running load-balanced fn against a pool of runners
## Motivation
You can run a load-balanced setup for fn to route requests to a group of one or more runners.
## Starting the components (as regular processes)
### API server
```bash
FN_NODE_TYPE=api ./fnserver
```
### Runners
```bash
mkdir /tmp/runnerdata
# first runner
FN_NODE_TYPE=pure-runner FN_PORT=8082 FN_GRPC_PORT=9190 ./fnserver
# on another terminal, start a second runner
FN_NODE_TYPE=pure-runner FN_PORT=8083 FN_GRPC_PORT=9191 ./fnserver
```
### LB
```bash
mkdir /tmp/lbdata
FN_NODE_TYPE=lb FN_PORT=8081 FN_RUNNER_API_URL=http://localhost:8080 FN_RUNNER_ADDRESSES=localhost:9190,localhost:9191 FN_LOG_LEVEL=DEBUG ./fnserver
```
## Starting the components (in Docker containers)
### Build the images
The images don't yet exist in a registry, so they need building first.
```bash
docker build -f images/lb/Dockerfile -t fnproject/lb:latest .
docker build -f images/api/Dockerfile -t fnproject/api:latest .
docker build -f images/runner/Dockerfile -t fnproject/runner:latest .
```
### Start the containers
This mode assumes that LB is started with a static set of runners in a single global pool. Note that this configuration does not support runner certificates and is that the communication between LB and runners is unencrypted.
### API
```
docker run -d \
--name api \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8080:8080 \
fnproject/api:latest
```
#### First runner
```bash
docker run -d \
--name runner \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9190:9190 \
-e FN_GRPC_PORT=9190 \
-p 8095:8080 \
fnproject/runner:latest
```
#### Second runner
```bash
docker run -d \
--name runner-2 \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 9191:9191 \
-e FN_GRPC_PORT=9191 \
-p 8096:8080 \
fnproject/runner:latest
```
### LB
Retrieve the IP addresses for the runners and the API:
```bash
export RUNNER1=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner`
export RUNNER2=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' runner-2`
export API=`docker inspect --format '{{ .NetworkSettings.IPAddress }}' api`
```
Pass in the static set of runners to _FN\_RUNNER\_ADDRESSES_:
```bash
docker run -d \
--name lb \
-v /var/run/docker.sock:/var/run/docker.sock \
-p 8081:8080 \
-e FN_RUNNER_API_URL=http://$API:8080 \
-e FN_RUNNER_ADDRESSES=$RUNNER1:9190,$RUNNER2:9191 \
fnproject/lb:latest
```

View File

@@ -38,9 +38,6 @@ type ExtServer interface {
// AddRouteEndpoint adds an endpoints to /v1/apps/:app/routes/:route/x
AddRouteEndpointFunc(method, path string, handler func(w http.ResponseWriter, r *http.Request, app *models.App, route *models.Route))
// WithRunnerPool overrides the default runner pool implementation when running in load-balanced mode
WithRunnerPool(runnerPool models.RunnerPool)
// Datastore returns the Datastore Fn is using
Datastore() models.Datastore
}

View File

@@ -1,208 +0,0 @@
package poolmanager
import (
"time"
model "github.com/fnproject/fn/poolmanager/grpc"
"github.com/golang/protobuf/ptypes"
"github.com/sirupsen/logrus"
)
const (
updatesBufferSize = 10000
)
type NodePoolManager interface {
AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error
GetRunners(lbgID string) ([]string, error)
Shutdown() error
}
type remoteNodePoolManager struct {
serverAddr string
client remoteClient
}
func NewNodePoolManager(serverAddr string, cert string, key string, ca string) NodePoolManager {
// TODO support reconnects
c, e := newRemoteClient(serverAddr, cert, key, ca)
if e != nil {
logrus.Error("Failed to connect to the node pool manager for sending capacity update")
}
return &remoteNodePoolManager{
serverAddr: serverAddr,
client: c,
}
}
func (npm *remoteNodePoolManager) Shutdown() error {
logrus.Info("Shutting down node pool manager client")
return npm.client.Shutdown()
}
func (npm *remoteNodePoolManager) GetRunners(lbGroupID string) ([]string, error) {
m, err := npm.client.GetLBGroup(&model.LBGroupId{Id: lbGroupID})
if err != nil {
return nil, err
}
logrus.WithField("runners", m.GetRunners()).Debug("Received runner list")
runnerList := make([]string, len(m.GetRunners()))
for i, r := range m.GetRunners() {
runnerList[i] = r.Address
}
return runnerList, nil
}
func (npm *remoteNodePoolManager) AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error {
logrus.Debugf("Advertising new capacity snapshot %+v", snapshots)
return npm.client.AdvertiseCapacity(snapshots)
}
//CapacityAdvertiser allows capacity to be assigned/released and advertised to a node pool manager
type CapacityAdvertiser interface {
AssignCapacity(r *CapacityRequest)
ReleaseCapacity(r *CapacityRequest)
Shutdown() error
}
type capacityEntry struct {
TotalMemoryMb uint64
}
type CapacityRequest struct {
TotalMemoryMb uint64
LBGroupID string
assignment bool
}
// true if this capacity requirement requires no resources
func (e *capacityEntry) isZero() bool {
return e.TotalMemoryMb == 0
}
func (e *capacityEntry) isNegative() bool {
return e.TotalMemoryMb < 0
}
type queueingAdvertiser struct {
updates chan *CapacityRequest
capacity map[string]*capacityEntry
shutdown chan interface{}
npm NodePoolManager
lbID string
}
//NewCapacityAdvertiser return a CapacityAdvertiser
func NewCapacityAdvertiser(npm NodePoolManager, lbID string, period time.Duration) CapacityAdvertiser {
agg := &queueingAdvertiser{
updates: make(chan *CapacityRequest, updatesBufferSize),
capacity: make(map[string]*capacityEntry),
shutdown: make(chan interface{}),
npm: npm,
lbID: lbID,
}
agg.scheduleUpdates(period)
return agg
}
func (a *queueingAdvertiser) scheduleUpdates(period time.Duration) {
ticker := time.NewTicker(period)
go func() {
defer ticker.Stop()
for {
select {
case <-ticker.C:
a.sendAdvertisements()
case update := <-a.updates:
a.mergeUpdate(update)
case <-a.shutdown:
return
}
}
}()
}
func (a *queueingAdvertiser) AssignCapacity(r *CapacityRequest) {
a.udpateCapacity(r, true)
}
func (a *queueingAdvertiser) ReleaseCapacity(r *CapacityRequest) {
a.udpateCapacity(r, false)
}
// don't leak implementation to caller
func (a *queueingAdvertiser) udpateCapacity(r *CapacityRequest, assignment bool) {
if r.LBGroupID == "" {
logrus.Warn("Missing LBG for capacity update!")
return
}
r.assignment = assignment
select {
case a.updates <- r:
// do not block
default:
logrus.Warn("Buffer size exceeded, dropping released capacity update before aggregation")
}
}
func (a *queueingAdvertiser) mergeUpdate(r *CapacityRequest) {
if e, ok := a.capacity[r.LBGroupID]; ok {
if r.assignment {
e.TotalMemoryMb += r.TotalMemoryMb
logrus.WithField("lbg_id", r.LBGroupID).Debugf("Increased assigned capacity to %vMB", e.TotalMemoryMb)
} else {
e.TotalMemoryMb -= r.TotalMemoryMb
logrus.WithField("lbg_id", r.LBGroupID).Debugf("Released assigned capacity to %vMB", e.TotalMemoryMb)
}
if e.isNegative() {
logrus.WithField("lbg_id", r.LBGroupID).Warn("Negative total memory")
}
} else {
if r.assignment {
a.capacity[r.LBGroupID] = &capacityEntry{TotalMemoryMb: r.TotalMemoryMb}
logrus.WithField("lbg_id", r.LBGroupID).Debugf("Assigned new capacity of %vMB", r.TotalMemoryMb)
} else {
logrus.WithField("lbg_id", r.LBGroupID).Warn("Attempted to release unknown assigned capacity!")
}
}
}
func (a *queueingAdvertiser) sendAdvertisements() {
snapshots := []*model.CapacitySnapshot{}
for lbgID, e := range a.capacity {
snapshots = append(snapshots, &model.CapacitySnapshot{
GroupId: &model.LBGroupId{Id: lbgID},
MemMbTotal: e.TotalMemoryMb,
})
// clean entries with zero capacity requirements
// after including them in advertisement
if e.isZero() {
logrus.WithField("lbg_id", lbgID).Debug("Purging nil capacity requirement")
delete(a.capacity, lbgID)
}
}
// don't block consuming updates while calling out to NPM
go func() {
a.npm.AdvertiseCapacity(&model.CapacitySnapshotList{
Snapshots: snapshots,
LbId: a.lbID,
Ts: ptypes.TimestampNow(),
})
}()
}
func (a *queueingAdvertiser) Shutdown() error {
logrus.Info("Shutting down capacity advertiser")
close(a.shutdown)
return nil
}

View File

@@ -1,121 +0,0 @@
package poolmanager
import (
"testing"
"time"
model "github.com/fnproject/fn/poolmanager/grpc"
)
const (
updatePeriod = 10 * time.Millisecond
)
type mockRemoteNodePoolManager struct {
emptyCapacityAdv bool //this is just for testing
capacity map[string]uint64
}
func (n *mockRemoteNodePoolManager) AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error {
for _, v := range snapshots.Snapshots {
n.capacity[v.GetGroupId().GetId()] = v.GetMemMbTotal()
}
if len(snapshots.Snapshots) == 0 {
//No new capacity requested we mark that we have seen it
n.emptyCapacityAdv = true
}
return nil
}
func (n *mockRemoteNodePoolManager) GetRunners(lbgID string) ([]string, error) {
return []string{}, nil
}
func (n *mockRemoteNodePoolManager) Shutdown() error { return nil }
func checkCapacity(t *testing.T, lbGroupID string, p time.Duration, npm NodePoolManager, expected uint64) {
// give time to spawn the scheduleUpdates
time.Sleep(2 * p)
n, ok := npm.(*mockRemoteNodePoolManager)
if !ok {
t.Error("Unable to cast to the mockRemoteNodePoolManager")
}
actual, ok := n.capacity[lbGroupID]
if !ok {
t.Errorf("Unable to find capacity for lbGroupID %s", lbGroupID)
}
if expected != actual {
t.Errorf("Wrong capacity reported, expected: %d got: %d", expected, actual)
}
}
func newQueueingAdvertiser() (CapacityAdvertiser, NodePoolManager) {
npm := &mockRemoteNodePoolManager{capacity: make(map[string]uint64)}
adv := NewCapacityAdvertiser(npm, "lb-test", updatePeriod)
return adv, npm
}
func TestQueueingAdvAddRelease(t *testing.T) {
lbGroupID := "lbg-test1"
qAdv, npm := newQueueingAdvertiser()
defer qAdv.Shutdown()
expected := uint64(128)
e := &CapacityRequest{TotalMemoryMb: expected, LBGroupID: lbGroupID}
qAdv.AssignCapacity(e)
checkCapacity(t, lbGroupID, updatePeriod, npm, expected)
// New Assignment
qAdv.AssignCapacity(e)
expected = expected + e.TotalMemoryMb
checkCapacity(t, lbGroupID, updatePeriod, npm, expected)
// Release capacity
qAdv.ReleaseCapacity(e)
expected = expected - e.TotalMemoryMb
checkCapacity(t, lbGroupID, updatePeriod, npm, expected)
}
func TestQueueingAdvAddReleaseMultiLBGroup(t *testing.T) {
lbGroupID1 := "lbg-test1"
lbGroupID2 := "lbg-test2"
qAdv, npm := newQueueingAdvertiser()
defer qAdv.Shutdown()
expected1 := uint64(128)
e := &CapacityRequest{TotalMemoryMb: expected1, LBGroupID: lbGroupID1}
qAdv.AssignCapacity(e)
checkCapacity(t, lbGroupID1, updatePeriod, npm, expected1)
expected2 := uint64(256)
e = &CapacityRequest{TotalMemoryMb: expected2, LBGroupID: lbGroupID2}
qAdv.AssignCapacity(e)
checkCapacity(t, lbGroupID2, updatePeriod, npm, expected2)
}
func TestQueueingAdvPurgeCapacity(t *testing.T) {
lbGroupID := "lbg-test1"
qAdv, npm := newQueueingAdvertiser()
defer qAdv.Shutdown()
expected := uint64(128)
e := &CapacityRequest{TotalMemoryMb: expected, LBGroupID: lbGroupID}
qAdv.AssignCapacity(e)
checkCapacity(t, lbGroupID, updatePeriod, npm, expected)
// Release capacity
expected = uint64(0)
qAdv.ReleaseCapacity(e)
// we want to assert that we have received an Adv with 0 capacity
checkCapacity(t, lbGroupID, updatePeriod, npm, 0)
// we expect to have advertised an empty CapacitySnapshotList
time.Sleep(2 * updatePeriod)
n, ok := npm.(*mockRemoteNodePoolManager)
if !ok {
t.Error("Unable to cast to the mockRemoteNodePoolManager")
}
if !n.emptyCapacityAdv {
t.Error("Expected to have seen an empty CapacitySnapshot advertised")
}
}

View File

@@ -1,61 +0,0 @@
package poolmanager
import (
"context"
"time"
"github.com/fnproject/fn/grpcutil"
model "github.com/fnproject/fn/poolmanager/grpc"
"github.com/sirupsen/logrus"
"google.golang.org/grpc"
)
type remoteClient interface {
AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error
GetLBGroup(id *model.LBGroupId) (*model.LBGroupMembership, error)
Shutdown() error
}
type grpcPoolManagerClient struct {
scaler model.NodePoolScalerClient
manager model.RunnerManagerClient
conn *grpc.ClientConn
}
func newRemoteClient(serverAddr string, cert string, key string, ca string) (remoteClient, error) {
logrus.WithField("npm_address", serverAddr).Info("Connecting to node pool manager")
ctx := context.Background()
creds, err := grpcutil.CreateCredentials(cert, key, ca)
if err != nil {
logrus.WithError(err).Error("Unable to create credentials to connect to runner node")
return nil, err
}
conn, err := grpcutil.DialWithBackoff(ctx, serverAddr, creds, 300*time.Millisecond, grpc.DefaultBackoffConfig)
if err != nil {
return nil, err
}
return &grpcPoolManagerClient{
scaler: model.NewNodePoolScalerClient(conn),
manager: model.NewRunnerManagerClient(conn),
conn: conn,
}, nil
}
func (c *grpcPoolManagerClient) AdvertiseCapacity(snapshots *model.CapacitySnapshotList) error {
_, err := c.scaler.AdvertiseCapacity(context.Background(), snapshots)
if err != nil {
logrus.WithError(err).Warn("Failed to advertise capacity")
return err
}
return nil
}
func (c *grpcPoolManagerClient) GetLBGroup(id *model.LBGroupId) (*model.LBGroupMembership, error) {
return c.manager.GetLBGroup(context.Background(), id)
}
func (c *grpcPoolManagerClient) Shutdown() error {
return c.conn.Close()
}

View File

@@ -1,334 +0,0 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: poolmanager.proto
/*
Package poolmanager is a generated protocol buffer package.
It is generated from these files:
poolmanager.proto
It has these top-level messages:
Runner
LBGroupId
LBGroupMembership
CapacitySnapshot
CapacitySnapshotList
*/
package poolmanager
import proto "github.com/golang/protobuf/proto"
import fmt "fmt"
import math "math"
import google_protobuf "github.com/golang/protobuf/ptypes/timestamp"
import google_protobuf1 "github.com/golang/protobuf/ptypes/empty"
import (
context "golang.org/x/net/context"
grpc "google.golang.org/grpc"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
// A compilation error at this line likely means your copy of the
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
type Runner struct {
Address string `protobuf:"bytes,1,opt,name=address" json:"address,omitempty"`
ClientKey []byte `protobuf:"bytes,2,opt,name=client_key,json=clientKey,proto3" json:"client_key,omitempty"`
}
func (m *Runner) Reset() { *m = Runner{} }
func (m *Runner) String() string { return proto.CompactTextString(m) }
func (*Runner) ProtoMessage() {}
func (*Runner) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
func (m *Runner) GetAddress() string {
if m != nil {
return m.Address
}
return ""
}
func (m *Runner) GetClientKey() []byte {
if m != nil {
return m.ClientKey
}
return nil
}
type LBGroupId struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
}
func (m *LBGroupId) Reset() { *m = LBGroupId{} }
func (m *LBGroupId) String() string { return proto.CompactTextString(m) }
func (*LBGroupId) ProtoMessage() {}
func (*LBGroupId) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
func (m *LBGroupId) GetId() string {
if m != nil {
return m.Id
}
return ""
}
type LBGroupMembership struct {
GroupId *LBGroupId `protobuf:"bytes,1,opt,name=group_id,json=groupId" json:"group_id,omitempty"`
Runners []*Runner `protobuf:"bytes,2,rep,name=runners" json:"runners,omitempty"`
}
func (m *LBGroupMembership) Reset() { *m = LBGroupMembership{} }
func (m *LBGroupMembership) String() string { return proto.CompactTextString(m) }
func (*LBGroupMembership) ProtoMessage() {}
func (*LBGroupMembership) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
func (m *LBGroupMembership) GetGroupId() *LBGroupId {
if m != nil {
return m.GroupId
}
return nil
}
func (m *LBGroupMembership) GetRunners() []*Runner {
if m != nil {
return m.Runners
}
return nil
}
type CapacitySnapshot struct {
GroupId *LBGroupId `protobuf:"bytes,1,opt,name=group_id,json=groupId" json:"group_id,omitempty"`
MemMbTotal uint64 `protobuf:"varint,2,opt,name=mem_mb_total,json=memMbTotal" json:"mem_mb_total,omitempty"`
}
func (m *CapacitySnapshot) Reset() { *m = CapacitySnapshot{} }
func (m *CapacitySnapshot) String() string { return proto.CompactTextString(m) }
func (*CapacitySnapshot) ProtoMessage() {}
func (*CapacitySnapshot) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
func (m *CapacitySnapshot) GetGroupId() *LBGroupId {
if m != nil {
return m.GroupId
}
return nil
}
func (m *CapacitySnapshot) GetMemMbTotal() uint64 {
if m != nil {
return m.MemMbTotal
}
return 0
}
type CapacitySnapshotList struct {
Ts *google_protobuf.Timestamp `protobuf:"bytes,1,opt,name=ts" json:"ts,omitempty"`
LbId string `protobuf:"bytes,2,opt,name=lb_id,json=lbId" json:"lb_id,omitempty"`
Snapshots []*CapacitySnapshot `protobuf:"bytes,3,rep,name=snapshots" json:"snapshots,omitempty"`
}
func (m *CapacitySnapshotList) Reset() { *m = CapacitySnapshotList{} }
func (m *CapacitySnapshotList) String() string { return proto.CompactTextString(m) }
func (*CapacitySnapshotList) ProtoMessage() {}
func (*CapacitySnapshotList) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
func (m *CapacitySnapshotList) GetTs() *google_protobuf.Timestamp {
if m != nil {
return m.Ts
}
return nil
}
func (m *CapacitySnapshotList) GetLbId() string {
if m != nil {
return m.LbId
}
return ""
}
func (m *CapacitySnapshotList) GetSnapshots() []*CapacitySnapshot {
if m != nil {
return m.Snapshots
}
return nil
}
func init() {
proto.RegisterType((*Runner)(nil), "Runner")
proto.RegisterType((*LBGroupId)(nil), "LBGroupId")
proto.RegisterType((*LBGroupMembership)(nil), "LBGroupMembership")
proto.RegisterType((*CapacitySnapshot)(nil), "CapacitySnapshot")
proto.RegisterType((*CapacitySnapshotList)(nil), "CapacitySnapshotList")
}
// Reference imports to suppress errors if they are not otherwise used.
var _ context.Context
var _ grpc.ClientConn
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
const _ = grpc.SupportPackageIsVersion4
// Client API for NodePoolScaler service
type NodePoolScalerClient interface {
AdvertiseCapacity(ctx context.Context, in *CapacitySnapshotList, opts ...grpc.CallOption) (*google_protobuf1.Empty, error)
}
type nodePoolScalerClient struct {
cc *grpc.ClientConn
}
func NewNodePoolScalerClient(cc *grpc.ClientConn) NodePoolScalerClient {
return &nodePoolScalerClient{cc}
}
func (c *nodePoolScalerClient) AdvertiseCapacity(ctx context.Context, in *CapacitySnapshotList, opts ...grpc.CallOption) (*google_protobuf1.Empty, error) {
out := new(google_protobuf1.Empty)
err := grpc.Invoke(ctx, "/NodePoolScaler/AdvertiseCapacity", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// Server API for NodePoolScaler service
type NodePoolScalerServer interface {
AdvertiseCapacity(context.Context, *CapacitySnapshotList) (*google_protobuf1.Empty, error)
}
func RegisterNodePoolScalerServer(s *grpc.Server, srv NodePoolScalerServer) {
s.RegisterService(&_NodePoolScaler_serviceDesc, srv)
}
func _NodePoolScaler_AdvertiseCapacity_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(CapacitySnapshotList)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(NodePoolScalerServer).AdvertiseCapacity(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/NodePoolScaler/AdvertiseCapacity",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(NodePoolScalerServer).AdvertiseCapacity(ctx, req.(*CapacitySnapshotList))
}
return interceptor(ctx, in, info, handler)
}
var _NodePoolScaler_serviceDesc = grpc.ServiceDesc{
ServiceName: "NodePoolScaler",
HandlerType: (*NodePoolScalerServer)(nil),
Methods: []grpc.MethodDesc{
{
MethodName: "AdvertiseCapacity",
Handler: _NodePoolScaler_AdvertiseCapacity_Handler,
},
},
Streams: []grpc.StreamDesc{},
Metadata: "poolmanager.proto",
}
// Client API for RunnerManager service
type RunnerManagerClient interface {
GetLBGroup(ctx context.Context, in *LBGroupId, opts ...grpc.CallOption) (*LBGroupMembership, error)
}
type runnerManagerClient struct {
cc *grpc.ClientConn
}
func NewRunnerManagerClient(cc *grpc.ClientConn) RunnerManagerClient {
return &runnerManagerClient{cc}
}
func (c *runnerManagerClient) GetLBGroup(ctx context.Context, in *LBGroupId, opts ...grpc.CallOption) (*LBGroupMembership, error) {
out := new(LBGroupMembership)
err := grpc.Invoke(ctx, "/RunnerManager/GetLBGroup", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// Server API for RunnerManager service
type RunnerManagerServer interface {
GetLBGroup(context.Context, *LBGroupId) (*LBGroupMembership, error)
}
func RegisterRunnerManagerServer(s *grpc.Server, srv RunnerManagerServer) {
s.RegisterService(&_RunnerManager_serviceDesc, srv)
}
func _RunnerManager_GetLBGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LBGroupId)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(RunnerManagerServer).GetLBGroup(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/RunnerManager/GetLBGroup",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(RunnerManagerServer).GetLBGroup(ctx, req.(*LBGroupId))
}
return interceptor(ctx, in, info, handler)
}
var _RunnerManager_serviceDesc = grpc.ServiceDesc{
ServiceName: "RunnerManager",
HandlerType: (*RunnerManagerServer)(nil),
Methods: []grpc.MethodDesc{
{
MethodName: "GetLBGroup",
Handler: _RunnerManager_GetLBGroup_Handler,
},
},
Streams: []grpc.StreamDesc{},
Metadata: "poolmanager.proto",
}
func init() { proto.RegisterFile("poolmanager.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
// 396 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0x5f, 0x6b, 0x14, 0x31,
0x14, 0xc5, 0xbb, 0xd3, 0xda, 0x75, 0x6f, 0x6b, 0x71, 0xe3, 0x1f, 0x86, 0x2d, 0xe2, 0x38, 0x20,
0x2c, 0x3e, 0x64, 0x61, 0xfc, 0x00, 0x52, 0xff, 0x50, 0x8a, 0x5d, 0x91, 0xb4, 0xf8, 0x22, 0x32,
0x64, 0x36, 0xd7, 0x69, 0x30, 0x99, 0x84, 0x24, 0x2b, 0xcc, 0x37, 0xf0, 0x63, 0xcb, 0xcc, 0x64,
0x54, 0xb6, 0x3e, 0xf4, 0x31, 0x27, 0x67, 0x7e, 0x73, 0xee, 0xc9, 0x85, 0xb9, 0x35, 0x46, 0x69,
0xde, 0xf0, 0x1a, 0x1d, 0xb5, 0xce, 0x04, 0xb3, 0x78, 0x5e, 0x1b, 0x53, 0x2b, 0x5c, 0xf5, 0xa7,
0x6a, 0xfb, 0x7d, 0x15, 0xa4, 0x46, 0x1f, 0xb8, 0xb6, 0xd1, 0x70, 0xba, 0x6b, 0x40, 0x6d, 0x43,
0x3b, 0x5c, 0xe6, 0x67, 0x70, 0xc8, 0xb6, 0x4d, 0x83, 0x8e, 0xa4, 0x30, 0xe5, 0x42, 0x38, 0xf4,
0x3e, 0x9d, 0x64, 0x93, 0xe5, 0x8c, 0x8d, 0x47, 0xf2, 0x0c, 0x60, 0xa3, 0x24, 0x36, 0xa1, 0xfc,
0x81, 0x6d, 0x9a, 0x64, 0x93, 0xe5, 0x31, 0x9b, 0x0d, 0xca, 0x47, 0x6c, 0xf3, 0x53, 0x98, 0x5d,
0xbe, 0x3d, 0x77, 0x66, 0x6b, 0x2f, 0x04, 0x39, 0x81, 0x44, 0x8a, 0x08, 0x48, 0xa4, 0xc8, 0xbf,
0xc1, 0x3c, 0x5e, 0xae, 0x51, 0x57, 0xe8, 0xfc, 0x8d, 0xb4, 0xe4, 0x25, 0xdc, 0xaf, 0x3b, 0xa9,
0x8c, 0xd6, 0xa3, 0x02, 0xe8, 0x1f, 0x04, 0x9b, 0xd6, 0x91, 0xf5, 0x02, 0xa6, 0xae, 0xcf, 0xe6,
0xd3, 0x24, 0xdb, 0x5f, 0x1e, 0x15, 0x53, 0x3a, 0x64, 0x65, 0xa3, 0x9e, 0x7f, 0x85, 0x87, 0xef,
0xb8, 0xe5, 0x1b, 0x19, 0xda, 0xab, 0x86, 0x5b, 0x7f, 0x63, 0xc2, 0x5d, 0xe9, 0x19, 0x1c, 0x6b,
0xd4, 0xa5, 0xae, 0xca, 0x60, 0x02, 0x57, 0xfd, 0x5c, 0x07, 0x0c, 0x34, 0xea, 0x75, 0x75, 0xdd,
0x29, 0xf9, 0xaf, 0x09, 0x3c, 0xde, 0xa5, 0x5f, 0x4a, 0x1f, 0xc8, 0x2b, 0x48, 0x82, 0x8f, 0xec,
0x05, 0x1d, 0xea, 0xa5, 0x63, 0xbd, 0xf4, 0x7a, 0xec, 0x9f, 0x25, 0xc1, 0x93, 0x47, 0x70, 0x4f,
0x55, 0x5d, 0x94, 0xa4, 0xef, 0xe4, 0x40, 0x55, 0x17, 0x82, 0xac, 0x60, 0xe6, 0x23, 0xd0, 0xa7,
0xfb, 0xfd, 0x6c, 0x73, 0xba, 0xfb, 0x2b, 0xf6, 0xd7, 0x53, 0x7c, 0x81, 0x93, 0x4f, 0x46, 0xe0,
0x67, 0x63, 0xd4, 0xd5, 0x86, 0x2b, 0x74, 0xe4, 0x3d, 0xcc, 0xcf, 0xc4, 0x4f, 0x74, 0x41, 0x7a,
0x1c, 0xbf, 0x24, 0x4f, 0xe8, 0xff, 0xf2, 0x2e, 0x9e, 0xde, 0xca, 0xf8, 0xa1, 0x5b, 0x81, 0x7c,
0xaf, 0x78, 0x03, 0x0f, 0x86, 0x4a, 0xd7, 0xc3, 0x4e, 0x11, 0x0a, 0x70, 0x8e, 0x21, 0xd6, 0x45,
0xfe, 0x29, 0x6e, 0x41, 0xe8, 0xad, 0x87, 0xcc, 0xf7, 0xaa, 0xc3, 0x1e, 0xf9, 0xfa, 0x77, 0x00,
0x00, 0x00, 0xff, 0xff, 0xac, 0x38, 0x4d, 0xb8, 0x99, 0x02, 0x00, 0x00,
}

View File

@@ -1,37 +0,0 @@
syntax = "proto3";
import "google/protobuf/timestamp.proto";
import "google/protobuf/empty.proto";
message Runner {
string address = 1;
bytes client_key = 2;
}
message LBGroupId {
string id = 1;
}
message LBGroupMembership {
LBGroupId group_id = 1;
repeated Runner runners = 2;
}
message CapacitySnapshot {
LBGroupId group_id = 1;
uint64 mem_mb_total = 2;
}
message CapacitySnapshotList {
google.protobuf.Timestamp ts = 1;
string lb_id = 2;
repeated CapacitySnapshot snapshots = 3;
}
service NodePoolScaler {
rpc AdvertiseCapacity (CapacitySnapshotList) returns (google.protobuf.Empty) {}
}
service RunnerManager {
rpc GetLBGroup(LBGroupId) returns (LBGroupMembership) {}
}

View File

@@ -1,426 +0,0 @@
package poolmanager
import (
"time"
"context"
"math"
"sync"
model "github.com/fnproject/fn/poolmanager/grpc"
"github.com/fnproject/fn/poolmanager/server/controlplane"
"github.com/sirupsen/logrus"
)
type CapacityManager interface {
LBGroup(lbgid string) LBGroup
Merge(*model.CapacitySnapshotList)
}
type LBGroup interface {
Id() string
UpdateRequirements(lb string, total int64)
Purge(time.Time, func(LBGroup, string)) int64 // Remove outdated requirements, return updated value
GetMembers() []string // Return *ACTIVE* members
}
type Predictor interface {
// Given a series of requirements, return the target scale to attempt to reach
GetScale(time.Time, int64) int64
}
type capacityManager struct {
ctx context.Context
mx sync.RWMutex
cp controlplane.ControlPlane
lbg map[string]LBGroup
predictorFactory func() Predictor
}
func NewCapacityManager(ctx context.Context, cp controlplane.ControlPlane, opts ...func(*capacityManager) error) (CapacityManager, error) {
cm := &capacityManager{
ctx: ctx,
cp: cp,
lbg: make(map[string]LBGroup),
predictorFactory: newPredictor,
}
for _, o := range opts {
if err := o(cm); err != nil {
logrus.WithError(err).Error("Error handling option for CapacityManager")
return nil, err
}
}
return cm, nil
}
func WithPredictorFactory(pf func() Predictor) func(*capacityManager) error {
return func(cm *capacityManager) error {
cm.predictorFactory = pf
return nil
}
}
func (m *capacityManager) LBGroup(lbgid string) LBGroup {
m.mx.RLock()
// Optimistic path
if lbg, ok := m.lbg[lbgid]; ok {
m.mx.RUnlock()
return lbg
}
// We don't have one: upgrade the lock and allocate
m.mx.RUnlock()
m.mx.Lock()
defer m.mx.Unlock()
// Need to check again
if lbg, ok := m.lbg[lbgid]; ok {
return lbg
}
logrus.Infof("Making new LBG to handle %v", lbgid)
lbg := newLBGroup(lbgid, m.ctx, m.cp, m.predictorFactory)
m.lbg[lbgid] = lbg
return lbg
}
func (m *capacityManager) Merge(list *model.CapacitySnapshotList) {
lbid := list.GetLbId()
for _, new_req := range list.Snapshots {
lbg := new_req.GetGroupId().GetId()
logrus.Debugf("Merging snapshot %+v for %v from %v", new_req, lbg, lbid)
m.LBGroup(lbg).UpdateRequirements(lbid, int64(new_req.GetMemMbTotal()))
}
}
type lbGroup struct {
ctx context.Context
id string
// Attributes for managing incoming capacity requirements
cap_mx sync.RWMutex
total_wanted int64
requirements map[string]*requirement // NuLB id -> (ts, total_wanted)
controlStream chan requirement
// Attributes for managing runner pool membership
run_mx sync.RWMutex
cp controlplane.ControlPlane
current_capacity int64 // Of all active runners
target_capacity int64 // All active runners plus any we've already asked for
runners map[string]*runner // A map of everything we know about
active_runners []*runner // Everything currently in use
draining_runners []*runner // We keep tabs on these separately
dead_runners []*runner // Waiting for control plane to remove
predictor Predictor
}
type requirement struct {
ts time.Time // Time of last update
total_wanted int64
}
const (
RUNNER_ACTIVE = iota
RUNNER_DRAINING = iota
RUNNER_DEAD = iota
)
type runner struct {
id string // The same address may get recycled; we'll need to disambiguate somehow.
address string
status int
capacity int64
// XXX: If we're draining, this is handy to simulate runner readiness for shutdown
kill_after time.Time
}
func newLBGroup(lbgid string, ctx context.Context, cp controlplane.ControlPlane, predictorFactory func() Predictor) LBGroup {
lbg := &lbGroup{
ctx: ctx,
id: lbgid,
requirements: make(map[string]*requirement),
controlStream: make(chan requirement),
cp: cp,
runners: make(map[string]*runner),
predictor: predictorFactory(),
}
go lbg.control()
return lbg
}
func (lbg *lbGroup) Id() string {
return lbg.id
}
func (lbg *lbGroup) UpdateRequirements(lb string, total int64) {
logrus.Debugf("Updating capacity requirements for %v, lb=%v", lbg.Id(), lb)
defer logrus.Debugf("Updated %v, lb=%v", lbg.Id(), lb)
lbg.cap_mx.Lock()
last, ok := lbg.requirements[lb]
// Add in the new requirements, removing the old ones if required.
if !ok {
// This is a new NuLB that we're just learning about
last = &requirement{}
lbg.requirements[lb] = last
}
// Update totals: remove this LB's previous capacity assertions
lbg.total_wanted -= last.total_wanted
// Update totals: add this LB's new assertions and record them
lbg.total_wanted += total
// Keep a copy of this requirement
now := time.Now()
last.ts = now
last.total_wanted = total
// TODO: new_req also has a generation for the runner information that LB held. If that's out of date, signal that we need to readvertise
// Send a new signal to the capacity control loop
lbg.cap_mx.Unlock()
logrus.Debugf("Sending new capacity requirement of %v", lbg.total_wanted)
lbg.controlStream <- requirement{ts: now, total_wanted: lbg.total_wanted}
}
func (lbg *lbGroup) Purge(oldest time.Time, cb func(LBGroup, string)) int64 {
lbg.cap_mx.Lock()
defer lbg.cap_mx.Unlock()
for lb, req := range lbg.requirements {
if req.ts.Before(oldest) {
// We need to nix this entry, it's utterly out-of-date
lbg.total_wanted -= req.total_wanted
delete(lbg.requirements, lb)
// TODO: use a callback here to handle the deletion?
cb(lbg, lb)
}
}
return lbg.total_wanted
}
const PURGE_INTERVAL = 5 * time.Second
const VALID_REQUEST_LIFETIME = 500 * time.Millisecond
const POLL_INTERVAL = time.Second
const LARGEST_REQUEST_AT_ONCE = 20
const MAX_DRAINDOWN_LIFETIME = 50 * time.Second // For the moment.
func (lbg *lbGroup) control() {
// Control loop. This should receive a series of requirements.
// Occasionally, we walk the set of LBs that have spoken to us, purging those that are out-of-date
lastPurge := time.Now()
nextPurge := lastPurge.Add(PURGE_INTERVAL)
nextPoll := lastPurge
for {
logrus.Debugf("In capacity management loop for %v", lbg.Id())
select {
// Poll CP for runners (this will change, it's a stub)
// We put this first (and run it immediately) because if the NPM has just been restarted we want to
// repopulate our knowledge of what runners are currently up, so we don't generate spurious scaling requests
// to the CP.
case <-time.After(nextPoll.Sub(time.Now())):
logrus.Debugf("Polling for runners for %v", lbg.Id())
lbg.pollForRunners()
nextPoll = time.Now().Add(POLL_INTERVAL)
logrus.Debugf("Polled for %v", lbg.Id())
// Manage capacity requests
case <-time.After(nextPurge.Sub(time.Now())):
logrus.Debugf("Purging for %v", lbg.Id())
need := lbg.Purge(lastPurge, func(lbg LBGroup, lb string) {
logrus.Warnf("Purging LB %v from %v - no communication received", lb, lbg.Id())
})
lastPurge = time.Now()
nextPurge = lastPurge.Add(PURGE_INTERVAL)
lbg.target(lastPurge, need)
logrus.Debugf("Purged for %v", lbg.Id())
case req := <-lbg.controlStream:
logrus.Debugf("New requirement received by control loop for %v", req.total_wanted)
lbg.target(req.ts, req.total_wanted)
logrus.Debugf("New requirement handled", lbg.Id())
}
}
}
func (lbg *lbGroup) target(ts time.Time, target int64) {
if time.Now().Sub(ts) > VALID_REQUEST_LIFETIME {
// We have a request that's too old; drop it.
logrus.Warnf("Request for capacity is too old: %v", ts)
return
}
lbg.run_mx.Lock()
defer lbg.run_mx.Unlock()
desiredScale := lbg.predictor.GetScale(ts, target)
logrus.Debugf("Targeting capacity requirement of %v gives desired scale of %v", target, desiredScale)
// We have:
// - total capacity in active runners
// - required total capacity
// - capacity per runner
// - any additional capacity we've already asked for
// We scale appropriately.
if desiredScale > lbg.target_capacity {
// Scale up.
// Even including capacity we are expecting to come down the pipe, we don't have enough stuff.
// Begin by reactivating any runners we're currently draining down.
for desiredScale > lbg.target_capacity && len(lbg.draining_runners) > 0 {
// Begin with the one we started draining last.
runner := lbg.draining_runners[len(lbg.draining_runners)-1]
logrus.Infof("Recovering runner %v at %v from draindown", runner.id, runner.address)
lbg.draining_runners = lbg.draining_runners[:len(lbg.draining_runners)-1]
runner.status = RUNNER_ACTIVE
lbg.active_runners = append(lbg.active_runners, runner)
lbg.current_capacity += runner.capacity
lbg.target_capacity += runner.capacity
}
if desiredScale > lbg.target_capacity {
// We still need additional capacity
wanted := math.Min(math.Ceil(float64(target-lbg.target_capacity)/controlplane.CapacityPerRunner), LARGEST_REQUEST_AT_ONCE)
asked_for, err := lbg.cp.ProvisionRunners(lbg.Id(), int(wanted)) // Send the request; they'll show up later
if err != nil {
// Some kind of error during attempt to scale up
logrus.WithError(err).Error("Error occured during attempt to scale up")
return
}
lbg.target_capacity += int64(asked_for) * controlplane.CapacityPerRunner
}
} else if desiredScale <= lbg.current_capacity-controlplane.CapacityPerRunner {
// Scale down.
// We pick a node to turn off and move it to the draining pool.
for target <= lbg.current_capacity-controlplane.CapacityPerRunner && len(lbg.active_runners) > 0 {
// Begin with the one we added last.
runner := lbg.active_runners[len(lbg.active_runners)-1]
logrus.Infof("Marking runner %v at %v for draindown", runner.id, runner.address)
lbg.active_runners = lbg.active_runners[:len(lbg.active_runners)-1]
runner.status = RUNNER_DRAINING
runner.kill_after = time.Now().Add(MAX_DRAINDOWN_LIFETIME)
lbg.draining_runners = append(lbg.draining_runners, runner)
lbg.current_capacity -= runner.capacity
lbg.target_capacity -= runner.capacity
}
}
}
// Pool membership management
func (lbg *lbGroup) GetMembers() []string {
lbg.run_mx.RLock()
defer lbg.run_mx.RUnlock()
members := make([]string, len(lbg.active_runners))
for i, runner := range lbg.active_runners {
members[i] = runner.address
}
return members
}
// Three things handled here.
// First, if any drained runners are due to die, shut them off.
// Secondly, if the CP supplies any new capacity, add that the to pool as active.
// Finally, if dead runners have been shut down, remove them
func (lbg *lbGroup) pollForRunners() {
lbg.run_mx.Lock()
defer lbg.run_mx.Unlock()
now := time.Now()
// The oldest draining runner will be in the front of the pipe.
for len(lbg.draining_runners) > 0 && now.After(lbg.draining_runners[0].kill_after) {
// Mark this runner as to be killed
runner := lbg.draining_runners[0]
logrus.Infof("Drain down for runner %v at %v complete: signalling shutdown", runner.id, runner.address)
lbg.draining_runners = lbg.draining_runners[1:]
runner.status = RUNNER_DEAD
lbg.dead_runners = append(lbg.dead_runners, runner)
if err := lbg.cp.RemoveRunner(lbg.Id(), runner.id); err != nil {
logrus.WithError(err).Errorf("Error attempting to close down runner %v at %v", runner.id, runner.address)
}
}
// Get CP status and process it. This might be smarter but for the moment we just loop over everything we're told.
logrus.Debugf("Getting hosts from ControlPlane for %v", lbg.Id())
latestHosts, err := lbg.cp.GetLBGRunners(lbg.Id())
if err != nil {
logrus.WithError(err).Errorf("Problem talking to the CP to fetch runner status")
return
}
seen := make(map[string]bool)
for _, host := range latestHosts {
_, ok := lbg.runners[host.Id]
if ok {
// We already know about this
logrus.Debugf(" ... host %v at %d is known", host.Id, host.Address)
} else {
logrus.Infof(" ... host %v at %d is new", host.Id, host.Address)
// This is a new runner. Bring it into the active pool
runner := &runner{
id: host.Id,
address: host.Address,
status: RUNNER_ACTIVE,
capacity: host.Capacity,
}
lbg.runners[host.Id] = runner
lbg.active_runners = append(lbg.active_runners, runner)
lbg.current_capacity += runner.capacity // The total capacity is already computed, since we asked for this
}
seen[host.Id] = true
}
// Work out if runners that we asked to be killed have been shut down
logrus.Debugf("Removing dead hosts for %v", lbg.Id())
// The control plane might pull active or draining hosts out from under us. Deal with that too.
lbg.active_runners = removeDead(seen, lbg.runners, lbg.active_runners)
lbg.draining_runners = removeDead(seen, lbg.runners, lbg.draining_runners)
lbg.dead_runners = removeDead(seen, lbg.runners, lbg.dead_runners)
}
func removeDead(seen map[string]bool, runnerMap map[string]*runner, runnerList []*runner) []*runner {
i := 0
for _, runner := range runnerList {
if _, ok := seen[runner.id]; ok {
// This runner isn't shut down yet
runnerList[i] = runner
i++
} else {
logrus.Infof("Removing runner %v at %v that has disappeared", runner.id, runner.address)
delete(runnerMap, runner.id)
}
}
return runnerList[:i]
}
// Predictions. Given a timestamp and an input total capacity requirement, return the scale we should attempt to reach
func newPredictor() Predictor {
return &conservativePredictor{}
}
type conservativePredictor struct{}
func (p *conservativePredictor) GetScale(ts time.Time, target int64) int64 {
// This is the most conservative approach. If we have a target capacity, attempt to reach that capacity by
// asking for sufficient scale to satisfy it all at once.
return target
}

View File

@@ -1,20 +0,0 @@
/*
Interface between the Node Pool Manager and the Control Plane
*/
package controlplane
const CapacityPerRunner = 4096
type Runner struct {
Id string
Address string
// Other: certs etc here as managed and installed by CP
Capacity int64
}
type ControlPlane interface {
GetLBGRunners(lgbId string) ([]*Runner, error)
ProvisionRunners(lgbId string, n int) (int, error)
RemoveRunner(lbgId string, id string) error
}

View File

@@ -1,124 +0,0 @@
/**
* Dummy implementation for the controlplane that just adds delays
*/
package main
import (
"crypto/rand"
"fmt"
"log"
"os"
"strings"
"sync"
"time"
"github.com/fnproject/fn/poolmanager/server/controlplane"
)
const (
EnvFixedRunners = "FN_RUNNER_ADDRESSES"
)
type noopControlPlane struct {
mx sync.RWMutex
runners map[string][]*controlplane.Runner
_fakeRunners []string
}
const REQUEST_DURATION = 5 * time.Second
func init() {
ControlPlane = noopControlPlane{
runners: make(map[string][]*controlplane.Runner),
_fakeRunners: strings.Split(getEnv(EnvFixedRunners), ","),
}
}
func main() {
}
func (cp *noopControlPlane) GetLBGRunners(lbgId string) ([]*controlplane.Runner, error) {
cp.mx.RLock()
defer cp.mx.RUnlock()
runners := make([]*controlplane.Runner, 0)
if hosts, ok := cp.runners[lbgId]; ok {
for _, host := range hosts {
runners = append(runners, host) // In this CP implementation, a Runner is an immutable thing, so passing the pointer is fine
}
}
return runners, nil
}
func (cp *noopControlPlane) ProvisionRunners(lbgId string, n int) (int, error) {
// Simulate some small amount of time for the CP to service this request
go func() {
time.Sleep(REQUEST_DURATION)
cp.mx.Lock()
defer cp.mx.Unlock()
runners, ok := cp.runners[lbgId]
if !ok {
runners = make([]*controlplane.Runner, 0)
}
for i := 0; i < n; i++ {
runners = append(runners, cp.makeRunners(lbgId)...)
}
cp.runners[lbgId] = runners
}()
// How many did we actually ask for?
return n, nil
}
// Make runner(s)
func (cp *noopControlPlane) makeRunners(lbg string) []*controlplane.Runner {
var runners []*controlplane.Runner
for _, fakeRunner := range cp._fakeRunners {
b := make([]byte, 16)
_, err := rand.Read(b)
if err != nil {
log.Panic("Error constructing UUID for runner: ", err)
}
uuid := fmt.Sprintf("%X-%X-%X-%X-%X", b[0:4], b[4:6], b[6:8], b[8:10], b[10:])
runners = append(runners, &controlplane.Runner{
Id: uuid,
Address: fakeRunner,
Capacity: controlplane.CapacityPerRunner,
})
}
return runners
}
// Ditch a runner from the pool.
// We do this immediately - no point modelling a wait here
// note: if this actually took time, we'd want to detect this properly so as to not confuse the NPM
func (cp *noopControlPlane) RemoveRunner(lbgId string, id string) error {
cp.mx.Lock()
defer cp.mx.Unlock()
if runners, ok := cp.runners[lbgId]; ok {
newRunners := make([]*controlplane.Runner, 0)
for _, host := range runners {
if host.Id != id {
newRunners = append(newRunners, host)
}
}
cp.runners[lbgId] = newRunners
}
return nil
}
func getEnv(key string) string {
value, ok := os.LookupEnv(key)
if !ok {
log.Panicf("Missing config key: %v", key)
}
return value
}
var ControlPlane noopControlPlane

View File

@@ -1,31 +0,0 @@
#Vagrant testing
We've created a control plane interface for local development using vagrant as a backend. It interacts with minikube, where the rest of the components of the fn project are expected to run for local development.
##Getting it working
In order to create virtual machines you're going to want to configure the minikube and the hosts provided to share a network adapter. If you haven't already, [download the binary](https://github.com/kubernetes/minikube) and run `minikube start --vm-provider=virtualbox`. This should configure a new virtual box host only network called `vboxnet0`. From there, you should be able to run thsis code as is to start VMs backed by virtual box.
##Issues
Occasionally, you may run into an issue with the DHCP server the virtual box configures and will not be able to start a server.
If you see a message like this when running `vagrant up`:
```
A host only network interface you're attempting to configure via DHCP
already has a conflicting host only adapter with DHCP enabled. The
DHCP on this adapter is incompatible with the DHCP settings. Two
host only network interfaces are not allowed to overlap, and each
host only network interface can have only one DHCP server. Please
reconfigure your host only network or remove the virtual machine
using the other host only network.
```
Running the following command should clear all of these collision problems:
`VBoxManage dhcpserver remove --netname HostInterfaceNetworking-vboxnet0`
##Support
This is only intended to be used to test distributed components any use further than this will not be supported.

View File

@@ -1,76 +0,0 @@
# -*- mode: ruby -*-
# vi: set ft=ruby :
# All Vagrant configuration is done below. The "2" in Vagrant.configure
# configures the configuration version (we support older styles for
# backwards compatibility). Please don't change it unless you know what
# you're doing.
Vagrant.configure("2") do |config|
# The most common configuration options are documented and commented below.
# For a complete reference, please see the online documentation at
# https://docs.vagrantup.com.
# Every Vagrant development environment requires a box. You can search for
# boxes at https://vagrantcloud.com/search.
config.vm.box = "terrywang/oraclelinux-7-x86_64"
# Disable automatic box update checking. If you disable this, then
# boxes will only be checked for updates when the user runs
# `vagrant box outdated`. This is not recommended.
# config.vm.box_check_update = false
#
config.vm.provider "virtualbox" do |vb|
config.vm.network "private_network", :type => 'dhcp', :name => 'vboxnet0', :adapter => 2
end
#
# Create a forwarded port mapping which allows access to a specific port
# within the machine from a port on the host machine. In the example below,
# accessing "localhost:8080" will access port 80 on the guest machine.
# NOTE: This will enable public access to the opened port
# config.vm.network "forwarded_port", guest: 80, host: 8080
# Create a forwarded port mapping which allows access to a specific port
# within the machine from a port on the host machine and only allow access
# via 127.0.0.1 to disable public access
# config.vm.network "forwarded_port", guest: 80, host: 8080, host_ip: "127.0.0.1"
# Create a private network, which allows host-only access to the machine
# using a specific IP.
# config.vm.network "private_network", ip: "192.168.33.10"
# Create a public network, which generally matched to bridged network.
# Bridged networks make the machine appear as another physical device on
# your network.
# config.vm.network "public_network"
# Share an additional folder to the guest VM. The first argument is
# the path on the host to the actual folder. The second argument is
# the path on the guest to mount the folder. And the optional third
# argument is a set of non-required options.
# config.vm.synced_folder "../data", "/vagrant_data"
# Provider-specific configuration so you can fine-tune various
# backing providers for Vagrant. These expose provider-specific options.
# Example for VirtualBox:
#
# config.vm.provider "virtualbox" do |vb|
# # Display the VirtualBox GUI when booting the machine
# vb.gui = true
#
# # Customize the amount of memory on the VM:
# vb.memory = "1024"
# end
#
# View the documentation for the provider you are using for more
# information on available options.
# Enable provisioning with a shell script. Additional provisioners such as
# Puppet, Chef, Ansible, Salt, and Docker are also available. Please see the
# documentation for more information about their specific syntax and use.
# config.vm.provision "shell", inline: <<-SHELL
# apt-get update
# apt-get install -y apache2
# SHELL
end

View File

@@ -1,231 +0,0 @@
package main
import (
"bytes"
"errors"
"fmt"
"io"
"io/ioutil"
"log"
"os"
"os/exec"
"strings"
"github.com/fnproject/fn/poolmanager/server/controlplane"
idgen "github.com/fnproject/fn/api/id"
)
const (
vagrantEnv = "VAGRANT_PATH"
vagrantNamePrefix = "fn-vagrant"
)
func init() {
vagrantPath, ok := os.LookupEnv(vagrantEnv)
if !ok {
log.Panicf("Missing config key: %v", vagrantEnv)
}
ControlPlane = VagrantCP{
runnerMap: make(map[string][]*controlplane.Runner),
vagrantPath: vagrantPath,
}
}
func main() {
}
type VagrantCP struct {
runnerMap map[string][]*controlplane.Runner
vagrantPath string
}
func (v *VagrantCP) provision() (*controlplane.Runner, error) {
//set up dir
wd, err := os.Getwd()
if err != nil {
return nil, err
}
defer func() {
os.Chdir(wd)
}()
node := newNodeName()
nodeDir, err := ioutil.TempDir(wd, node)
if err != nil {
return nil, err
}
//copy vagrant file into there
newVagrantFile := fmt.Sprintf("%s/%s", nodeDir, "Vagrantfile")
err = copyFile(v.vagrantPath, newVagrantFile)
if err != nil {
return nil, err
}
err = os.Chdir(nodeDir)
if err != nil {
log.Println(err.Error())
return nil, err
}
vagrantUp := exec.Command("vagrant", "up")
err = vagrantUp.Run()
if err != nil {
log.Println(err.Error())
return nil, err
}
//Get the broadcast addr and call it a day
return getRunner(node)
}
//Gets the address that its broadcasting at
//VBoxManage guestproperty get "cp_default_1520116902053_77841" "/VirtualBox/GuestInfo/Net/1/V4/Broadcast"
func getRunner(node string) (*controlplane.Runner, error) {
//TODO make the vagrant file templated
vmsCmd := exec.Command("VBoxManage", "list", "vms")
var vmsOut bytes.Buffer
vmsCmd.Stdout = &vmsOut
err := vmsCmd.Run()
if err != nil {
return nil, err
}
vms := strings.Split(vmsOut.String(), "\n")
var realNode string
for _, candidate := range vms {
if strings.Contains(candidate, node) {
spl := strings.Split(candidate, " ")
realNode = spl[0]
}
}
//strip the quotes
if strings.Contains(realNode, "\"") {
realNode = realNode[1 : len(realNode)-1]
}
//guestproperty get "fn-vagrant-6ae28c23-445e-4b0b-a2cf-0102e66ec57a766389779_default_1520288274551_74039" /VirtualBox/GuestInfo/Net/1/V4/Broadcast
args := []string{"guestproperty", "get", realNode, "/VirtualBox/GuestInfo/Net/1/V4/Broadcast"}
broadCastAddrCmd := exec.Command("VBoxManage", args...)
var out bytes.Buffer
broadCastAddrCmd.Stdout = &out
var stdErr bytes.Buffer
broadCastAddrCmd.Stderr = &stdErr
err = broadCastAddrCmd.Run()
if err != nil {
log.Println("error running", err.Error(), stdErr.String())
return nil, err
}
addr := strings.Split(out.String(), ":")
if len(addr) != 2 {
return nil, fmt.Errorf("Unable to get address got:'%s' as output", out.String())
}
return &controlplane.Runner{
Id: realNode,
Address: addr[1],
}, nil
}
func (v *VagrantCP) GetLBGRunners(lgbID string) ([]*controlplane.Runner, error) {
runners, ok := v.runnerMap[lgbID]
if !ok {
return nil, errors.New("Not Found")
}
return runners, nil
}
func (v *VagrantCP) ProvisionRunners(lgbID string, n int) (int, error) {
runners := make([]*controlplane.Runner, 0, n)
for i := 0; i < n; i++ {
runner, err := v.provision()
runners = append(runners, runner)
if err != nil {
return 0, err
}
}
v.runnerMap[lgbID] = runners
return n, nil
}
func (v *VagrantCP) RemoveRunner(lbgID string, id string) error {
wd, err := os.Getwd()
if err != nil {
return err
}
defer func() {
os.Chdir(wd)
}()
runners, ok := v.runnerMap[lbgID]
if !ok {
return errors.New("No lgbID with this name")
}
//look for it in the customers map
found := false
for _, r := range runners {
if id == r.Id {
found = true
break
}
}
if found == false {
return errors.New("No VM by this ID")
}
//switch to the dir and remove it
//vm name is fn-vagrant-7183faa4-7321-47e9-8fd9-4a0aa1ac818e497509110_default_1520299457972_92567 everything before the first _
split := strings.Split(id, "_")
dirName := split[0]
err = os.Chdir(dirName)
if err != nil {
log.Println(err.Error())
return err
}
destroyCmd := exec.Command("vagrant", "destroy", "-f")
err = destroyCmd.Run()
if err != nil {
log.Println(err.Error())
return err
}
// back to working dir and rm -rf ignore these erro
err = os.Chdir(wd)
if err != nil {
return err
}
err = os.RemoveAll(dirName)
if err != nil {
return err
}
return nil
}
func newNodeName() string {
id := idgen.New()
return fmt.Sprintf("%s-%s", vagrantNamePrefix, id.String())
}
//TODO move to a util folder if needed again
func copyFile(src string, dst string) error {
// Open the source file for reading
s, err := os.Open(src)
if err != nil {
return err
}
defer s.Close()
// Open the destination file for writing
d, err := os.Create(dst)
if err != nil {
return err
}
// Copy the contents of the source file into the destination file
if _, err := io.Copy(d, s); err != nil {
d.Close()
return err
}
// Return any errors that result from closing the destination file
// Will return nil if no errors occurred
return d.Close()
}
var ControlPlane VagrantCP

View File

@@ -1,191 +0,0 @@
package main
import (
"context"
"net"
google_protobuf1 "github.com/golang/protobuf/ptypes/empty"
"google.golang.org/grpc"
"github.com/fnproject/fn/poolmanager"
model "github.com/fnproject/fn/poolmanager/grpc"
"github.com/fnproject/fn/poolmanager/server/controlplane"
"crypto/tls"
"crypto/x509"
"errors"
"fmt"
"io/ioutil"
"log"
"os"
"path/filepath"
"plugin"
"github.com/sirupsen/logrus"
"google.golang.org/grpc/credentials"
)
type npmService struct {
// Control plane "client"
capMan poolmanager.CapacityManager
}
func newNPMService(ctx context.Context, cp controlplane.ControlPlane) *npmService {
cm, err := poolmanager.NewCapacityManager(ctx, cp)
if err != nil {
logrus.Panic("Cannot construct capacity manager")
}
return &npmService{
capMan: cm,
}
}
func (npm *npmService) AdvertiseCapacity(ctx context.Context, snapshots *model.CapacitySnapshotList) (*google_protobuf1.Empty, error) {
npm.capMan.Merge(snapshots)
logrus.Debugf("Merged capacity requests %+v", snapshots)
return &google_protobuf1.Empty{}, nil
}
func (npm *npmService) GetLBGroup(ctx context.Context, gid *model.LBGroupId) (*model.LBGroupMembership, error) {
lbg := npm.capMan.LBGroup(gid.GetId())
membership := &model.LBGroupMembership{GroupId: gid}
members := lbg.GetMembers()
runners := make([]*model.Runner, len(members))
for i, r := range members {
runners[i] = &model.Runner{Address: r}
}
membership.Runners = runners
if len(runners) > 0 {
logrus.Infof("LBGroup membership for %v is %+v", gid.GetId(), runners)
} else {
logrus.Debugf("LBGroup membership for %v is %+v", gid.GetId(), runners)
}
return membership, nil
}
const (
// Certificates to communicate with other FN nodes
EnvCert = "FN_NODE_CERT"
EnvCertKey = "FN_NODE_CERT_KEY"
EnvCertAuth = "FN_NODE_CERT_AUTHORITY"
EnvPort = "FN_PORT"
ControlPlaneSO = "CONTROL_PLANE_SO"
)
func getAndCheckFile(envVar string) (string, error) {
filename := getEnv(envVar)
if filename == "" {
return "", fmt.Errorf("Please provide a valid file path in the %v variable", envVar)
}
abs, err := filepath.Abs(filename)
if err != nil {
return "", fmt.Errorf("Unable to resolve %v: please specify a valid and readable file", filename)
}
_, err = os.Stat(abs)
if err != nil {
return "", fmt.Errorf("Cannot stat %v: please specify a valid and readable file", abs)
}
return abs, nil
}
func createGrpcCreds(cert string, key string, ca string) (grpc.ServerOption, error) {
// Load the certificates from disk
certificate, err := tls.LoadX509KeyPair(cert, key)
if err != nil {
return nil, fmt.Errorf("could not load server key pair: %s", err)
}
// Create a certificate pool from the certificate authority
certPool := x509.NewCertPool()
authority, err := ioutil.ReadFile(ca)
if err != nil {
return nil, fmt.Errorf("could not read ca certificate: %s", err)
}
if ok := certPool.AppendCertsFromPEM(authority); !ok {
return nil, errors.New("failed to append client certs")
}
creds := credentials.NewTLS(&tls.Config{
ClientAuth: tls.RequireAndVerifyClientCert,
Certificates: []tls.Certificate{certificate},
ClientCAs: certPool,
})
return grpc.Creds(creds), nil
}
func newPluggableControlPlane() controlplane.ControlPlane {
pluginLocation := getEnv(ControlPlaneSO)
controlPlanePlugin, err := plugin.Open(pluginLocation)
if err != nil {
panic(err)
}
cpSymbol, err := controlPlanePlugin.Lookup("ControlPlane")
if err != nil {
panic(err)
}
cp := cpSymbol.(controlplane.ControlPlane)
logrus.Infof("Started controlplane : %s", cp)
return cp
}
func main() {
level, err := logrus.ParseLevel(getEnv("FN_LOG_LEVEL"))
if err != nil {
logrus.Panic("Set a valid FN_LOG_LEVEL")
}
logrus.SetLevel(level)
// Obtain certificate paths
cert, err := getAndCheckFile(EnvCert)
if err != nil {
logrus.Fatal(err)
}
key, err := getAndCheckFile(EnvCertKey)
if err != nil {
logrus.Fatal(err)
}
ca, err := getAndCheckFile(EnvCertAuth)
if err != nil {
logrus.Fatal(err)
}
gRPCCreds, err := createGrpcCreds(cert, key, ca)
if err != nil {
logrus.Fatal(err)
}
gRPCServer := grpc.NewServer(gRPCCreds)
logrus.Info("Starting Node Pool Manager gRPC service")
svc := newNPMService(context.Background(), newPluggableControlPlane())
model.RegisterNodePoolScalerServer(gRPCServer, svc)
model.RegisterRunnerManagerServer(gRPCServer, svc)
port := getEnv(EnvPort)
l, err := net.Listen("tcp", fmt.Sprintf("0.0.0.0:%s", port))
if err != nil {
logrus.Fatalf("could not listen on port %s: %s", port, err)
}
if err := gRPCServer.Serve(l); err != nil {
logrus.Fatalf("grpc serve error: %s", err)
}
}
func getEnv(key string) string {
value, ok := os.LookupEnv(key)
if !ok {
log.Panicf("Missing config key: %v", key)
}
return value
}

View File

@@ -7,8 +7,8 @@ import (
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/agent/hybrid"
agent_grpc "github.com/fnproject/fn/api/agent/nodepool/grpc"
"github.com/fnproject/fn/api/models"
pool "github.com/fnproject/fn/api/runnerpool"
"github.com/fnproject/fn/api/server"
"github.com/sirupsen/logrus"
@@ -24,17 +24,17 @@ import (
)
type SystemTestNodePool struct {
runners []agent.Runner
runners []pool.Runner
}
func NewSystemTestNodePool() (agent.NodePool, error) {
func NewSystemTestNodePool() (pool.RunnerPool, error) {
myAddr := whoAmI()
runners := []string{
fmt.Sprintf("%s:9190", myAddr),
fmt.Sprintf("%s:9191", myAddr),
fmt.Sprintf("%s:9192", myAddr),
}
return agent_grpc.DefaultStaticNodePool(runners), nil
return agent.DefaultStaticRunnerPool(runners), nil
}
func SetUpSystem() error {

View File

@@ -1,7 +0,0 @@
language: go
go:
- 1.4
- 1.5
- 1.7
- tip

View File

@@ -1,69 +0,0 @@
SipHash (Go)
============
[![Build Status](https://travis-ci.org/dchest/siphash.svg)](https://travis-ci.org/dchest/siphash)
Go implementation of SipHash-2-4, a fast short-input PRF created by
Jean-Philippe Aumasson and Daniel J. Bernstein (http://131002.net/siphash/).
## Installation
$ go get github.com/dchest/siphash
## Usage
import "github.com/dchest/siphash"
There are two ways to use this package.
The slower one is to use the standard hash.Hash64 interface:
h := siphash.New(key)
h.Write([]byte("Hello"))
sum := h.Sum(nil) // returns 8-byte []byte
or
sum64 := h.Sum64() // returns uint64
The faster one is to use Hash() function, which takes two uint64 parts of
16-byte key and a byte slice, and returns uint64 hash:
sum64 := siphash.Hash(key0, key1, []byte("Hello"))
The keys and output are little-endian.
## Functions
### func Hash(k0, k1 uint64, p []byte) uint64
Hash returns the 64-bit SipHash-2-4 of the given byte slice with two
64-bit parts of 128-bit key: k0 and k1.
### func Hash128(k0, k1 uint64, p []byte) (uint64, uint64)
Hash128 returns the 128-bit SipHash-2-4 of the given byte slice with two
64-bit parts of 128-bit key: k0 and k1.
Note that 128-bit SipHash is considered experimental by SipHash authors at this time.
### func New(key []byte) hash.Hash64
New returns a new hash.Hash64 computing SipHash-2-4 with 16-byte key.
### func New128(key []byte) hash.Hash
New128 returns a new hash.Hash computing SipHash-2-4 with 16-byte key and 16-byte output.
Note that 16-byte output is considered experimental by SipHash authors at this time.
## Public domain dedication
Written by Dmitry Chestnykh and Damian Gryski.
To the extent possible under law, the authors have dedicated all copyright
and related and neighboring rights to this software to the public domain
worldwide. This software is distributed without any warranty.
http://creativecommons.org/publicdomain/zero/1.0/

View File

@@ -1,148 +0,0 @@
// +build !arm,!amd64 appengine gccgo
package siphash
func once(d *digest) {
blocks(d, d.x[:])
}
func finalize(d *digest) uint64 {
d0 := *d
once(&d0)
v0, v1, v2, v3 := d0.v0, d0.v1, d0.v2, d0.v3
v2 ^= 0xff
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
return v0 ^ v1 ^ v2 ^ v3
}
func blocks(d *digest, p []uint8) {
v0, v1, v2, v3 := d.v0, d.v1, d.v2, d.v3
for len(p) >= BlockSize {
m := uint64(p[0]) | uint64(p[1])<<8 | uint64(p[2])<<16 | uint64(p[3])<<24 |
uint64(p[4])<<32 | uint64(p[5])<<40 | uint64(p[6])<<48 | uint64(p[7])<<56
v3 ^= m
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
v0 ^= m
p = p[BlockSize:]
}
d.v0, d.v1, d.v2, d.v3 = v0, v1, v2, v3
}

View File

@@ -1,86 +0,0 @@
// +build amd64,!appengine,!gccgo
#define ROUND(v0, v1, v2, v3) \
ADDQ v1, v0; \
RORQ $51, v1; \
ADDQ v3, v2; \
XORQ v0, v1; \
RORQ $48, v3; \
RORQ $32, v0; \
XORQ v2, v3; \
ADDQ v1, v2; \
ADDQ v3, v0; \
RORQ $43, v3; \
RORQ $47, v1; \
XORQ v0, v3; \
XORQ v2, v1; \
RORQ $32, v2
// blocks(d *digest, data []uint8)
TEXT ·blocks(SB),4,$0-32
MOVQ d+0(FP), BX
MOVQ 0(BX), R9 // R9 = v0
MOVQ 8(BX), R10 // R10 = v1
MOVQ 16(BX), R11 // R11 = v2
MOVQ 24(BX), R12 // R12 = v3
MOVQ p_base+8(FP), DI // DI = *uint64
MOVQ p_len+16(FP), SI // SI = nblocks
XORL DX, DX // DX = index (0)
SHRQ $3, SI // SI /= 8
body:
CMPQ DX, SI
JGE end
MOVQ 0(DI)(DX*8), CX // CX = m
XORQ CX, R12
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
XORQ CX, R9
ADDQ $1, DX
JMP body
end:
MOVQ R9, 0(BX)
MOVQ R10, 8(BX)
MOVQ R11, 16(BX)
MOVQ R12, 24(BX)
RET
// once(d *digest)
TEXT ·once(SB),4,$0-8
MOVQ d+0(FP), BX
MOVQ 0(BX), R9 // R9 = v0
MOVQ 8(BX), R10 // R10 = v1
MOVQ 16(BX), R11 // R11 = v2
MOVQ 24(BX), R12 // R12 = v3
MOVQ 48(BX), CX // CX = d.x[:]
XORQ CX, R12
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
XORQ CX, R9
MOVQ R9, 0(BX)
MOVQ R10, 8(BX)
MOVQ R11, 16(BX)
MOVQ R12, 24(BX)
RET
// finalize(d *digest) uint64
TEXT ·finalize(SB),4,$0-16
MOVQ d+0(FP), BX
MOVQ 0(BX), R9 // R9 = v0
MOVQ 8(BX), R10 // R10 = v1
MOVQ 16(BX), R11 // R11 = v2
MOVQ 24(BX), R12 // R12 = v3
MOVQ 48(BX), CX // CX = d.x[:]
XORQ CX, R12
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
XORQ CX, R9
NOTB R11
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
ROUND(R9, R10, R11, R12)
XORQ R12, R11
XORQ R10, R9
XORQ R11, R9
MOVQ R9, ret+8(FP)
RET

View File

@@ -1,144 +0,0 @@
#include "textflag.h"
#define R10 g
#define ROUND()\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<13,R0,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R1,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R4,R4;\
ADC R7,R5,R5;\
EOR R6<<16,R4,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R5,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<21,R1,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R0,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R4,R4;\
ADC R11,R5,R5;\
EOR R8<<17,R4,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R5,R3;\
EOR R11<<17,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<13,R1,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R0,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R5,R5;\
ADC R7,R4,R4;\
EOR R6<<16,R5,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R4,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<21,R0,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R1,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R5,R5;\
ADC R11,R4,R4;\
EOR R8<<17,R5,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R4,R3;\
EOR R11<<17,R3,R3;\
// once(d *digest)
TEXT ·once(SB),NOSPLIT,$4-4
MOVW d+0(FP),R8
MOVM.IA (R8),[R0,R1,R2,R3,R4,R5,R6,R7]
MOVW 48(R8),R12
MOVW 52(R8),R14
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
MOVW d+0(FP),R8
MOVM.IA [R0,R1,R2,R3,R4,R5,R6,R7],(R8)
RET
// finalize(d *digest) uint64
TEXT ·finalize(SB),NOSPLIT,$4-12
MOVW d+0(FP),R8
MOVM.IA (R8),[R0,R1,R2,R3,R4,R5,R6,R7]
MOVW 48(R8),R12
MOVW 52(R8),R14
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
EOR $255,R4
ROUND()
ROUND()
EOR R2,R0,R0
EOR R3,R1,R1
EOR R6,R4,R4
EOR R7,R5,R5
EOR R4,R0,R0
EOR R5,R1,R1
MOVW R0,ret_lo+4(FP)
MOVW R1,ret_hi+8(FP)
RET
// blocks(d *digest, data []uint8)
TEXT ·blocks(SB),NOSPLIT,$8-16
MOVW R10,sav-8(SP)
MOVW d+0(FP),R8
MOVM.IA (R8),[R0,R1,R2,R3,R4,R5,R6,R7]
MOVW p+4(FP),R10
MOVW p_len+8(FP),R11
ADD R10,R11,R11
MOVW R11,endp-4(SP)
AND.S $3,R10,R8
BNE blocksunaligned
blocksloop:
MOVM.IA.W (R10),[R12,R14]
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
MOVW endp-4(SP),R11
CMP R11,R10
BLO blocksloop
MOVW d+0(FP),R8
MOVM.IA [R0,R1,R2,R3,R4,R5,R6,R7],(R8)
MOVW sav-8(SP),R10
RET
blocksunaligned:
MOVB (R10),R12
MOVB 1(R10),R11
ORR R11<<8,R12,R12
MOVB 2(R10),R11
ORR R11<<16,R12,R12
MOVB 3(R10),R11
ORR R11<<24,R12,R12
MOVB 4(R10),R14
MOVB 5(R10),R11
ORR R11<<8,R14,R14
MOVB 6(R10),R11
ORR R11<<16,R14,R14
MOVB 7(R10),R11
ORR R11<<24,R14,R14
ADD $8,R10,R10
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
MOVW endp-4(SP),R11
CMP R11,R10
BLO blocksunaligned
MOVW d+0(FP),R8
MOVM.IA [R0,R1,R2,R3,R4,R5,R6,R7],(R8)
MOVW sav-8(SP),R10
RET

View File

@@ -1,216 +0,0 @@
// +build !arm,!amd64 appengine gccgo
// Written in 2012 by Dmitry Chestnykh.
//
// To the extent possible under law, the author have dedicated all copyright
// and related and neighboring rights to this software to the public domain
// worldwide. This software is distributed without any warranty.
// http://creativecommons.org/publicdomain/zero/1.0/
package siphash
// Hash returns the 64-bit SipHash-2-4 of the given byte slice with two 64-bit
// parts of 128-bit key: k0 and k1.
func Hash(k0, k1 uint64, p []byte) uint64 {
// Initialization.
v0 := k0 ^ 0x736f6d6570736575
v1 := k1 ^ 0x646f72616e646f6d
v2 := k0 ^ 0x6c7967656e657261
v3 := k1 ^ 0x7465646279746573
t := uint64(len(p)) << 56
// Compression.
for len(p) >= BlockSize {
m := uint64(p[0]) | uint64(p[1])<<8 | uint64(p[2])<<16 | uint64(p[3])<<24 |
uint64(p[4])<<32 | uint64(p[5])<<40 | uint64(p[6])<<48 | uint64(p[7])<<56
v3 ^= m
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
v0 ^= m
p = p[BlockSize:]
}
// Compress last block.
switch len(p) {
case 7:
t |= uint64(p[6]) << 48
fallthrough
case 6:
t |= uint64(p[5]) << 40
fallthrough
case 5:
t |= uint64(p[4]) << 32
fallthrough
case 4:
t |= uint64(p[3]) << 24
fallthrough
case 3:
t |= uint64(p[2]) << 16
fallthrough
case 2:
t |= uint64(p[1]) << 8
fallthrough
case 1:
t |= uint64(p[0])
}
v3 ^= t
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
v0 ^= t
// Finalization.
v2 ^= 0xff
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
return v0 ^ v1 ^ v2 ^ v3
}

View File

@@ -1,302 +0,0 @@
// +build !arm,!amd64 appengine gccgo
// Written in 2012 by Dmitry Chestnykh.
// Modifications 2014 for 128-bit hash function by Damian Gryski.
//
// To the extent possible under law, the authors have dedicated all copyright
// and related and neighboring rights to this software to the public domain
// worldwide. This software is distributed without any warranty.
// http://creativecommons.org/publicdomain/zero/1.0/
package siphash
// Hash returns the 128-bit SipHash-2-4 of the given byte slice with two 64-bit
// parts of 128-bit key: k0 and k1.
//
// Note that 128-bit SipHash is considered experimental by SipHash authors at this time.
func Hash128(k0, k1 uint64, p []byte) (uint64, uint64) {
// Initialization.
v0 := k0 ^ 0x736f6d6570736575
v1 := k1 ^ 0x646f72616e646f6d
v2 := k0 ^ 0x6c7967656e657261
v3 := k1 ^ 0x7465646279746573
t := uint64(len(p)) << 56
v1 ^= 0xee
// Compression.
for len(p) >= BlockSize {
m := uint64(p[0]) | uint64(p[1])<<8 | uint64(p[2])<<16 | uint64(p[3])<<24 |
uint64(p[4])<<32 | uint64(p[5])<<40 | uint64(p[6])<<48 | uint64(p[7])<<56
v3 ^= m
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
v0 ^= m
p = p[BlockSize:]
}
// Compress last block.
switch len(p) {
case 7:
t |= uint64(p[6]) << 48
fallthrough
case 6:
t |= uint64(p[5]) << 40
fallthrough
case 5:
t |= uint64(p[4]) << 32
fallthrough
case 4:
t |= uint64(p[3]) << 24
fallthrough
case 3:
t |= uint64(p[2]) << 16
fallthrough
case 2:
t |= uint64(p[1]) << 8
fallthrough
case 1:
t |= uint64(p[0])
}
v3 ^= t
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
v0 ^= t
// Finalization.
v2 ^= 0xee
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
r0 := v0 ^ v1 ^ v2 ^ v3
v1 ^= 0xdd
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
r1 := v0 ^ v1 ^ v2 ^ v3
return r0, r1
}

View File

@@ -1,292 +0,0 @@
// +build amd64,!appengine,!gccgo
// This is a translation of the gcc output of FloodyBerry's pure-C public
// domain siphash implementation at https://github.com/floodyberry/siphash
// This assembly code has been modified from the 64-bit output to the experiment 128-bit output.
// SI = v0
// AX = v1
// CX = v2
// DX = v3
// func Hash128(k0, k1 uint64, b []byte) (r0 uint64, r1 uint64)
TEXT ·Hash128(SB),4,$0-56
MOVQ k0+0(FP),CX
MOVQ $0x736F6D6570736575,R9
MOVQ k1+8(FP),DI
MOVQ $0x6C7967656E657261,BX
MOVQ $0x646F72616E646F6D,AX
MOVQ b_len+24(FP),DX
XORQ $0xEE,AX
MOVQ DX,R11
MOVQ DX,R10
XORQ CX,R9
XORQ CX,BX
MOVQ $0x7465646279746573,CX
XORQ DI,AX
XORQ DI,CX
SHLQ $0x38,R11
XORQ DI,DI
MOVQ b_base+16(FP),SI
ANDQ $0xFFFFFFFFFFFFFFF8,R10
JE afterLoop
XCHGQ AX,AX
loopBody:
MOVQ 0(SI)(DI*1),R8
ADDQ AX,R9
RORQ $0x33,AX
XORQ R9,AX
RORQ $0x20,R9
ADDQ $0x8,DI
XORQ R8,CX
ADDQ CX,BX
RORQ $0x30,CX
XORQ BX,CX
ADDQ AX,BX
RORQ $0x2F,AX
ADDQ CX,R9
RORQ $0x2B,CX
XORQ BX,AX
XORQ R9,CX
RORQ $0x20,BX
ADDQ AX,R9
ADDQ CX,BX
RORQ $0x33,AX
RORQ $0x30,CX
XORQ R9,AX
XORQ BX,CX
RORQ $0x20,R9
ADDQ AX,BX
ADDQ CX,R9
RORQ $0x2F,AX
RORQ $0x2B,CX
XORQ BX,AX
RORQ $0x20,BX
XORQ R9,CX
XORQ R8,R9
CMPQ R10,DI
JA loopBody
afterLoop:
SUBQ R10,DX
CMPQ DX,$0x7
JA afterSwitch
// no support for jump tables
CMPQ DX,$0x7
JE sw7
CMPQ DX,$0x6
JE sw6
CMPQ DX,$0x5
JE sw5
CMPQ DX,$0x4
JE sw4
CMPQ DX,$0x3
JE sw3
CMPQ DX,$0x2
JE sw2
CMPQ DX,$0x1
JE sw1
JMP afterSwitch
sw7: MOVBQZX 6(SI)(DI*1),DX
SHLQ $0x30,DX
ORQ DX,R11
sw6: MOVBQZX 0x5(SI)(DI*1),DX
SHLQ $0x28,DX
ORQ DX,R11
sw5: MOVBQZX 0x4(SI)(DI*1),DX
SHLQ $0x20,DX
ORQ DX,R11
sw4: MOVBQZX 0x3(SI)(DI*1),DX
SHLQ $0x18,DX
ORQ DX,R11
sw3: MOVBQZX 0x2(SI)(DI*1),DX
SHLQ $0x10,DX
ORQ DX,R11
sw2: MOVBQZX 0x1(SI)(DI*1),DX
SHLQ $0x8,DX
ORQ DX,R11
sw1: MOVBQZX 0(SI)(DI*1),DX
ORQ DX,R11
afterSwitch:
LEAQ (AX)(R9*1),SI
XORQ R11,CX
RORQ $0x33,AX
ADDQ CX,BX
MOVQ CX,DX
XORQ SI,AX
RORQ $0x30,DX
RORQ $0x20,SI
LEAQ 0(BX)(AX*1),CX
XORQ BX,DX
RORQ $0x2F,AX
ADDQ DX,SI
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
XORQ SI,AX
RORQ $0x30,DX
RORQ $0x20,SI
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
ADDQ DX,SI
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ SI,DX
XORQ R11,SI
XORB $0xEE,CL
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
RORQ $0x30,DX
XORQ SI,AX
XORQ CX,DX
RORQ $0x20,SI
ADDQ AX,CX
ADDQ DX,SI
RORQ $0x2F,AX
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ SI,AX
RORQ $0x20,SI
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
ADDQ DX,SI
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ CX,DX
XORQ SI,AX
RORQ $0x20,SI
ADDQ DX,SI
ADDQ AX,CX
RORQ $0x2F,AX
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ SI,DX
// gcc optimized the tail end of this function differently. However,
// we need to preserve out registers to carry out the second stage of
// the finalization. This is a duplicate of an earlier finalization
// round.
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
RORQ $0x30,DX
XORQ SI,AX
XORQ CX,DX
RORQ $0x20,SI
ADDQ AX,CX
ADDQ DX,SI
RORQ $0x2F,AX
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
// Stuff the result into BX instead of AX as gcc had done
MOVQ SI,BX
XORQ AX,BX
XORQ DX,BX
XORQ CX,BX
MOVQ BX,ret+40(FP)
// Start the second finalization round
XORB $0xDD,AL
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
RORQ $0x30,DX
XORQ SI,AX
XORQ CX,DX
RORQ $0x20,SI
ADDQ AX,CX
ADDQ DX,SI
RORQ $0x2F,AX
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ SI,AX
RORQ $0x20,SI
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
ADDQ DX,SI
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ CX,DX
XORQ SI,AX
RORQ $0x20,SI
ADDQ DX,SI
ADDQ AX,CX
RORQ $0x2F,AX
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ SI,DX
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
RORQ $0x30,DX
XORQ SI,AX
XORQ CX,DX
RORQ $0x20,SI
ADDQ AX,CX
ADDQ DX,SI
RORQ $0x2F,AX
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
MOVQ SI,BX
XORQ AX,BX
XORQ DX,BX
XORQ CX,BX
MOVQ BX,ret1+48(FP)
RET

View File

@@ -1,169 +0,0 @@
#include "textflag.h"
#define R10 g
#define ROUND()\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<13,R0,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R1,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R4,R4;\
ADC R7,R5,R5;\
EOR R6<<16,R4,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R5,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<21,R1,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R0,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R4,R4;\
ADC R11,R5,R5;\
EOR R8<<17,R4,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R5,R3;\
EOR R11<<17,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<13,R1,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R0,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R5,R5;\
ADC R7,R4,R4;\
EOR R6<<16,R5,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R4,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<21,R0,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R1,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R5,R5;\
ADC R11,R4,R4;\
EOR R8<<17,R5,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R4,R3;\
EOR R11<<17,R3,R3;\
// Hash128(k0, k1 uint64, b []byte) (uint64, uint64)
TEXT ·Hash128(SB),NOSPLIT,$8-44
MOVW R10,sav-8(SP)
MOVW k0_lo+0(FP),R12
MOVW k0_hi+4(FP),R14
MOVW $0x70736575,R0
MOVW $0x736f6d65,R1
MOVW $0x6e657261,R4
MOVW $0x6c796765,R5
EOR R12,R0,R0
EOR R14,R1,R1
EOR R12,R4,R4
EOR R14,R5,R5
MOVW k1_lo+8(FP),R12
MOVW k1_hi+12(FP),R14
MOVW $0x6e646f83,R2
MOVW $0x646f7261,R3
MOVW $0x79746573,R6
MOVW $0x74656462,R7
EOR R12,R2,R2
EOR R14,R3,R3
EOR R12,R6,R6
EOR R14,R7,R7
MOVW b+16(FP),R10
MOVW b_len+20(FP),R11
ADD R10,R11,R11
MOVW R11,endb-4(SP)
hashloop128:
MOVW endb-4(SP),R11
SUB R10,R11,R11
SUB.S $8,R11
BLO hashend128
MOVM.IA.W (R10),[R12,R14]
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
B hashloop128
hashloop128unaligned:
MOVW endb-4(SP),R11
SUB R10,R11,R11
SUB.S $8,R11
BLO hashend128
MOVB (R10),R12
MOVB 1(R10),R11
ORR R11<<8,R12,R12
MOVB 2(R10),R11
ORR R11<<16,R12,R12
MOVB 3(R10),R11
ORR R11<<24,R12,R12
MOVB 4(R10),R14
MOVB 5(R10),R11
ORR R11<<8,R14,R14
MOVB 6(R10),R11
ORR R11<<16,R14,R14
MOVB 7(R10),R11
ORR R11<<24,R14,R14
ADD $8,R10,R10
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
B hashloop128unaligned
hashend128:
MOVW $0x0,R12
MOVW $0x0,R14
RSB $0,R11,R11
AND.S $7,R11
BEQ hashlast128
MOVW (R10),R12
SLL $3,R11
AND $63,R11
SUB.S $32,R11,R11
BEQ hashlast128
BLO hashhi128
MOVW R12<<R11,R12
MOVW R12>>R11,R12
B hashlast128
hashhi128:
ADD $32,R11
MOVW 4(R10),R14
MOVW R14<<R11,R14
MOVW R14>>R11,R14
hashlast128:
MOVW b_len+20(FP),R11
ORR R11<<24,R14,R14
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
EOR $238,R4
ROUND()
ROUND()
EOR R0,R2,R12
EOR R1,R3,R14
EOR R4,R12,R12
EOR R5,R14,R14
EOR R6,R12,R12
EOR R7,R14,R14
MOVW R12,ret_lo+28(FP)
MOVW R14,ret_hi+32(FP)
EOR $221,R2
ROUND()
ROUND()
EOR R0,R2,R12
EOR R1,R3,R14
EOR R4,R12,R12
EOR R5,R14,R14
EOR R6,R12,R12
EOR R7,R14,R14
MOVW R12,unnamed_lo+36(FP)
MOVW R14,unnamed_hi+40(FP)
MOVW sav-8(SP),R10
RET

View File

@@ -1,201 +0,0 @@
// +build amd64,!appengine,!gccgo
// This is a translation of the gcc output of FloodyBerry's pure-C public
// domain siphash implementation at https://github.com/floodyberry/siphash
// func Hash(k0, k1 uint64, b []byte) uint64
TEXT ·Hash(SB),4,$0-48
MOVQ k0+0(FP),CX
MOVQ $0x736F6D6570736575,R9
MOVQ k1+8(FP),DI
MOVQ $0x6C7967656E657261,BX
MOVQ $0x646F72616E646F6D,AX
MOVQ b_len+24(FP),DX
MOVQ DX,R11
MOVQ DX,R10
XORQ CX,R9
XORQ CX,BX
MOVQ $0x7465646279746573,CX
XORQ DI,AX
XORQ DI,CX
SHLQ $0x38,R11
XORQ DI,DI
MOVQ b_base+16(FP),SI
ANDQ $0xFFFFFFFFFFFFFFF8,R10
JE afterLoop
XCHGQ AX,AX
loopBody:
MOVQ 0(SI)(DI*1),R8
ADDQ AX,R9
RORQ $0x33,AX
XORQ R9,AX
RORQ $0x20,R9
ADDQ $0x8,DI
XORQ R8,CX
ADDQ CX,BX
RORQ $0x30,CX
XORQ BX,CX
ADDQ AX,BX
RORQ $0x2F,AX
ADDQ CX,R9
RORQ $0x2B,CX
XORQ BX,AX
XORQ R9,CX
RORQ $0x20,BX
ADDQ AX,R9
ADDQ CX,BX
RORQ $0x33,AX
RORQ $0x30,CX
XORQ R9,AX
XORQ BX,CX
RORQ $0x20,R9
ADDQ AX,BX
ADDQ CX,R9
RORQ $0x2F,AX
RORQ $0x2B,CX
XORQ BX,AX
RORQ $0x20,BX
XORQ R9,CX
XORQ R8,R9
CMPQ R10,DI
JA loopBody
afterLoop:
SUBQ R10,DX
CMPQ DX,$0x7
JA afterSwitch
// no support for jump tables
CMPQ DX,$0x7
JE sw7
CMPQ DX,$0x6
JE sw6
CMPQ DX,$0x5
JE sw5
CMPQ DX,$0x4
JE sw4
CMPQ DX,$0x3
JE sw3
CMPQ DX,$0x2
JE sw2
CMPQ DX,$0x1
JE sw1
JMP afterSwitch
sw7: MOVBQZX 6(SI)(DI*1),DX
SHLQ $0x30,DX
ORQ DX,R11
sw6: MOVBQZX 0x5(SI)(DI*1),DX
SHLQ $0x28,DX
ORQ DX,R11
sw5: MOVBQZX 0x4(SI)(DI*1),DX
SHLQ $0x20,DX
ORQ DX,R11
sw4: MOVBQZX 0x3(SI)(DI*1),DX
SHLQ $0x18,DX
ORQ DX,R11
sw3: MOVBQZX 0x2(SI)(DI*1),DX
SHLQ $0x10,DX
ORQ DX,R11
sw2: MOVBQZX 0x1(SI)(DI*1),DX
SHLQ $0x8,DX
ORQ DX,R11
sw1: MOVBQZX 0(SI)(DI*1),DX
ORQ DX,R11
afterSwitch:
LEAQ (AX)(R9*1),SI
XORQ R11,CX
RORQ $0x33,AX
ADDQ CX,BX
MOVQ CX,DX
XORQ SI,AX
RORQ $0x30,DX
RORQ $0x20,SI
LEAQ 0(BX)(AX*1),CX
XORQ BX,DX
RORQ $0x2F,AX
ADDQ DX,SI
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
XORQ SI,AX
RORQ $0x30,DX
RORQ $0x20,SI
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
ADDQ DX,SI
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ SI,DX
XORQ R11,SI
XORB $0xFF,CL
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
RORQ $0x30,DX
XORQ SI,AX
XORQ CX,DX
RORQ $0x20,SI
ADDQ AX,CX
ADDQ DX,SI
RORQ $0x2F,AX
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ SI,AX
RORQ $0x20,SI
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
ADDQ DX,SI
RORQ $0x2B,DX
XORQ CX,AX
XORQ SI,DX
RORQ $0x20,CX
ADDQ AX,SI
ADDQ DX,CX
RORQ $0x33,AX
RORQ $0x30,DX
XORQ CX,DX
XORQ SI,AX
RORQ $0x20,SI
ADDQ DX,SI
ADDQ AX,CX
RORQ $0x2F,AX
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ SI,DX
ADDQ AX,SI
RORQ $0x33,AX
ADDQ DX,CX
XORQ SI,AX
RORQ $0x30,DX
XORQ CX,DX
ADDQ AX,CX
RORQ $0x2F,AX
XORQ CX,AX
RORQ $0x2B,DX
RORQ $0x20,CX
XORQ DX,AX
XORQ CX,AX
MOVQ AX,ret+40(FP)
RET

View File

@@ -1,160 +0,0 @@
#include "textflag.h"
#define R10 g
#define ROUND()\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<13,R0,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R1,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R4,R4;\
ADC R7,R5,R5;\
EOR R6<<16,R4,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R5,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<21,R1,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R0,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R4,R4;\
ADC R11,R5,R5;\
EOR R8<<17,R4,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R5,R3;\
EOR R11<<17,R3,R3;\
ADD.S R2,R1,R1;\
ADC R3,R0,R0;\
EOR R2<<13,R1,R8;\
EOR R3>>19,R8,R8;\
EOR R2>>19,R0,R11;\
EOR R3<<13,R11,R11;\
ADD.S R6,R5,R5;\
ADC R7,R4,R4;\
EOR R6<<16,R5,R2;\
EOR R7>>16,R2,R2;\
EOR R6>>16,R4,R3;\
EOR R7<<16,R3,R3;\
ADD.S R2,R0,R0;\
ADC R3,R1,R1;\
EOR R2<<21,R0,R6;\
EOR R3>>11,R6,R6;\
EOR R2>>11,R1,R7;\
EOR R3<<21,R7,R7;\
ADD.S R8,R5,R5;\
ADC R11,R4,R4;\
EOR R8<<17,R5,R2;\
EOR R11>>15,R2,R2;\
EOR R8>>15,R4,R3;\
EOR R11<<17,R3,R3;\
// Hash(k0, k1 uint64, b []byte) uint64
TEXT ·Hash(SB),NOSPLIT,$8-36
MOVW R10,sav-8(SP)
MOVW k0_lo+0(FP),R12
MOVW k0_hi+4(FP),R14
MOVW $0x70736575,R0
MOVW $0x736f6d65,R1
MOVW $0x6e657261,R4
MOVW $0x6c796765,R5
EOR R12,R0,R0
EOR R14,R1,R1
EOR R12,R4,R4
EOR R14,R5,R5
MOVW k1_lo+8(FP),R12
MOVW k1_hi+12(FP),R14
MOVW $0x6e646f6d,R2
MOVW $0x646f7261,R3
MOVW $0x79746573,R6
MOVW $0x74656462,R7
EOR R12,R2,R2
EOR R14,R3,R3
EOR R12,R6,R6
EOR R14,R7,R7
MOVW b+16(FP),R10
MOVW b_len+20(FP),R11
ADD R10,R11,R11
MOVW R11,endb-4(SP)
AND.S $3,R10,R8
BNE hashloopunaligned
hashloop:
MOVW endb-4(SP),R11
SUB R10,R11,R11
SUB.S $8,R11
BLO hashend
MOVM.IA.W (R10),[R12,R14]
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
B hashloop
hashloopunaligned:
MOVW endb-4(SP),R11
SUB R10,R11,R11
SUB.S $8,R11
BLO hashend
MOVB (R10),R12
MOVB 1(R10),R11
ORR R11<<8,R12,R12
MOVB 2(R10),R11
ORR R11<<16,R12,R12
MOVB 3(R10),R11
ORR R11<<24,R12,R12
MOVB 4(R10),R14
MOVB 5(R10),R11
ORR R11<<8,R14,R14
MOVB 6(R10),R11
ORR R11<<16,R14,R14
MOVB 7(R10),R11
ORR R11<<24,R14,R14
ADD $8,R10,R10
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
B hashloopunaligned
hashend:
MOVW $0x0,R12
MOVW $0x0,R14
RSB $0,R11,R11
AND.S $7,R11
BEQ hashlast
MOVW (R10),R12
SLL $3,R11
AND $63,R11
SUB.S $32,R11,R11
BEQ hashlast
BLO hashhi
MOVW R12<<R11,R12
MOVW R12>>R11,R12
B hashlast
hashhi:
ADD $32,R11
MOVW 4(R10),R14
MOVW R14<<R11,R14
MOVW R14>>R11,R14
hashlast:
MOVW b_len+20(FP),R11
ORR R11<<24,R14,R14
EOR R12,R6,R6
EOR R14,R7,R7
ROUND()
EOR R12,R0,R0
EOR R14,R1,R1
EOR $255,R4
ROUND()
ROUND()
EOR R2,R0,R0
EOR R3,R1,R1
EOR R6,R4,R4
EOR R7,R5,R5
EOR R4,R0,R0
EOR R5,R1,R1
MOVW sav-8(SP),R10
MOVW R0,ret_lo+28(FP)
MOVW R1,ret_hi+32(FP)
RET

View File

@@ -1,33 +0,0 @@
// +build arm amd64,!appengine,!gccgo
// Written in 2012 by Dmitry Chestnykh.
//
// To the extent possible under law, the author have dedicated all copyright
// and related and neighboring rights to this software to the public domain
// worldwide. This software is distributed without any warranty.
// http://creativecommons.org/publicdomain/zero/1.0/
// This file contains a function definition for use with assembly implementations of Hash()
package siphash
//go:noescape
// Hash returns the 64-bit SipHash-2-4 of the given byte slice with two 64-bit
// parts of 128-bit key: k0 and k1.
func Hash(k0, k1 uint64, b []byte) uint64
//go:noescape
// Hash128 returns the 128-bit SipHash-2-4 of the given byte slice with two
// 64-bit parts of 128-bit key: k0 and k1.
func Hash128(k0, k1 uint64, b []byte) (uint64, uint64)
//go:noescape
func blocks(d *digest, p []uint8)
//go:noescape
func finalize(d *digest) uint64
//go:noescape
func once(d *digest)

View File

@@ -1,318 +0,0 @@
// Written in 2012-2014 by Dmitry Chestnykh.
//
// To the extent possible under law, the author have dedicated all copyright
// and related and neighboring rights to this software to the public domain
// worldwide. This software is distributed without any warranty.
// http://creativecommons.org/publicdomain/zero/1.0/
// Package siphash implements SipHash-2-4, a fast short-input PRF
// created by Jean-Philippe Aumasson and Daniel J. Bernstein.
package siphash
import "hash"
const (
// BlockSize is the block size of hash algorithm in bytes.
BlockSize = 8
// Size is the size of hash output in bytes.
Size = 8
// Size128 is the size of 128-bit hash output in bytes.
Size128 = 16
)
type digest struct {
v0, v1, v2, v3 uint64 // state
k0, k1 uint64 // two parts of key
x [8]byte // buffer for unprocessed bytes
nx int // number of bytes in buffer x
size int // output size in bytes (8 or 16)
t uint8 // message bytes counter (mod 256)
}
// newDigest returns a new digest with the given output size in bytes (must be 8 or 16).
func newDigest(size int, key []byte) *digest {
if size != Size && size != Size128 {
panic("size must be 8 or 16")
}
d := new(digest)
d.k0 = uint64(key[0]) | uint64(key[1])<<8 | uint64(key[2])<<16 | uint64(key[3])<<24 |
uint64(key[4])<<32 | uint64(key[5])<<40 | uint64(key[6])<<48 | uint64(key[7])<<56
d.k1 = uint64(key[8]) | uint64(key[9])<<8 | uint64(key[10])<<16 | uint64(key[11])<<24 |
uint64(key[12])<<32 | uint64(key[13])<<40 | uint64(key[14])<<48 | uint64(key[15])<<56
d.size = size
d.Reset()
return d
}
// New returns a new hash.Hash64 computing SipHash-2-4 with 16-byte key and 8-byte output.
func New(key []byte) hash.Hash64 {
return newDigest(Size, key)
}
// New128 returns a new hash.Hash computing SipHash-2-4 with 16-byte key and 16-byte output.
//
// Note that 16-byte output is considered experimental by SipHash authors at this time.
func New128(key []byte) hash.Hash {
return newDigest(Size128, key)
}
func (d *digest) Reset() {
d.v0 = d.k0 ^ 0x736f6d6570736575
d.v1 = d.k1 ^ 0x646f72616e646f6d
d.v2 = d.k0 ^ 0x6c7967656e657261
d.v3 = d.k1 ^ 0x7465646279746573
d.t = 0
d.nx = 0
if d.size == Size128 {
d.v1 ^= 0xee
}
}
func (d *digest) Size() int { return d.size }
func (d *digest) BlockSize() int { return BlockSize }
func (d *digest) Write(p []byte) (nn int, err error) {
nn = len(p)
d.t += uint8(nn)
if d.nx > 0 {
n := len(p)
if n > BlockSize-d.nx {
n = BlockSize - d.nx
}
d.nx += copy(d.x[d.nx:], p)
if d.nx == BlockSize {
once(d)
d.nx = 0
}
p = p[n:]
}
if len(p) >= BlockSize {
n := len(p) &^ (BlockSize - 1)
blocks(d, p[:n])
p = p[n:]
}
if len(p) > 0 {
d.nx = copy(d.x[:], p)
}
return
}
func (d *digest) Sum64() uint64 {
for i := d.nx; i < BlockSize-1; i++ {
d.x[i] = 0
}
d.x[7] = d.t
return finalize(d)
}
func (d0 *digest) sum128() (r0, r1 uint64) {
// Make a copy of d0 so that caller can keep writing and summing.
d := *d0
for i := d.nx; i < BlockSize-1; i++ {
d.x[i] = 0
}
d.x[7] = d.t
blocks(&d, d.x[:])
v0, v1, v2, v3 := d.v0, d.v1, d.v2, d.v3
v2 ^= 0xee
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
r0 = v0 ^ v1 ^ v2 ^ v3
v1 ^= 0xdd
// Round 1.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 2.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 3.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
// Round 4.
v0 += v1
v1 = v1<<13 | v1>>(64-13)
v1 ^= v0
v0 = v0<<32 | v0>>(64-32)
v2 += v3
v3 = v3<<16 | v3>>(64-16)
v3 ^= v2
v0 += v3
v3 = v3<<21 | v3>>(64-21)
v3 ^= v0
v2 += v1
v1 = v1<<17 | v1>>(64-17)
v1 ^= v2
v2 = v2<<32 | v2>>(64-32)
r1 = v0 ^ v1 ^ v2 ^ v3
return r0, r1
}
func (d *digest) Sum(in []byte) []byte {
if d.size == Size {
r := d.Sum64()
in = append(in,
byte(r),
byte(r>>8),
byte(r>>16),
byte(r>>24),
byte(r>>32),
byte(r>>40),
byte(r>>48),
byte(r>>56))
} else {
r0, r1 := d.sum128()
in = append(in,
byte(r0),
byte(r0>>8),
byte(r0>>16),
byte(r0>>24),
byte(r0>>32),
byte(r0>>40),
byte(r0>>48),
byte(r0>>56),
byte(r1),
byte(r1>>8),
byte(r1>>16),
byte(r1>>24),
byte(r1>>32),
byte(r1>>40),
byte(r1>>48),
byte(r1>>56))
}
return in
}

View File

@@ -1,591 +0,0 @@
// Written in 2012 by Dmitry Chestnykh.
//
// To the extent possible under law, the author have dedicated all copyright
// and related and neighboring rights to this software to the public domain
// worldwide. This software is distributed without any warranty.
// http://creativecommons.org/publicdomain/zero/1.0/
package siphash
import (
"bytes"
"encoding/binary"
"testing"
)
var zeroKey = make([]byte, 16)
var golden = []struct {
k []byte
m []byte
r uint64
}{
{
[]byte{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f},
[]byte{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e},
0xa129ca6149be45e5,
},
{
zeroKey,
[]byte("Hello world"),
0xc9e8a3021f3822d9,
},
{
zeroKey,
[]byte{}, // zero-length message
0x1e924b9d737700d7,
},
{
zeroKey,
[]byte("12345678123"),
0xf95d77ccdb0649f,
},
{
zeroKey,
make([]byte, 8),
0xe849e8bb6ffe2567,
},
{
zeroKey,
make([]byte, 1535),
0xe74d1c0ab64b2afa,
},
}
// Test vectors from reference implementation.
//
// SipHash-2-4 output with
// k = 00 01 02 ...
// and
// in = (empty string)
// in = 00 (1 byte)
// in = 00 01 (2 bytes)
// in = 00 01 02 (3 bytes)
// ...
// in = 00 01 02 ... 3e (63 bytes)
var goldenRef = [][]byte{
{0x31, 0x0e, 0x0e, 0xdd, 0x47, 0xdb, 0x6f, 0x72},
{0xfd, 0x67, 0xdc, 0x93, 0xc5, 0x39, 0xf8, 0x74},
{0x5a, 0x4f, 0xa9, 0xd9, 0x09, 0x80, 0x6c, 0x0d},
{0x2d, 0x7e, 0xfb, 0xd7, 0x96, 0x66, 0x67, 0x85},
{0xb7, 0x87, 0x71, 0x27, 0xe0, 0x94, 0x27, 0xcf},
{0x8d, 0xa6, 0x99, 0xcd, 0x64, 0x55, 0x76, 0x18},
{0xce, 0xe3, 0xfe, 0x58, 0x6e, 0x46, 0xc9, 0xcb},
{0x37, 0xd1, 0x01, 0x8b, 0xf5, 0x00, 0x02, 0xab},
{0x62, 0x24, 0x93, 0x9a, 0x79, 0xf5, 0xf5, 0x93},
{0xb0, 0xe4, 0xa9, 0x0b, 0xdf, 0x82, 0x00, 0x9e},
{0xf3, 0xb9, 0xdd, 0x94, 0xc5, 0xbb, 0x5d, 0x7a},
{0xa7, 0xad, 0x6b, 0x22, 0x46, 0x2f, 0xb3, 0xf4},
{0xfb, 0xe5, 0x0e, 0x86, 0xbc, 0x8f, 0x1e, 0x75},
{0x90, 0x3d, 0x84, 0xc0, 0x27, 0x56, 0xea, 0x14},
{0xee, 0xf2, 0x7a, 0x8e, 0x90, 0xca, 0x23, 0xf7},
{0xe5, 0x45, 0xbe, 0x49, 0x61, 0xca, 0x29, 0xa1},
{0xdb, 0x9b, 0xc2, 0x57, 0x7f, 0xcc, 0x2a, 0x3f},
{0x94, 0x47, 0xbe, 0x2c, 0xf5, 0xe9, 0x9a, 0x69},
{0x9c, 0xd3, 0x8d, 0x96, 0xf0, 0xb3, 0xc1, 0x4b},
{0xbd, 0x61, 0x79, 0xa7, 0x1d, 0xc9, 0x6d, 0xbb},
{0x98, 0xee, 0xa2, 0x1a, 0xf2, 0x5c, 0xd6, 0xbe},
{0xc7, 0x67, 0x3b, 0x2e, 0xb0, 0xcb, 0xf2, 0xd0},
{0x88, 0x3e, 0xa3, 0xe3, 0x95, 0x67, 0x53, 0x93},
{0xc8, 0xce, 0x5c, 0xcd, 0x8c, 0x03, 0x0c, 0xa8},
{0x94, 0xaf, 0x49, 0xf6, 0xc6, 0x50, 0xad, 0xb8},
{0xea, 0xb8, 0x85, 0x8a, 0xde, 0x92, 0xe1, 0xbc},
{0xf3, 0x15, 0xbb, 0x5b, 0xb8, 0x35, 0xd8, 0x17},
{0xad, 0xcf, 0x6b, 0x07, 0x63, 0x61, 0x2e, 0x2f},
{0xa5, 0xc9, 0x1d, 0xa7, 0xac, 0xaa, 0x4d, 0xde},
{0x71, 0x65, 0x95, 0x87, 0x66, 0x50, 0xa2, 0xa6},
{0x28, 0xef, 0x49, 0x5c, 0x53, 0xa3, 0x87, 0xad},
{0x42, 0xc3, 0x41, 0xd8, 0xfa, 0x92, 0xd8, 0x32},
{0xce, 0x7c, 0xf2, 0x72, 0x2f, 0x51, 0x27, 0x71},
{0xe3, 0x78, 0x59, 0xf9, 0x46, 0x23, 0xf3, 0xa7},
{0x38, 0x12, 0x05, 0xbb, 0x1a, 0xb0, 0xe0, 0x12},
{0xae, 0x97, 0xa1, 0x0f, 0xd4, 0x34, 0xe0, 0x15},
{0xb4, 0xa3, 0x15, 0x08, 0xbe, 0xff, 0x4d, 0x31},
{0x81, 0x39, 0x62, 0x29, 0xf0, 0x90, 0x79, 0x02},
{0x4d, 0x0c, 0xf4, 0x9e, 0xe5, 0xd4, 0xdc, 0xca},
{0x5c, 0x73, 0x33, 0x6a, 0x76, 0xd8, 0xbf, 0x9a},
{0xd0, 0xa7, 0x04, 0x53, 0x6b, 0xa9, 0x3e, 0x0e},
{0x92, 0x59, 0x58, 0xfc, 0xd6, 0x42, 0x0c, 0xad},
{0xa9, 0x15, 0xc2, 0x9b, 0xc8, 0x06, 0x73, 0x18},
{0x95, 0x2b, 0x79, 0xf3, 0xbc, 0x0a, 0xa6, 0xd4},
{0xf2, 0x1d, 0xf2, 0xe4, 0x1d, 0x45, 0x35, 0xf9},
{0x87, 0x57, 0x75, 0x19, 0x04, 0x8f, 0x53, 0xa9},
{0x10, 0xa5, 0x6c, 0xf5, 0xdf, 0xcd, 0x9a, 0xdb},
{0xeb, 0x75, 0x09, 0x5c, 0xcd, 0x98, 0x6c, 0xd0},
{0x51, 0xa9, 0xcb, 0x9e, 0xcb, 0xa3, 0x12, 0xe6},
{0x96, 0xaf, 0xad, 0xfc, 0x2c, 0xe6, 0x66, 0xc7},
{0x72, 0xfe, 0x52, 0x97, 0x5a, 0x43, 0x64, 0xee},
{0x5a, 0x16, 0x45, 0xb2, 0x76, 0xd5, 0x92, 0xa1},
{0xb2, 0x74, 0xcb, 0x8e, 0xbf, 0x87, 0x87, 0x0a},
{0x6f, 0x9b, 0xb4, 0x20, 0x3d, 0xe7, 0xb3, 0x81},
{0xea, 0xec, 0xb2, 0xa3, 0x0b, 0x22, 0xa8, 0x7f},
{0x99, 0x24, 0xa4, 0x3c, 0xc1, 0x31, 0x57, 0x24},
{0xbd, 0x83, 0x8d, 0x3a, 0xaf, 0xbf, 0x8d, 0xb7},
{0x0b, 0x1a, 0x2a, 0x32, 0x65, 0xd5, 0x1a, 0xea},
{0x13, 0x50, 0x79, 0xa3, 0x23, 0x1c, 0xe6, 0x60},
{0x93, 0x2b, 0x28, 0x46, 0xe4, 0xd7, 0x06, 0x66},
{0xe1, 0x91, 0x5f, 0x5c, 0xb1, 0xec, 0xa4, 0x6c},
{0xf3, 0x25, 0x96, 0x5c, 0xa1, 0x6d, 0x62, 0x9f},
{0x57, 0x5f, 0xf2, 0x8e, 0x60, 0x38, 0x1b, 0xe5},
{0x72, 0x45, 0x06, 0xeb, 0x4c, 0x32, 0x8a, 0x95},
}
var goldenRef128 = [][]byte{
{0xa3, 0x81, 0x7f, 0x04, 0xba, 0x25, 0xa8, 0xe6, 0x6d, 0xf6, 0x72, 0x14, 0xc7, 0x55, 0x02, 0x93},
{0xda, 0x87, 0xc1, 0xd8, 0x6b, 0x99, 0xaf, 0x44, 0x34, 0x76, 0x59, 0x11, 0x9b, 0x22, 0xfc, 0x45},
{0x81, 0x77, 0x22, 0x8d, 0xa4, 0xa4, 0x5d, 0xc7, 0xfc, 0xa3, 0x8b, 0xde, 0xf6, 0x0a, 0xff, 0xe4},
{0x9c, 0x70, 0xb6, 0x0c, 0x52, 0x67, 0xa9, 0x4e, 0x5f, 0x33, 0xb6, 0xb0, 0x29, 0x85, 0xed, 0x51},
{0xf8, 0x81, 0x64, 0xc1, 0x2d, 0x9c, 0x8f, 0xaf, 0x7d, 0x0f, 0x6e, 0x7c, 0x7b, 0xcd, 0x55, 0x79},
{0x13, 0x68, 0x87, 0x59, 0x80, 0x77, 0x6f, 0x88, 0x54, 0x52, 0x7a, 0x07, 0x69, 0x0e, 0x96, 0x27},
{0x14, 0xee, 0xca, 0x33, 0x8b, 0x20, 0x86, 0x13, 0x48, 0x5e, 0xa0, 0x30, 0x8f, 0xd7, 0xa1, 0x5e},
{0xa1, 0xf1, 0xeb, 0xbe, 0xd8, 0xdb, 0xc1, 0x53, 0xc0, 0xb8, 0x4a, 0xa6, 0x1f, 0xf0, 0x82, 0x39},
{0x3b, 0x62, 0xa9, 0xba, 0x62, 0x58, 0xf5, 0x61, 0x0f, 0x83, 0xe2, 0x64, 0xf3, 0x14, 0x97, 0xb4},
{0x26, 0x44, 0x99, 0x06, 0x0a, 0xd9, 0xba, 0xab, 0xc4, 0x7f, 0x8b, 0x02, 0xbb, 0x6d, 0x71, 0xed},
{0x00, 0x11, 0x0d, 0xc3, 0x78, 0x14, 0x69, 0x56, 0xc9, 0x54, 0x47, 0xd3, 0xf3, 0xd0, 0xfb, 0xba},
{0x01, 0x51, 0xc5, 0x68, 0x38, 0x6b, 0x66, 0x77, 0xa2, 0xb4, 0xdc, 0x6f, 0x81, 0xe5, 0xdc, 0x18},
{0xd6, 0x26, 0xb2, 0x66, 0x90, 0x5e, 0xf3, 0x58, 0x82, 0x63, 0x4d, 0xf6, 0x85, 0x32, 0xc1, 0x25},
{0x98, 0x69, 0xe2, 0x47, 0xe9, 0xc0, 0x8b, 0x10, 0xd0, 0x29, 0x93, 0x4f, 0xc4, 0xb9, 0x52, 0xf7},
{0x31, 0xfc, 0xef, 0xac, 0x66, 0xd7, 0xde, 0x9c, 0x7e, 0xc7, 0x48, 0x5f, 0xe4, 0x49, 0x49, 0x02},
{0x54, 0x93, 0xe9, 0x99, 0x33, 0xb0, 0xa8, 0x11, 0x7e, 0x08, 0xec, 0x0f, 0x97, 0xcf, 0xc3, 0xd9},
{0x6e, 0xe2, 0xa4, 0xca, 0x67, 0xb0, 0x54, 0xbb, 0xfd, 0x33, 0x15, 0xbf, 0x85, 0x23, 0x05, 0x77},
{0x47, 0x3d, 0x06, 0xe8, 0x73, 0x8d, 0xb8, 0x98, 0x54, 0xc0, 0x66, 0xc4, 0x7a, 0xe4, 0x77, 0x40},
{0xa4, 0x26, 0xe5, 0xe4, 0x23, 0xbf, 0x48, 0x85, 0x29, 0x4d, 0xa4, 0x81, 0xfe, 0xae, 0xf7, 0x23},
{0x78, 0x01, 0x77, 0x31, 0xcf, 0x65, 0xfa, 0xb0, 0x74, 0xd5, 0x20, 0x89, 0x52, 0x51, 0x2e, 0xb1},
{0x9e, 0x25, 0xfc, 0x83, 0x3f, 0x22, 0x90, 0x73, 0x3e, 0x93, 0x44, 0xa5, 0xe8, 0x38, 0x39, 0xeb},
{0x56, 0x8e, 0x49, 0x5a, 0xbe, 0x52, 0x5a, 0x21, 0x8a, 0x22, 0x14, 0xcd, 0x3e, 0x07, 0x1d, 0x12},
{0x4a, 0x29, 0xb5, 0x45, 0x52, 0xd1, 0x6b, 0x9a, 0x46, 0x9c, 0x10, 0x52, 0x8e, 0xff, 0x0a, 0xae},
{0xc9, 0xd1, 0x84, 0xdd, 0xd5, 0xa9, 0xf5, 0xe0, 0xcf, 0x8c, 0xe2, 0x9a, 0x9a, 0xbf, 0x69, 0x1c},
{0x2d, 0xb4, 0x79, 0xae, 0x78, 0xbd, 0x50, 0xd8, 0x88, 0x2a, 0x8a, 0x17, 0x8a, 0x61, 0x32, 0xad},
{0x8e, 0xce, 0x5f, 0x04, 0x2d, 0x5e, 0x44, 0x7b, 0x50, 0x51, 0xb9, 0xea, 0xcb, 0x8d, 0x8f, 0x6f},
{0x9c, 0x0b, 0x53, 0xb4, 0xb3, 0xc3, 0x07, 0xe8, 0x7e, 0xae, 0xe0, 0x86, 0x78, 0x14, 0x1f, 0x66},
{0xab, 0xf2, 0x48, 0xaf, 0x69, 0xa6, 0xea, 0xe4, 0xbf, 0xd3, 0xeb, 0x2f, 0x12, 0x9e, 0xeb, 0x94},
{0x06, 0x64, 0xda, 0x16, 0x68, 0x57, 0x4b, 0x88, 0xb9, 0x35, 0xf3, 0x02, 0x73, 0x58, 0xae, 0xf4},
{0xaa, 0x4b, 0x9d, 0xc4, 0xbf, 0x33, 0x7d, 0xe9, 0x0c, 0xd4, 0xfd, 0x3c, 0x46, 0x7c, 0x6a, 0xb7},
{0xea, 0x5c, 0x7f, 0x47, 0x1f, 0xaf, 0x6b, 0xde, 0x2b, 0x1a, 0xd7, 0xd4, 0x68, 0x6d, 0x22, 0x87},
{0x29, 0x39, 0xb0, 0x18, 0x32, 0x23, 0xfa, 0xfc, 0x17, 0x23, 0xde, 0x4f, 0x52, 0xc4, 0x3d, 0x35},
{0x7c, 0x39, 0x56, 0xca, 0x5e, 0xea, 0xfc, 0x3e, 0x36, 0x3e, 0x9d, 0x55, 0x65, 0x46, 0xeb, 0x68},
{0x77, 0xc6, 0x07, 0x71, 0x46, 0xf0, 0x1c, 0x32, 0xb6, 0xb6, 0x9d, 0x5f, 0x4e, 0xa9, 0xff, 0xcf},
{0x37, 0xa6, 0x98, 0x6c, 0xb8, 0x84, 0x7e, 0xdf, 0x09, 0x25, 0xf0, 0xf1, 0x30, 0x9b, 0x54, 0xde},
{0xa7, 0x05, 0xf0, 0xe6, 0x9d, 0xa9, 0xa8, 0xf9, 0x07, 0x24, 0x1a, 0x2e, 0x92, 0x3c, 0x8c, 0xc8},
{0x3d, 0xc4, 0x7d, 0x1f, 0x29, 0xc4, 0x48, 0x46, 0x1e, 0x9e, 0x76, 0xed, 0x90, 0x4f, 0x67, 0x11},
{0x0d, 0x62, 0xbf, 0x01, 0xe6, 0xfc, 0x0e, 0x1a, 0x0d, 0x3c, 0x47, 0x51, 0xc5, 0xd3, 0x69, 0x2b},
{0x8c, 0x03, 0x46, 0x8b, 0xca, 0x7c, 0x66, 0x9e, 0xe4, 0xfd, 0x5e, 0x08, 0x4b, 0xbe, 0xe7, 0xb5},
{0x52, 0x8a, 0x5b, 0xb9, 0x3b, 0xaf, 0x2c, 0x9c, 0x44, 0x73, 0xcc, 0xe5, 0xd0, 0xd2, 0x2b, 0xd9},
{0xdf, 0x6a, 0x30, 0x1e, 0x95, 0xc9, 0x5d, 0xad, 0x97, 0xae, 0x0c, 0xc8, 0xc6, 0x91, 0x3b, 0xd8},
{0x80, 0x11, 0x89, 0x90, 0x2c, 0x85, 0x7f, 0x39, 0xe7, 0x35, 0x91, 0x28, 0x5e, 0x70, 0xb6, 0xdb},
{0xe6, 0x17, 0x34, 0x6a, 0xc9, 0xc2, 0x31, 0xbb, 0x36, 0x50, 0xae, 0x34, 0xcc, 0xca, 0x0c, 0x5b},
{0x27, 0xd9, 0x34, 0x37, 0xef, 0xb7, 0x21, 0xaa, 0x40, 0x18, 0x21, 0xdc, 0xec, 0x5a, 0xdf, 0x89},
{0x89, 0x23, 0x7d, 0x9d, 0xed, 0x9c, 0x5e, 0x78, 0xd8, 0xb1, 0xc9, 0xb1, 0x66, 0xcc, 0x73, 0x42},
{0x4a, 0x6d, 0x80, 0x91, 0xbf, 0x5e, 0x7d, 0x65, 0x11, 0x89, 0xfa, 0x94, 0xa2, 0x50, 0xb1, 0x4c},
{0x0e, 0x33, 0xf9, 0x60, 0x55, 0xe7, 0xae, 0x89, 0x3f, 0xfc, 0x0e, 0x3d, 0xcf, 0x49, 0x29, 0x02},
{0xe6, 0x1c, 0x43, 0x2b, 0x72, 0x0b, 0x19, 0xd1, 0x8e, 0xc8, 0xd8, 0x4b, 0xdc, 0x63, 0x15, 0x1b},
{0xf7, 0xe5, 0xae, 0xf5, 0x49, 0xf7, 0x82, 0xcf, 0x37, 0x90, 0x55, 0xa6, 0x08, 0x26, 0x9b, 0x16},
{0x43, 0x8d, 0x03, 0x0f, 0xd0, 0xb7, 0xa5, 0x4f, 0xa8, 0x37, 0xf2, 0xad, 0x20, 0x1a, 0x64, 0x03},
{0xa5, 0x90, 0xd3, 0xee, 0x4f, 0xbf, 0x04, 0xe3, 0x24, 0x7e, 0x0d, 0x27, 0xf2, 0x86, 0x42, 0x3f},
{0x5f, 0xe2, 0xc1, 0xa1, 0x72, 0xfe, 0x93, 0xc4, 0xb1, 0x5c, 0xd3, 0x7c, 0xae, 0xf9, 0xf5, 0x38},
{0x2c, 0x97, 0x32, 0x5c, 0xbd, 0x06, 0xb3, 0x6e, 0xb2, 0x13, 0x3d, 0xd0, 0x8b, 0x3a, 0x01, 0x7c},
{0x92, 0xc8, 0x14, 0x22, 0x7a, 0x6b, 0xca, 0x94, 0x9f, 0xf0, 0x65, 0x9f, 0x00, 0x2a, 0xd3, 0x9e},
{0xdc, 0xe8, 0x50, 0x11, 0x0b, 0xd8, 0x32, 0x8c, 0xfb, 0xd5, 0x08, 0x41, 0xd6, 0x91, 0x1d, 0x87},
{0x67, 0xf1, 0x49, 0x84, 0xc7, 0xda, 0x79, 0x12, 0x48, 0xe3, 0x2b, 0xb5, 0x92, 0x25, 0x83, 0xda},
{0x19, 0x38, 0xf2, 0xcf, 0x72, 0xd5, 0x4e, 0xe9, 0x7e, 0x94, 0x16, 0x6f, 0xa9, 0x1d, 0x2a, 0x36},
{0x74, 0x48, 0x1e, 0x96, 0x46, 0xed, 0x49, 0xfe, 0x0f, 0x62, 0x24, 0x30, 0x16, 0x04, 0x69, 0x8e},
{0x57, 0xfc, 0xa5, 0xde, 0x98, 0xa9, 0xd6, 0xd8, 0x00, 0x64, 0x38, 0xd0, 0x58, 0x3d, 0x8a, 0x1d},
{0x9f, 0xec, 0xde, 0x1c, 0xef, 0xdc, 0x1c, 0xbe, 0xd4, 0x76, 0x36, 0x74, 0xd9, 0x57, 0x53, 0x59},
{0xe3, 0x04, 0x0c, 0x00, 0xeb, 0x28, 0xf1, 0x53, 0x66, 0xca, 0x73, 0xcb, 0xd8, 0x72, 0xe7, 0x40},
{0x76, 0x97, 0x00, 0x9a, 0x6a, 0x83, 0x1d, 0xfe, 0xcc, 0xa9, 0x1c, 0x59, 0x93, 0x67, 0x0f, 0x7a},
{0x58, 0x53, 0x54, 0x23, 0x21, 0xf5, 0x67, 0xa0, 0x05, 0xd5, 0x47, 0xa4, 0xf0, 0x47, 0x59, 0xbd},
{0x51, 0x50, 0xd1, 0x77, 0x2f, 0x50, 0x83, 0x4a, 0x50, 0x3e, 0x06, 0x9a, 0x97, 0x3f, 0xbd, 0x7c},
}
func TestSum64(t *testing.T) {
for i, v := range golden {
h := New(v.k)
h.Write(v.m)
if sum := h.Sum64(); sum != v.r {
t.Errorf(`%d: expected "%x", got "%x"`, i, v.r, sum)
}
}
}
func TestSum(t *testing.T) {
var r [8]byte
for i, v := range golden {
binary.LittleEndian.PutUint64(r[:], v.r)
h := New(v.k)
h.Write(v.m)
if sum := h.Sum(nil); !bytes.Equal(sum, r[:]) {
t.Errorf(`%d: expected "%x", got "%x"`, i, r, sum)
}
}
var k [16]byte
var in [64]byte
for i := range k {
k[i] = byte(i)
}
for i := 0; i < 64; i++ {
in[i] = byte(i)
h := New(k[:])
h.Write(in[:i])
if sum := h.Sum(nil); !bytes.Equal(sum, goldenRef[i]) {
t.Errorf(`%d: expected "%x", got "%x"`, i, goldenRef[i], sum)
}
}
}
func TestSumUnaligned(t *testing.T) {
const align = 8
var k [16]byte
var in [64 + align]byte
for i := range k {
k[i] = byte(i)
}
for a := 1; a < align; a++ {
for i := 0; i < 64; i++ {
in[a+i] = byte(i)
h := New(k[:])
h.Write(in[a : a+i])
if sum := h.Sum(nil); !bytes.Equal(sum, goldenRef[i]) {
t.Errorf(`%d: expected "%x", got "%x"`, i, goldenRef[i], sum)
}
}
}
}
func TestSum128(t *testing.T) {
var k [16]byte
var in [64]byte
for i := range k {
k[i] = byte(i)
}
for i := 0; i < 64; i++ {
in[i] = byte(i)
h := New128(k[:])
h.Write(in[:i])
if sum := h.Sum(nil); !bytes.Equal(sum, goldenRef128[i]) {
t.Errorf(`%d: expected "%x", got "%x"`, i, goldenRef128[i], sum)
}
}
}
func TestHash(t *testing.T) {
var k0, k1 uint64
for i, v := range golden {
k0 = binary.LittleEndian.Uint64(v.k[0:8])
k1 = binary.LittleEndian.Uint64(v.k[8:16])
if sum := Hash(k0, k1, v.m); sum != v.r {
t.Errorf(`%d: expected "%x", got "%x"`, i, v.r, sum)
}
}
var k [16]byte
var in [64]byte
for i := range k {
k[i] = byte(i)
}
k0 = binary.LittleEndian.Uint64(k[0:8])
k1 = binary.LittleEndian.Uint64(k[8:16])
for i := 0; i < 64; i++ {
in[i] = byte(i)
ref := binary.LittleEndian.Uint64(goldenRef[i])
if sum := Hash(k0, k1, in[:i]); sum != ref {
t.Errorf(`%d: expected "%x", got "%x"`, i, ref, sum)
}
}
}
func TestHashUnaligned(t *testing.T) {
const align = 8
var k0, k1 uint64
var k [16]byte
var in [64 + align]byte
for i := range k {
k[i] = byte(i)
}
k0 = binary.LittleEndian.Uint64(k[0:8])
k1 = binary.LittleEndian.Uint64(k[8:16])
for a := 1; a < align; a++ {
for i := 0; i < 64; i++ {
in[a+i] = byte(i)
ref := binary.LittleEndian.Uint64(goldenRef[i])
if sum := Hash(k0, k1, in[a:a+i]); sum != ref {
t.Errorf(`%d: expected "%x", got "%x"`, i, ref, sum)
}
}
}
}
func TestHash128(t *testing.T) {
var k0, k1 uint64
var k [16]byte
var in [64]byte
for i := range k {
k[i] = byte(i)
}
k0 = binary.LittleEndian.Uint64(k[0:8])
k1 = binary.LittleEndian.Uint64(k[8:16])
for i := 0; i < 64; i++ {
in[i] = byte(i)
ref0 := binary.LittleEndian.Uint64(goldenRef128[i][0:])
ref1 := binary.LittleEndian.Uint64(goldenRef128[i][8:])
if sum0, sum1 := Hash128(k0, k1, in[:i]); sum0 != ref0 || sum1 != ref1 {
t.Errorf(`%d: expected "%x, %x", got "%x, %x"`, i, ref0, ref1, sum0, sum1)
}
}
}
var (
key = zeroKey
key0, key1 uint64
bench = New(key)
bench128 = New128(key)
buf = make([]byte, 8<<10)
)
func BenchmarkHash8(b *testing.B) {
b.SetBytes(8)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:8])
}
}
func BenchmarkHash16(b *testing.B) {
b.SetBytes(16)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:16])
}
}
func BenchmarkHash40(b *testing.B) {
b.SetBytes(40)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:40])
}
}
func BenchmarkHash64(b *testing.B) {
b.SetBytes(64)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:64])
}
}
func BenchmarkHash128(b *testing.B) {
b.SetBytes(128)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:128])
}
}
func BenchmarkHash1K(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[:1024])
}
}
func BenchmarkHash1Kunaligned(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf[1:1025])
}
}
func BenchmarkHash8K(b *testing.B) {
b.SetBytes(int64(len(buf)))
for i := 0; i < b.N; i++ {
Hash(key0, key1, buf)
}
}
func BenchmarkHash128_8(b *testing.B) {
b.SetBytes(8)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:8])
}
}
func BenchmarkHash128_16(b *testing.B) {
b.SetBytes(16)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:16])
}
}
func BenchmarkHash128_40(b *testing.B) {
b.SetBytes(40)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:40])
}
}
func BenchmarkHash128_64(b *testing.B) {
b.SetBytes(64)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:64])
}
}
func BenchmarkHash128_128(b *testing.B) {
b.SetBytes(128)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:128])
}
}
func BenchmarkHash128_1K(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf[:1024])
}
}
func BenchmarkHash128_8K(b *testing.B) {
b.SetBytes(int64(len(buf)))
for i := 0; i < b.N; i++ {
Hash128(key0, key1, buf)
}
}
func BenchmarkFull8(b *testing.B) {
b.SetBytes(8)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:8])
bench.Sum64()
}
}
func BenchmarkFull16(b *testing.B) {
b.SetBytes(16)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:16])
bench.Sum64()
}
}
func BenchmarkFull40(b *testing.B) {
b.SetBytes(24)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:16])
bench.Sum64()
}
}
func BenchmarkFull64(b *testing.B) {
b.SetBytes(64)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:64])
bench.Sum64()
}
}
func BenchmarkFull128(b *testing.B) {
b.SetBytes(128)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:64])
bench.Sum64()
}
}
func BenchmarkFull1K(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[:1024])
bench.Sum64()
}
}
func BenchmarkFull1Kunaligned(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf[1:1025])
bench.Sum64()
}
}
func BenchmarkFull8K(b *testing.B) {
b.SetBytes(int64(len(buf)))
for i := 0; i < b.N; i++ {
bench.Reset()
bench.Write(buf)
bench.Sum64()
}
}
func BenchmarkFull128_8(b *testing.B) {
b.SetBytes(8)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:8])
bench128.Sum(nil)
}
}
func BenchmarkFull128_16(b *testing.B) {
b.SetBytes(16)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:16])
bench128.Sum(nil)
}
}
func BenchmarkFull128_40(b *testing.B) {
b.SetBytes(24)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:16])
bench128.Sum(nil)
}
}
func BenchmarkFull128_64(b *testing.B) {
b.SetBytes(64)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:64])
bench128.Sum(nil)
}
}
func BenchmarkFull128_128(b *testing.B) {
b.SetBytes(128)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:64])
bench128.Sum(nil)
}
}
func BenchmarkFull128_1K(b *testing.B) {
b.SetBytes(1024)
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf[:1024])
bench128.Sum(nil)
}
}
func BenchmarkFull128_8K(b *testing.B) {
b.SetBytes(int64(len(buf)))
for i := 0; i < b.N; i++ {
bench128.Reset()
bench128.Write(buf)
bench128.Sum(nil)
}
}