From e1d124f93f27e106dd7368e6c8200a14b1fbf4c3 Mon Sep 17 00:00:00 2001 From: Alan Parra Date: Fri, 14 Jul 2017 12:18:25 +0100 Subject: [PATCH] Quota increase factor and other minor adjustments (#737) * Add token increase factor * Get tokens correctly for batch requests * Don't impose quota on Admin requests * Add quota_increase_factor flag * Add quota_dry_run mode --- log/sequencer.go | 49 +++++++--- log/sequencer_test.go | 125 +++++++++++++++++++++++-- server/interceptor/interceptor.go | 68 ++++++++++---- server/interceptor/interceptor_test.go | 96 +++++++++++++++---- server/trillian_log_server/main.go | 2 + server/trillian_log_signer/main.go | 11 ++- server/trillian_map_server/main.go | 2 + 7 files changed, 295 insertions(+), 58 deletions(-) diff --git a/log/sequencer.go b/log/sequencer.go index 8ee58b20c4..1a61bfae0d 100644 --- a/log/sequencer.go +++ b/log/sequencer.go @@ -51,8 +51,26 @@ var ( seqStoreRootLatency monitoring.Histogram seqCommitLatency monitoring.Histogram seqCounter monitoring.Counter + + // QuotaIncreaseFactor is the multiplier used for the number of tokens added back to + // sequencing-based quotas. The resulting PutTokens call is equivalent to + // "PutTokens(_, numLeaves * QuotaIncreaseFactor, _)". + // A factor >1 adds resilience to token leakage, on the risk of a system that's overly + // optimistic in face of true token shortages. The higher the factor, the higher the quota + // "optimism" is. A factor that's too high (say, >1.5) is likely a sign that the quota + // configuration should be changed instead. + // A factor <1 WILL lead to token shortages, therefore it'll be normalized to 1. + QuotaIncreaseFactor = 1.1 ) +func quotaIncreaseFactor() float64 { + if QuotaIncreaseFactor < 1 { + QuotaIncreaseFactor = 1 + return 1 + } + return QuotaIncreaseFactor +} + func createMetrics(mf monitoring.MetricFactory) { if mf == nil { mf = monitoring.InertMetricFactory{} @@ -256,7 +274,8 @@ func (s Sequencer) SequenceBatch(ctx context.Context, logID int64, limit int, gu // There might be no work to be done. But we possibly still need to create an signed root if the // current one is too old. If there's work to be done then we'll be creating a root anyway. - if len(leaves) == 0 { + numLeaves := len(leaves) + if numLeaves == 0 { nowNanos := s.timeSource.Now().UnixNano() interval := time.Duration(nowNanos - currentRoot.TimestampNanos) if maxRootDurationInterval == 0 || interval < maxRootDurationInterval { @@ -292,8 +311,8 @@ func (s Sequencer) SequenceBatch(ctx context.Context, logID int64, limit int, gu stageStart = s.timeSource.Now() // We should still have the same number of leaves - if want, got := len(leaves), len(sequencedLeaves); want != got { - return 0, fmt.Errorf("%v: wanted: %v leaves after sequencing but we got: %v", logID, want, got) + if want := len(sequencedLeaves); numLeaves != want { + return 0, fmt.Errorf("%v: wanted: %v leaves after sequencing but we got: %v", logID, want, numLeaves) } // Write the new sequence numbers to the leaves in the DB @@ -360,18 +379,22 @@ func (s Sequencer) SequenceBatch(ctx context.Context, logID int64, limit int, gu // TODO(codingllama): Consider adding a source-aware replenish method // (eg, qm.Replenish(ctx, tokens, specs, quota.SequencerSource)), so there's no ambiguity as to // where the tokens come from. - if err := s.qm.PutTokens(ctx, len(leaves), []quota.Spec{ - {Group: quota.Tree, Kind: quota.Read, TreeID: logID}, - {Group: quota.Tree, Kind: quota.Write, TreeID: logID}, - {Group: quota.Global, Kind: quota.Read}, - {Group: quota.Global, Kind: quota.Write}, - }); err != nil { - glog.Warningf("Failed to replenish tokens for tree %v: %v", logID, err) + if numLeaves > 0 { + tokens := int(float64(numLeaves) * quotaIncreaseFactor()) + glog.V(2).Infof("Replenishing %v tokens for tree %v (numLeaves = %v)", tokens, logID, leaves) + if err := s.qm.PutTokens(ctx, tokens, []quota.Spec{ + {Group: quota.Tree, Kind: quota.Read, TreeID: logID}, + {Group: quota.Tree, Kind: quota.Write, TreeID: logID}, + {Group: quota.Global, Kind: quota.Read}, + {Group: quota.Global, Kind: quota.Write}, + }); err != nil { + glog.Warningf("Failed to replenish %v tokens for tree %v: %v", tokens, logID, err) + } } - seqCounter.Add(float64(len(leaves)), label) - glog.Infof("%v: sequenced %v leaves, size %v, tree-revision %v", logID, len(leaves), newLogRoot.TreeSize, newLogRoot.TreeRevision) - return len(leaves), nil + seqCounter.Add(float64(numLeaves), label) + glog.Infof("%v: sequenced %v leaves, size %v, tree-revision %v", logID, numLeaves, newLogRoot.TreeSize, newLogRoot.TreeRevision) + return numLeaves, nil } // SignRoot wraps up all the operations for creating a new log signed root. diff --git a/log/sequencer_test.go b/log/sequencer_test.go index aa71aa5332..376a15b435 100644 --- a/log/sequencer_test.go +++ b/log/sequencer_test.go @@ -326,7 +326,6 @@ func TestSequenceBatch(t *testing.T) { dequeuedLeaves: noLeaves, skipStoreSignedRoot: true, }, - wantCount: 0, }, { desc: "nothing-queued-within-max", @@ -339,7 +338,6 @@ func TestSequenceBatch(t *testing.T) { skipStoreSignedRoot: true, }, maxRootDuration: 15 * time.Millisecond, - wantCount: 0, }, { desc: "nothing-queued-after-max", @@ -356,7 +354,6 @@ func TestSequenceBatch(t *testing.T) { storeSignedRoot: &newRoot16, }, maxRootDuration: 9 * time.Millisecond, - wantCount: 0, }, { desc: "nothing-queued-on-max", @@ -373,7 +370,6 @@ func TestSequenceBatch(t *testing.T) { storeSignedRoot: &newRoot16, }, maxRootDuration: 10 * time.Millisecond, - wantCount: 0, }, { // Tests that the guard interval is being passed to storage correctly. @@ -514,12 +510,13 @@ func TestSequenceBatch(t *testing.T) { for _, test := range tests { func() { ctrl := gomock.NewController(t) + defer ctrl.Finish() + + qm := quota.NewMockManager(ctrl) + test.params.qm = qm if test.wantCount > 0 { - qm := quota.NewMockManager(ctrl) qm.EXPECT().PutTokens(gomock.Any(), test.wantCount, specs).Return(nil) - test.params.qm = qm } - defer ctrl.Finish() c, ctx := createTestContext(ctrl, test.params) got, err := c.sequencer.SequenceBatch(ctx, test.params.logID, 1, test.guardWindow, test.maxRootDuration) @@ -538,6 +535,120 @@ func TestSequenceBatch(t *testing.T) { } } +func TestSequenceBatch_PutTokens(t *testing.T) { + cryptoSigner, err := newSignerWithFixedSig(expectedSignedRoot.Signature) + if err != nil { + t.Fatalf("Failed to create test signer (%v)", err) + } + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Needed to create a signer + hasher := rfc6962.DefaultHasher + ts := util.NewFakeTimeSource(fakeTimeForTest) + signer := crypto.NewSHA256Signer(cryptoSigner) + + // Needed for SequenceBatch calls + const treeID int64 = 1234 + const limit = 1000 + const guardWindow = 10 * time.Second + const maxRootDuration = 1 * time.Hour + + // Expected PutTokens specs + specs := []quota.Spec{ + {Group: quota.Tree, Kind: quota.Read, TreeID: treeID}, + {Group: quota.Tree, Kind: quota.Write, TreeID: treeID}, + {Group: quota.Global, Kind: quota.Read}, + {Group: quota.Global, Kind: quota.Write}, + } + + oneHundredLeaves := make([]*trillian.LogLeaf, 100) + for i := range oneHundredLeaves { + oneHundredLeaves[i] = &trillian.LogLeaf{ + LeafValue: []byte(fmt.Sprintf("leaf-%v", i)), + } + } + + tests := []struct { + desc string + leaves []*trillian.LogLeaf + quotaFactor float64 + wantLeaves, wantTokens int + }{ + {desc: "noLeaves"}, + { + desc: "singleLeaf", + leaves: []*trillian.LogLeaf{getLeaf42()}, + wantLeaves: 1, + wantTokens: 1, + }, + { + desc: "badFactor", + leaves: oneHundredLeaves, + quotaFactor: 0.7, // factor <1 is normalized to 1 + wantLeaves: 100, + wantTokens: 100, + }, + { + desc: "factorOne", + leaves: oneHundredLeaves, + quotaFactor: 1, + wantLeaves: 100, + wantTokens: 100, + }, + { + desc: "10%-factor", + leaves: oneHundredLeaves, + quotaFactor: 1.1, + wantLeaves: 100, + wantTokens: 110, + }, + } + + any := gomock.Any() + ctx := context.Background() + for _, test := range tests { + func() { + if test.quotaFactor != 0 { + defer func(qf float64) { + QuotaIncreaseFactor = qf + }(QuotaIncreaseFactor) + QuotaIncreaseFactor = test.quotaFactor + } + + // Correctness of operation is tested elsewhere. The focus here is the interaction + // between Sequencer and quota.Manager. + logTX := storage.NewMockLogTreeTX(ctrl) + logTX.EXPECT().DequeueLeaves(any, any, any).Return(test.leaves, nil) + logTX.EXPECT().LatestSignedLogRoot(any).Return(testRoot16, nil) + logTX.EXPECT().WriteRevision().AnyTimes().Return(testRoot16.TreeRevision + 1) + logTX.EXPECT().UpdateSequencedLeaves(any, any).AnyTimes().Return(nil) + logTX.EXPECT().SetMerkleNodes(any, any).AnyTimes().Return(nil) + logTX.EXPECT().StoreSignedLogRoot(any, any).AnyTimes().Return(nil) + logTX.EXPECT().Commit().Return(nil) + logTX.EXPECT().Close().Return(nil) + logStorage := storage.NewMockLogStorage(ctrl) + logStorage.EXPECT().BeginForTree(any, any).Return(logTX, nil) + + qm := quota.NewMockManager(ctrl) + if test.wantTokens > 0 { + qm.EXPECT().PutTokens(any, test.wantTokens, specs) + } + + sequencer := NewSequencer(hasher, ts, logStorage, signer, nil /* mf */, qm) + leaves, err := sequencer.SequenceBatch(ctx, treeID, limit, guardWindow, maxRootDuration) + if err != nil { + t.Errorf("%v: SequenceBatch() returned err = %v", test.desc, err) + return + } + if leaves != test.wantLeaves { + t.Errorf("%v: SequenceBatch() returned %v leaves, want = %v", test.desc, leaves, test.wantLeaves) + } + }() + } +} + func TestSignRoot(t *testing.T) { signer0, err := newSignerWithFixedSig(expectedSignedRoot0.Signature) if err != nil { diff --git a/server/interceptor/interceptor.go b/server/interceptor/interceptor.go index 1d35cfafde..f5aecf67f2 100644 --- a/server/interceptor/interceptor.go +++ b/server/interceptor/interceptor.go @@ -18,6 +18,7 @@ package interceptor import ( "fmt" + "github.com/golang/glog" "github.com/google/trillian" "github.com/google/trillian/quota" "github.com/google/trillian/server/errors" @@ -36,6 +37,10 @@ import ( type TrillianInterceptor struct { Admin storage.AdminStorage QuotaManager quota.Manager + + // QuotaDryRun controls whether lack of tokens actually blocks requests (if set to true, no + // requests are blocked by lack of tokens). + QuotaDryRun bool } // UnaryInterceptor executes the TrillianInterceptor logic for unary RPCs. @@ -59,8 +64,13 @@ func (i *TrillianInterceptor) UnaryInterceptor(ctx context.Context, req interfac // TODO(codingllama): Add auth interception } - if err := i.QuotaManager.GetTokens(ctx, 1 /* numTokens */, rpcInfo.specs); err != nil { - return nil, status.Errorf(codes.ResourceExhausted, "quota exhausted: %v", err) + if len(rpcInfo.specs) > 0 && rpcInfo.tokens > 0 { + if err := i.QuotaManager.GetTokens(ctx, rpcInfo.tokens, rpcInfo.specs); err != nil { + if !i.QuotaDryRun { + return nil, status.Errorf(codes.ResourceExhausted, "quota exhausted: %v", err) + } + glog.Warningf("(QuotaDryRun) Request %+v not denied due to dry run mode: %v", req, err) + } } return handler(ctx, req) @@ -77,6 +87,9 @@ type rpcInfo struct { // specs contains the quota specifications for this RPC. specs []quota.Spec + + // tokens is number of quota tokens consumed by this request. + tokens int } // getRPCInfo returns the rpcInfo for the given request, or an error if the request is not mapped. @@ -113,12 +126,18 @@ func getRPCInfo(req interface{}, quotaUser string) (*rpcInfo, error) { kind = quota.Write } var specs []quota.Spec - if treeID == 0 { + switch { + case treeType == trillian.TreeType_UNKNOWN_TREE_TYPE: + // Don't impose quota on Admin requests. + // Sequencing-based replenishment is not tied in any way to Admin, so charging tokens for it + // leads to direct leakage. + // Admin is meant to be internal and unlikely to be a source of high QPS, in any case. + case treeID == 0: specs = []quota.Spec{ {Group: quota.User, Kind: kind, User: quotaUser}, {Group: quota.Global, Kind: kind}, } - } else { + default: specs = []quota.Spec{ {Group: quota.User, Kind: kind, User: quotaUser}, {Group: quota.Tree, Kind: kind, TreeID: treeID}, @@ -126,29 +145,38 @@ func getRPCInfo(req interface{}, quotaUser string) (*rpcInfo, error) { } } + tokens := 1 + switch req := req.(type) { + case logLeavesRequest: + tokens = len(req.GetLeaves()) + case mapLeavesRequest: + tokens = len(req.GetLeaves()) + } + return &rpcInfo{ treeID: treeID, opts: trees.GetOpts{TreeType: treeType, Readonly: readonly}, specs: specs, + tokens: tokens, }, nil } func getRequestInfo(req interface{}) (trillian.TreeType, bool, error) { - if ok, readonly := getAdminRequestInfo(req); ok { + if readonly, ok := getAdminRequestInfo(req); ok { return trillian.TreeType_UNKNOWN_TREE_TYPE, readonly, nil } - if ok, readonly := getLogRequestInfo(req); ok { + if readonly, ok := getLogRequestInfo(req); ok { return trillian.TreeType_LOG, readonly, nil } - if ok, readonly := getMapRequestInfo(req); ok { + if readonly, ok := getMapRequestInfo(req); ok { return trillian.TreeType_MAP, readonly, nil } return trillian.TreeType_UNKNOWN_TREE_TYPE, false, fmt.Errorf("unmapped request type: %T", req) } func getAdminRequestInfo(req interface{}) (bool, bool) { - isAdmin := true readonly := false + ok := true switch req.(type) { case *trillian.GetTreeRequest, *trillian.ListTreesRequest: @@ -157,14 +185,14 @@ func getAdminRequestInfo(req interface{}) (bool, bool) { *trillian.DeleteTreeRequest, *trillian.UpdateTreeRequest: default: - isAdmin = false + ok = false } - return isAdmin, readonly + return readonly, ok } func getLogRequestInfo(req interface{}) (bool, bool) { - isLog := true readonly := false + ok := true switch req.(type) { case *trillian.GetConsistencyProofRequest, *trillian.GetEntryAndProofRequest, @@ -178,14 +206,14 @@ func getLogRequestInfo(req interface{}) (bool, bool) { case *trillian.QueueLeafRequest, *trillian.QueueLeavesRequest: default: - isLog = false + ok = false } - return isLog, readonly + return readonly, ok } func getMapRequestInfo(req interface{}) (bool, bool) { - isMap := true readonly := false + ok := true switch req.(type) { case *trillian.GetMapLeavesRequest, *trillian.GetSignedMapRootByRevisionRequest, @@ -193,9 +221,9 @@ func getMapRequestInfo(req interface{}) (bool, bool) { readonly = true case *trillian.SetMapLeavesRequest: default: - isMap = false + ok = false } - return isMap, readonly + return readonly, ok } type treeIDRequest interface { @@ -214,6 +242,14 @@ type mapIDRequest interface { GetMapId() int64 } +type logLeavesRequest interface { + GetLeaves() []*trillian.LogLeaf +} + +type mapLeavesRequest interface { + GetLeaves() []*trillian.MapLeaf +} + // Combine combines unary interceptors. // They are nested in order, so interceptor[0] calls on to (and sees the result of) interceptor[1], etc. func Combine(interceptors ...grpc.UnaryServerInterceptor) grpc.UnaryServerInterceptor { diff --git a/server/interceptor/interceptor_test.go b/server/interceptor/interceptor_test.go index ae1ec0cd9c..c5586e6e7b 100644 --- a/server/interceptor/interceptor_test.go +++ b/server/interceptor/interceptor_test.go @@ -134,76 +134,134 @@ func TestTrillianInterceptor_QuotaInterception(t *testing.T) { logTree := *testonly.LogTree logTree.TreeId = 10 + mapTree := *testonly.MapTree + mapTree.TreeId = 11 + admin := storage.NewMockAdminStorage(ctrl) adminTX := storage.NewMockReadOnlyAdminTX(ctrl) admin.EXPECT().Snapshot(gomock.Any()).AnyTimes().Return(adminTX, nil) adminTX.EXPECT().GetTree(gomock.Any(), logTree.TreeId).AnyTimes().Return(&logTree, nil) + adminTX.EXPECT().GetTree(gomock.Any(), mapTree.TreeId).AnyTimes().Return(&mapTree, nil) adminTX.EXPECT().Close().AnyTimes().Return(nil) adminTX.EXPECT().Commit().AnyTimes().Return(nil) user := "llama" tests := []struct { desc string + dryRun bool req interface{} - spec []quota.Spec + specs []quota.Spec getTokensErr error wantCode codes.Code + wantTokens int }{ { - desc: "createTree", + desc: "adminWrite", req: &trillian.CreateTreeRequest{Tree: testonly.LogTree}, - spec: []quota.Spec{ - {Group: quota.User, Kind: quota.Write, User: user}, - {Group: quota.Global, Kind: quota.Write}, - }, }, { - desc: "listTrees", + desc: "adminRead", req: &trillian.ListTreesRequest{}, - spec: []quota.Spec{ - {Group: quota.User, Kind: quota.Read, User: user}, - {Group: quota.Global, Kind: quota.Read}, - }, }, { - desc: "readonlyWithTree", + desc: "logReadonly", req: &trillian.GetLatestSignedLogRootRequest{LogId: logTree.TreeId}, - spec: []quota.Spec{ + specs: []quota.Spec{ {Group: quota.User, Kind: quota.Read, User: user}, {Group: quota.Tree, Kind: quota.Read, TreeID: logTree.TreeId}, {Group: quota.Global, Kind: quota.Read}, }, + wantTokens: 1, }, { - desc: "rwWithTree", + desc: "logRW", req: &trillian.QueueLeafRequest{LogId: logTree.TreeId}, - spec: []quota.Spec{ + specs: []quota.Spec{ {Group: quota.User, Kind: quota.Write, User: user}, {Group: quota.Tree, Kind: quota.Write, TreeID: logTree.TreeId}, {Group: quota.Global, Kind: quota.Write}, }, + wantTokens: 1, + }, + { + desc: "mapReadonly", + req: &trillian.GetMapLeavesRequest{MapId: mapTree.TreeId}, + specs: []quota.Spec{ + {Group: quota.User, Kind: quota.Read, User: user}, + {Group: quota.Tree, Kind: quota.Read, TreeID: mapTree.TreeId}, + {Group: quota.Global, Kind: quota.Read}, + }, + wantTokens: 1, }, { desc: "quotaError", req: &trillian.GetLatestSignedLogRootRequest{LogId: logTree.TreeId}, - spec: []quota.Spec{ + specs: []quota.Spec{ {Group: quota.User, Kind: quota.Read, User: user}, {Group: quota.Tree, Kind: quota.Read, TreeID: logTree.TreeId}, {Group: quota.Global, Kind: quota.Read}, }, getTokensErr: errors.New("not enough tokens"), wantCode: codes.ResourceExhausted, + wantTokens: 1, + }, + { + desc: "quotaError-dryRun", + dryRun: true, + req: &trillian.GetLatestSignedLogRootRequest{LogId: logTree.TreeId}, + specs: []quota.Spec{ + {Group: quota.User, Kind: quota.Read, User: user}, + {Group: quota.Tree, Kind: quota.Read, TreeID: logTree.TreeId}, + {Group: quota.Global, Kind: quota.Read}, + }, + getTokensErr: errors.New("not enough tokens"), + wantTokens: 1, + }, + { + desc: "multiTokens-logLeavesRequest", + req: &trillian.QueueLeavesRequest{ + LogId: logTree.TreeId, + Leaves: []*trillian.LogLeaf{{}, {}, {}}, + }, + specs: []quota.Spec{ + {Group: quota.User, Kind: quota.Write, User: user}, + {Group: quota.Tree, Kind: quota.Write, TreeID: logTree.TreeId}, + {Group: quota.Global, Kind: quota.Write}, + }, + wantTokens: 3, + }, + { + desc: "multiTokens-zeroTokens", + req: &trillian.QueueLeavesRequest{ + LogId: logTree.TreeId, + Leaves: nil, + }, + }, + { + desc: "multiTokens-mapLeavesRequest", + req: &trillian.SetMapLeavesRequest{ + MapId: mapTree.TreeId, + Leaves: []*trillian.MapLeaf{{}, {}, {}, {}, {}}, + }, + specs: []quota.Spec{ + {Group: quota.User, Kind: quota.Write, User: user}, + {Group: quota.Tree, Kind: quota.Write, TreeID: mapTree.TreeId}, + {Group: quota.Global, Kind: quota.Write}, + }, + wantTokens: 5, }, } ctx := context.Background() for _, test := range tests { qm := quota.NewMockManager(ctrl) - qm.EXPECT().GetUser(gomock.Any(), test.req).Return(user) - qm.EXPECT().GetTokens(gomock.Any(), 1 /* numTokens */, test.spec).Return(test.getTokensErr) + qm.EXPECT().GetUser(gomock.Any(), test.req).MaxTimes(1).Return(user) + if test.wantTokens > 0 { + qm.EXPECT().GetTokens(gomock.Any(), test.wantTokens, test.specs).Return(test.getTokensErr) + } handler := &fakeHandler{resp: "ok"} - intercept := &TrillianInterceptor{Admin: admin, QuotaManager: qm} + intercept := &TrillianInterceptor{Admin: admin, QuotaManager: qm, QuotaDryRun: test.dryRun} // resp and handler assertions are done by TestTrillianInterceptor_TreeInterception, // we're only concerned with the quota logic here. diff --git a/server/trillian_log_server/main.go b/server/trillian_log_server/main.go index a62413f314..e6d3065c7b 100644 --- a/server/trillian_log_server/main.go +++ b/server/trillian_log_server/main.go @@ -48,6 +48,7 @@ var ( etcdService = flag.String("etcd_service", "trillian-logserver", "Service name to announce ourselves under") etcdHTTPService = flag.String("etcd_http_service", "trillian-logserver-http", "Service name to announce our HTTP endpoint under") maxUnsequencedRows = flag.Int("max_unsequenced_rows", mysqlq.DefaultMaxUnsequenced, "Max number of unsequenced rows before rate limiting kicks in") + quotaDryRun = flag.Bool("quota_dry_run", false, "If true no requests are blocked due to lack of tokens") pkcs11ModulePath = flag.String("pkcs11_module_path", "", "Path to the PKCS#11 module to use for keys that use the PKCS#11 interface") @@ -104,6 +105,7 @@ func main() { ti := &interceptor.TrillianInterceptor{ Admin: registry.AdminStorage, QuotaManager: registry.QuotaManager, + QuotaDryRun: *quotaDryRun, } netInterceptor := interceptor.Combine(stats.Interceptor(), interceptor.ErrorWrapper, ti.UnaryInterceptor) s := grpc.NewServer(grpc.UnaryInterceptor(netInterceptor)) diff --git a/server/trillian_log_signer/main.go b/server/trillian_log_signer/main.go index 3ee04f8946..3a2cf39054 100644 --- a/server/trillian_log_signer/main.go +++ b/server/trillian_log_signer/main.go @@ -22,13 +22,11 @@ import ( "os" "time" - _ "github.com/go-sql-driver/mysql" // Load MySQL driver "github.com/golang/glog" "github.com/google/trillian/cmd" "github.com/google/trillian/crypto/keys" "github.com/google/trillian/extension" - _ "github.com/google/trillian/merkle/objhasher" // Load hashers - _ "github.com/google/trillian/merkle/rfc6962" // Load hashers + "github.com/google/trillian/log" "github.com/google/trillian/monitoring/prometheus" "github.com/google/trillian/quota" "github.com/google/trillian/server" @@ -37,6 +35,10 @@ import ( "github.com/google/trillian/util/etcd" "github.com/prometheus/client_golang/prometheus/promhttp" "golang.org/x/net/context" + + _ "github.com/go-sql-driver/mysql" // Load MySQL driver + _ "github.com/google/trillian/merkle/objhasher" // Load hashers + _ "github.com/google/trillian/merkle/rfc6962" // Load hashers ) var ( @@ -50,6 +52,8 @@ var ( etcdServers = flag.String("etcd_servers", "", "A comma-separated list of etcd servers") etcdHTTPService = flag.String("etcd_http_service", "trillian-logsigner-http", "Service name to announce our HTTP endpoint under") lockDir = flag.String("lock_file_path", "/test/multimaster", "etcd lock file directory path") + quotaIncreaseFactor = flag.Float64("quota_increase_factor", log.QuotaIncreaseFactor, + "Increase factor for tokens replenished by sequencing-based quotas (1 means a 1:1 relationship between sequenced leaves and replenished tokens).") preElectionPause = flag.Duration("pre_election_pause", 1*time.Second, "Maximum time to wait before starting elections") masterCheckInterval = flag.Duration("master_check_interval", 5*time.Second, "Interval between checking mastership still held") @@ -127,6 +131,7 @@ func main() { // Start the sequencing loop, which will run until we terminate the process. This controls // both sequencing and signing. // TODO(Martin2112): Should respect read only mode and the flags in tree control etc + log.QuotaIncreaseFactor = *quotaIncreaseFactor sequencerManager := server.NewSequencerManager(registry, *sequencerGuardWindowFlag) info := server.LogOperationInfo{ Registry: registry, diff --git a/server/trillian_map_server/main.go b/server/trillian_map_server/main.go index bf29fe55a2..f2e9bc2517 100644 --- a/server/trillian_map_server/main.go +++ b/server/trillian_map_server/main.go @@ -43,6 +43,7 @@ var ( rpcEndpoint = flag.String("rpc_endpoint", "localhost:8090", "Endpoint for RPC requests (host:port)") httpEndpoint = flag.String("http_endpoint", "localhost:8091", "Endpoint for HTTP metrics and REST requests on (host:port, empty means disabled)") maxUnsequencedRows = flag.Int("max_unsequenced_rows", mysqlq.DefaultMaxUnsequenced, "Max number of unsequenced rows before rate limiting kicks in") + quotaDryRun = flag.Bool("quota_dry_run", false, "If true no requests are blocked due to lack of tokens") configFile = flag.String("config", "", "Config file containing flags, file contents can be overridden by command line flags") ) @@ -75,6 +76,7 @@ func main() { ti := &interceptor.TrillianInterceptor{ Admin: registry.AdminStorage, QuotaManager: registry.QuotaManager, + QuotaDryRun: *quotaDryRun, } netInterceptor := interceptor.Combine(stats.Interceptor(), interceptor.ErrorWrapper, ti.UnaryInterceptor) s := grpc.NewServer(grpc.UnaryInterceptor(netInterceptor))