[OSS] gRPC Blocking Queries (#17426)

* feat: initial grpc blocking queries

* changelog and docs update
This commit is contained in:
Dan Stough 2023-05-23 17:29:10 -04:00 committed by GitHub
parent cdc47ea200
commit e502be8c6e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
42 changed files with 1377 additions and 1155 deletions

5
.changelog/17426.txt Normal file
View File

@ -0,0 +1,5 @@
```release-note:improvement
peering: gRPC queries for TrustBundleList, TrustBundleRead, PeeringList, and PeeringRead now support blocking semantics,
reducing network and CPU demand.
The HTTP APIs for Peering List and Read have been updated to support blocking.
```

View File

@ -0,0 +1,208 @@
package blockingquery
import (
"context"
"errors"
"fmt"
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/lib"
)
// Sentinel errors that must be used with blockingQuery
var (
ErrNotFound = fmt.Errorf("no data found for query")
ErrNotChanged = fmt.Errorf("data did not change for query")
)
// QueryFn is used to perform a query operation. See Server.blockingQuery for
// the requirements of this function.
type QueryFn func(memdb.WatchSet, *state.Store) error
// RequestOptions are options used by Server.blockingQuery to modify the
// behaviour of the query operation, or to populate response metadata.
type RequestOptions interface {
GetToken() string
GetMinQueryIndex() uint64
GetMaxQueryTime() (time.Duration, error)
GetRequireConsistent() bool
}
// ResponseMeta is an interface used to populate the response struct
// with metadata about the query and the state of the server.
type ResponseMeta interface {
SetLastContact(time.Duration)
SetKnownLeader(bool)
GetIndex() uint64
SetIndex(uint64)
SetResultsFilteredByACLs(bool)
}
// FSMServer is interface into the stateful components of a Consul server, such
// as memdb or raft leadership.
type FSMServer interface {
ConsistentRead() error
DecrementBlockingQueries() uint64
GetShutdownChannel() chan struct{}
GetState() *state.Store
IncrementBlockingQueries() uint64
RPCQueryTimeout(time.Duration) time.Duration
SetQueryMeta(ResponseMeta, string)
}
// Query performs a blocking query if opts.GetMinQueryIndex is
// greater than 0, otherwise performs a non-blocking query. Blocking queries will
// block until responseMeta.Index is greater than opts.GetMinQueryIndex,
// or opts.GetMaxQueryTime is reached. Non-blocking queries return immediately
// after performing the query.
//
// If opts.GetRequireConsistent is true, blockingQuery will first verify it is
// still the cluster leader before performing the query.
//
// The query function is expected to be a closure that has access to responseMeta
// so that it can set the Index. The actual result of the query is opaque to blockingQuery.
//
// The query function can return ErrNotFound, which is a sentinel error. Returning
// ErrNotFound indicates that the query found no results, which allows
// blockingQuery to keep blocking until the query returns a non-nil error.
// The query function must take care to set the actual result of the query to
// nil in these cases, otherwise when blockingQuery times out it may return
// a previous result. ErrNotFound will never be returned to the caller, it is
// converted to nil before returning.
//
// The query function can return ErrNotChanged, which is a sentinel error. This
// can only be returned on calls AFTER the first call, as it would not be
// possible to detect the absence of a change on the first call. Returning
// ErrNotChanged indicates that the query results are identical to the prior
// results which allows blockingQuery to keep blocking until the query returns
// a real changed result.
//
// The query function must take care to ensure the actual result of the query
// is either left unmodified or explicitly left in a good state before
// returning, otherwise when blockingQuery times out it may return an
// incomplete or unexpected result. ErrNotChanged will never be returned to the
// caller, it is converted to nil before returning.
//
// If query function returns any other error, the error is returned to the caller
// immediately.
//
// The query function must follow these rules:
//
// 1. to access data it must use the passed in state.Store.
// 2. it must set the responseMeta.Index to an index greater than
// opts.GetMinQueryIndex if the results return by the query have changed.
// 3. any channels added to the memdb.WatchSet must unblock when the results
// returned by the query have changed.
//
// To ensure optimal performance of the query, the query function should make a
// best-effort attempt to follow these guidelines:
//
// 1. only set responseMeta.Index to an index greater than
// opts.GetMinQueryIndex when the results returned by the query have changed.
// 2. any channels added to the memdb.WatchSet should only unblock when the
// results returned by the query have changed.
func Query(
fsmServer FSMServer,
requestOpts RequestOptions,
responseMeta ResponseMeta,
query QueryFn,
) error {
var ctx context.Context = &lib.StopChannelContext{StopCh: fsmServer.GetShutdownChannel()}
metrics.IncrCounter([]string{"rpc", "query"}, 1)
minQueryIndex := requestOpts.GetMinQueryIndex()
// Perform a non-blocking query
if minQueryIndex == 0 {
if requestOpts.GetRequireConsistent() {
if err := fsmServer.ConsistentRead(); err != nil {
return err
}
}
var ws memdb.WatchSet
err := query(ws, fsmServer.GetState())
fsmServer.SetQueryMeta(responseMeta, requestOpts.GetToken())
if errors.Is(err, ErrNotFound) || errors.Is(err, ErrNotChanged) {
return nil
}
return err
}
maxQueryTimeout, err := requestOpts.GetMaxQueryTime()
if err != nil {
return err
}
timeout := fsmServer.RPCQueryTimeout(maxQueryTimeout)
ctx, cancel := context.WithTimeout(ctx, timeout)
defer cancel()
count := fsmServer.IncrementBlockingQueries()
metrics.SetGauge([]string{"rpc", "queries_blocking"}, float32(count))
// decrement the count when the function returns.
defer fsmServer.DecrementBlockingQueries()
var (
notFound bool
ranOnce bool
)
for {
if requestOpts.GetRequireConsistent() {
if err := fsmServer.ConsistentRead(); err != nil {
return err
}
}
// Operate on a consistent set of state. This makes sure that the
// abandon channel goes with the state that the caller is using to
// build watches.
store := fsmServer.GetState()
ws := memdb.NewWatchSet()
// This channel will be closed if a snapshot is restored and the
// whole state store is abandoned.
ws.Add(store.AbandonCh())
err := query(ws, store)
fsmServer.SetQueryMeta(responseMeta, requestOpts.GetToken())
switch {
case errors.Is(err, ErrNotFound):
if notFound {
// query result has not changed
minQueryIndex = responseMeta.GetIndex()
}
notFound = true
case errors.Is(err, ErrNotChanged):
if ranOnce {
// query result has not changed
minQueryIndex = responseMeta.GetIndex()
}
case err != nil:
return err
}
ranOnce = true
if responseMeta.GetIndex() > minQueryIndex {
return nil
}
// block until something changes, or the timeout
if err := ws.WatchCtx(ctx); err != nil {
// exit if we've reached the timeout, or other cancellation
return nil
}
// exit if the state store has been abandoned
select {
case <-store.AbandonCh():
return nil
default:
}
}
}

View File

@ -0,0 +1,4 @@
package blockingquery
// TODO: move tests from the consul package, rpc_test.go, TestServer_blockingQuery
// here using mock for FSMServer w/ structs.QueryOptions and structs.QueryOptions

View File

@ -7,22 +7,24 @@ import (
"context" "context"
"fmt" "fmt"
"strconv" "strconv"
"time"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/proto/private/pbpeering"
"github.com/mitchellh/hashstructure" "github.com/mitchellh/hashstructure"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/status" "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/private/pbpeering"
) )
// PeeringListName is the recommended name for registration. // PeeringListName is the recommended name for registration.
const PeeringListName = "peers" const PeeringListName = "peers"
// PeeringListRequest represents the combination of request payload
// and options that would normally be sent over headers.
type PeeringListRequest struct { type PeeringListRequest struct {
Request *pbpeering.PeeringListRequest Request *pbpeering.PeeringListRequest
structs.QueryOptions structs.QueryOptions
@ -32,13 +34,10 @@ func (r *PeeringListRequest) CacheInfo() cache.RequestInfo {
info := cache.RequestInfo{ info := cache.RequestInfo{
Token: r.Token, Token: r.Token,
Datacenter: "", Datacenter: "",
MinIndex: 0, MinIndex: r.MinQueryIndex,
Timeout: 0, Timeout: r.MaxQueryTime,
MustRevalidate: false, MaxAge: r.MaxAge,
MustRevalidate: r.MustRevalidate,
// OPTIMIZE(peering): Cache.notifyPollingQuery polls at this interval. We need to revisit how that polling works.
// Using an exponential backoff when the result hasn't changed may be preferable.
MaxAge: 1 * time.Second,
} }
v, err := hashstructure.Hash([]interface{}{ v, err := hashstructure.Hash([]interface{}{
@ -56,7 +55,7 @@ func (r *PeeringListRequest) CacheInfo() cache.RequestInfo {
// Peerings supports fetching the list of peers for a given partition or wildcard-specifier. // Peerings supports fetching the list of peers for a given partition or wildcard-specifier.
type Peerings struct { type Peerings struct {
RegisterOptionsNoRefresh RegisterOptionsBlockingRefresh
Client PeeringLister Client PeeringLister
} }
@ -67,7 +66,7 @@ type PeeringLister interface {
) (*pbpeering.PeeringListResponse, error) ) (*pbpeering.PeeringListResponse, error)
} }
func (t *Peerings) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchResult, error) { func (t *Peerings) Fetch(opts cache.FetchOptions, req cache.Request) (cache.FetchResult, error) {
var result cache.FetchResult var result cache.FetchResult
// The request should be a PeeringListRequest. // The request should be a PeeringListRequest.
@ -79,10 +78,17 @@ func (t *Peerings) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchRe
"Internal cache failure: request wrong type: %T", req) "Internal cache failure: request wrong type: %T", req)
} }
// Always allow stale - there's no point in hitting leader if the request is // Lightweight copy this object so that manipulating QueryOptions doesn't race.
// going to be served from cache and end up arbitrarily stale anyway. This dup := *reqReal
// allows cached service-discover to automatically read scale across all reqReal = &dup
// servers too.
// Set the minimum query index to our current index, so we block
reqReal.QueryOptions.MinQueryIndex = opts.MinIndex
reqReal.QueryOptions.MaxQueryTime = opts.Timeout
// We allow stale queries here to spread out the RPC load, but peerstream information, including the STATUS,
// will not be returned. Right now this is fine for the watch in proxycfg/mesh_gateway.go,
// but it could be a problem for a future consumer.
reqReal.QueryOptions.SetAllowStale(true) reqReal.QueryOptions.SetAllowStale(true)
ctx, err := external.ContextWithQueryOptions(context.Background(), reqReal.QueryOptions) ctx, err := external.ContextWithQueryOptions(context.Background(), reqReal.QueryOptions)
@ -91,7 +97,8 @@ func (t *Peerings) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchRe
} }
// Fetch // Fetch
reply, err := t.Client.PeeringList(ctx, reqReal.Request) var header metadata.MD
reply, err := t.Client.PeeringList(ctx, reqReal.Request, grpc.Header(&header))
if err != nil { if err != nil {
// Return an empty result if the error is due to peering being disabled. // Return an empty result if the error is due to peering being disabled.
// This allows mesh gateways to receive an update and confirm that the watch is set. // This allows mesh gateways to receive an update and confirm that the watch is set.
@ -103,8 +110,19 @@ func (t *Peerings) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchRe
return result, err return result, err
} }
// This first case is using the legacy index field
// It should be removed in a future version in favor of the index from QueryMeta
if reply.OBSOLETE_Index != 0 {
result.Index = reply.OBSOLETE_Index
} else {
meta, err := external.QueryMetaFromGRPCMeta(header)
if err != nil {
return result, fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
}
result.Index = meta.GetIndex()
}
result.Value = reply result.Value = reply
result.Index = reply.Index
return result, nil return result, nil
} }

View File

@ -8,14 +8,16 @@ import (
"testing" "testing"
"time" "time"
"github.com/mitchellh/copystructure"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
grpcstatus "google.golang.org/grpc/status" grpcstatus "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/private/pbpeering" "github.com/hashicorp/consul/proto/private/pbpeering"
) )
@ -24,7 +26,6 @@ func TestPeerings(t *testing.T) {
typ := &Peerings{Client: client} typ := &Peerings{Client: client}
resp := &pbpeering.PeeringListResponse{ resp := &pbpeering.PeeringListResponse{
Index: 48,
Peerings: []*pbpeering.Peering{ Peerings: []*pbpeering.Peering{
{ {
Name: "peer1", Name: "peer1",
@ -36,12 +37,38 @@ func TestPeerings(t *testing.T) {
} }
// Expect the proper call. // Expect the proper call.
// This also returns the canned response above. // This also set the gRPC metadata returned by pointer.
client.On("PeeringList", mock.Anything, mock.Anything). client.On("PeeringList", mock.Anything, mock.Anything, mock.Anything).
Return(resp, nil) Return(resp, nil).
Run(func(args mock.Arguments) {
// Validate Query Options
ctx := args.Get(0).(context.Context)
out, ok := metadata.FromOutgoingContext(ctx)
require.True(t, ok)
ctx = metadata.NewIncomingContext(ctx, out)
options, err := external.QueryOptionsFromContext(ctx)
require.NoError(t, err)
require.Equal(t, uint64(28), options.MinQueryIndex)
require.Equal(t, time.Duration(1100), options.MaxQueryTime)
require.True(t, options.AllowStale)
// Send back Query Meta on pointer of header
header := args.Get(2).(grpc.HeaderCallOption)
qm := structs.QueryMeta{
Index: 48,
}
md, err := external.GRPCMetadataFromQueryMeta(qm)
require.NoError(t, err)
*header.HeaderAddr = md
})
// Fetch and assert against the result. // Fetch and assert against the result.
result, err := typ.Fetch(cache.FetchOptions{}, &PeeringListRequest{ result, err := typ.Fetch(cache.FetchOptions{
MinIndex: 28,
Timeout: time.Duration(1100),
}, &PeeringListRequest{
Request: &pbpeering.PeeringListRequest{}, Request: &pbpeering.PeeringListRequest{},
}) })
require.NoError(t, err) require.NoError(t, err)
@ -58,7 +85,7 @@ func TestPeerings_PeeringDisabled(t *testing.T) {
var resp *pbpeering.PeeringListResponse var resp *pbpeering.PeeringListResponse
// Expect the proper call, but return the peering disabled error // Expect the proper call, but return the peering disabled error
client.On("PeeringList", mock.Anything, mock.Anything). client.On("PeeringList", mock.Anything, mock.Anything, mock.Anything).
Return(resp, grpcstatus.Error(codes.FailedPrecondition, "peering must be enabled to use this endpoint")) Return(resp, grpcstatus.Error(codes.FailedPrecondition, "peering must be enabled to use this endpoint"))
// Fetch and assert against the result. // Fetch and assert against the result.
@ -81,54 +108,3 @@ func TestPeerings_badReqType(t *testing.T) {
require.Error(t, err) require.Error(t, err)
require.Contains(t, err.Error(), "wrong type") require.Contains(t, err.Error(), "wrong type")
} }
// This test asserts that we can continuously poll this cache type, given that it doesn't support blocking.
func TestPeerings_MultipleUpdates(t *testing.T) {
c := cache.New(cache.Options{})
client := NewMockPeeringLister(t)
// On each mock client call to PeeringList we will increment the index by 1
// to simulate new data arriving.
resp := &pbpeering.PeeringListResponse{
Index: uint64(0),
}
client.On("PeeringList", mock.Anything, mock.Anything).
Return(func(ctx context.Context, in *pbpeering.PeeringListRequest, opts ...grpc.CallOption) *pbpeering.PeeringListResponse {
resp.Index++
// Avoids triggering the race detection by copying the output
copyResp, err := copystructure.Copy(resp)
require.NoError(t, err)
output := copyResp.(*pbpeering.PeeringListResponse)
return output
}, nil)
c.RegisterType(PeeringListName, &Peerings{Client: client})
ch := make(chan cache.UpdateEvent)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
t.Cleanup(cancel)
require.NoError(t, c.Notify(ctx, PeeringListName, &PeeringListRequest{
Request: &pbpeering.PeeringListRequest{},
}, "updates", ch))
i := uint64(1)
for {
select {
case <-ctx.Done():
t.Fatal("context deadline exceeded")
return
case update := <-ch:
// Expect to receive updates for increasing indexes serially.
actual := update.Result.(*pbpeering.PeeringListResponse)
require.Equal(t, i, actual.Index)
i++
if i > 3 {
return
}
}
}
}

View File

@ -7,10 +7,10 @@ import (
"context" "context"
"fmt" "fmt"
"strconv" "strconv"
"time"
"github.com/mitchellh/hashstructure" "github.com/mitchellh/hashstructure"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
external "github.com/hashicorp/consul/agent/grpc-external" external "github.com/hashicorp/consul/agent/grpc-external"
@ -21,6 +21,8 @@ import (
// Recommended name for registration. // Recommended name for registration.
const TrustBundleReadName = "peer-trust-bundle" const TrustBundleReadName = "peer-trust-bundle"
// TrustBundleReadRequest represents the combination of request payload
// and options that would normally be sent over headers.
type TrustBundleReadRequest struct { type TrustBundleReadRequest struct {
Request *pbpeering.TrustBundleReadRequest Request *pbpeering.TrustBundleReadRequest
structs.QueryOptions structs.QueryOptions
@ -30,13 +32,10 @@ func (r *TrustBundleReadRequest) CacheInfo() cache.RequestInfo {
info := cache.RequestInfo{ info := cache.RequestInfo{
Token: r.Token, Token: r.Token,
Datacenter: "", Datacenter: "",
MinIndex: 0, MinIndex: r.MinQueryIndex,
Timeout: 0, Timeout: r.MaxQueryTime,
MustRevalidate: false, MaxAge: r.MaxAge,
MustRevalidate: r.MustRevalidate,
// OPTIMIZE(peering): Cache.notifyPollingQuery polls at this interval. We need to revisit how that polling works.
// Using an exponential backoff when the result hasn't changed may be preferable.
MaxAge: 1 * time.Second,
} }
v, err := hashstructure.Hash([]interface{}{ v, err := hashstructure.Hash([]interface{}{
@ -56,7 +55,7 @@ func (r *TrustBundleReadRequest) CacheInfo() cache.RequestInfo {
// TrustBundle supports fetching discovering service instances via prepared // TrustBundle supports fetching discovering service instances via prepared
// queries. // queries.
type TrustBundle struct { type TrustBundle struct {
RegisterOptionsNoRefresh RegisterOptionsBlockingRefresh
Client TrustBundleReader Client TrustBundleReader
} }
@ -67,7 +66,7 @@ type TrustBundleReader interface {
) (*pbpeering.TrustBundleReadResponse, error) ) (*pbpeering.TrustBundleReadResponse, error)
} }
func (t *TrustBundle) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchResult, error) { func (t *TrustBundle) Fetch(opts cache.FetchOptions, req cache.Request) (cache.FetchResult, error) {
var result cache.FetchResult var result cache.FetchResult
// The request should be a TrustBundleReadRequest. // The request should be a TrustBundleReadRequest.
@ -79,6 +78,14 @@ func (t *TrustBundle) Fetch(_ cache.FetchOptions, req cache.Request) (cache.Fetc
"Internal cache failure: request wrong type: %T", req) "Internal cache failure: request wrong type: %T", req)
} }
// Lightweight copy this object so that manipulating QueryOptions doesn't race.
dup := *reqReal
reqReal = &dup
// Set the minimum query index to our current index, so we block
reqReal.QueryOptions.MinQueryIndex = opts.MinIndex
reqReal.QueryOptions.MaxQueryTime = opts.Timeout
// Always allow stale - there's no point in hitting leader if the request is // Always allow stale - there's no point in hitting leader if the request is
// going to be served from cache and end up arbitrarily stale anyway. This // going to be served from cache and end up arbitrarily stale anyway. This
// allows cached service-discover to automatically read scale across all // allows cached service-discover to automatically read scale across all
@ -91,13 +98,25 @@ func (t *TrustBundle) Fetch(_ cache.FetchOptions, req cache.Request) (cache.Fetc
return result, err return result, err
} }
reply, err := t.Client.TrustBundleRead(ctx, reqReal.Request) var header metadata.MD
reply, err := t.Client.TrustBundleRead(ctx, reqReal.Request, grpc.Header(&header))
if err != nil { if err != nil {
return result, err return result, err
} }
// This first case is using the legacy index field
// It should be removed in a future version in favor of the index from QueryMeta
if reply.OBSOLETE_Index != 0 {
result.Index = reply.OBSOLETE_Index
} else {
meta, err := external.QueryMetaFromGRPCMeta(header)
if err != nil {
return result, fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
}
result.Index = meta.GetIndex()
}
result.Value = reply result.Value = reply
result.Index = reply.Index
return result, nil return result, nil
} }

View File

@ -10,8 +10,12 @@ import (
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/private/pbpeering" "github.com/hashicorp/consul/proto/private/pbpeering"
) )
@ -20,7 +24,6 @@ func TestTrustBundle(t *testing.T) {
typ := &TrustBundle{Client: client} typ := &TrustBundle{Client: client}
resp := &pbpeering.TrustBundleReadResponse{ resp := &pbpeering.TrustBundleReadResponse{
Index: 48,
Bundle: &pbpeering.PeeringTrustBundle{ Bundle: &pbpeering.PeeringTrustBundle{
PeerName: "peer1", PeerName: "peer1",
RootPEMs: []string{"peer1-roots"}, RootPEMs: []string{"peer1-roots"},
@ -29,15 +32,41 @@ func TestTrustBundle(t *testing.T) {
// Expect the proper call. // Expect the proper call.
// This also returns the canned response above. // This also returns the canned response above.
client.On("TrustBundleRead", mock.Anything, mock.Anything). client.On("TrustBundleRead", mock.Anything, mock.Anything, mock.Anything).
Run(func(args mock.Arguments) { Run(func(args mock.Arguments) {
// Validate Query Options
ctx := args.Get(0).(context.Context)
out, ok := metadata.FromOutgoingContext(ctx)
require.True(t, ok)
ctx = metadata.NewIncomingContext(ctx, out)
options, err := external.QueryOptionsFromContext(ctx)
require.NoError(t, err)
require.Equal(t, uint64(28), options.MinQueryIndex)
require.Equal(t, time.Duration(1100), options.MaxQueryTime)
require.True(t, options.AllowStale)
// Validate Request
req := args.Get(1).(*pbpeering.TrustBundleReadRequest) req := args.Get(1).(*pbpeering.TrustBundleReadRequest)
require.Equal(t, "foo", req.Name) require.Equal(t, "foo", req.Name)
// Send back Query Meta on pointer of header
header := args.Get(2).(grpc.HeaderCallOption)
qm := structs.QueryMeta{
Index: 48,
}
md, err := external.GRPCMetadataFromQueryMeta(qm)
require.NoError(t, err)
*header.HeaderAddr = md
}). }).
Return(resp, nil) Return(resp, nil)
// Fetch and assert against the result. // Fetch and assert against the result.
result, err := typ.Fetch(cache.FetchOptions{}, &TrustBundleReadRequest{ result, err := typ.Fetch(cache.FetchOptions{
MinIndex: 28,
Timeout: time.Duration(1100),
}, &TrustBundleReadRequest{
Request: &pbpeering.TrustBundleReadRequest{ Request: &pbpeering.TrustBundleReadRequest{
Name: "foo", Name: "foo",
}, },
@ -59,55 +88,3 @@ func TestTrustBundle_badReqType(t *testing.T) {
require.Error(t, err) require.Error(t, err)
require.Contains(t, err.Error(), "wrong type") require.Contains(t, err.Error(), "wrong type")
} }
// This test asserts that we can continuously poll this cache type, given that it doesn't support blocking.
func TestTrustBundle_MultipleUpdates(t *testing.T) {
c := cache.New(cache.Options{})
client := NewMockTrustBundleReader(t)
// On each mock client call to TrustBundleList by service we will increment the index by 1
// to simulate new data arriving.
resp := &pbpeering.TrustBundleReadResponse{
Index: uint64(0),
}
client.On("TrustBundleRead", mock.Anything, mock.Anything).
Run(func(args mock.Arguments) {
req := args.Get(1).(*pbpeering.TrustBundleReadRequest)
require.Equal(t, "foo", req.Name)
// Increment on each call.
resp.Index++
}).
Return(resp, nil)
c.RegisterType(TrustBundleReadName, &TrustBundle{Client: client})
ch := make(chan cache.UpdateEvent)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
t.Cleanup(cancel)
err := c.Notify(ctx, TrustBundleReadName, &TrustBundleReadRequest{
Request: &pbpeering.TrustBundleReadRequest{Name: "foo"},
}, "updates", ch)
require.NoError(t, err)
i := uint64(1)
for {
select {
case <-ctx.Done():
t.Fatal("context deadline exceeded")
return
case update := <-ch:
// Expect to receive updates for increasing indexes serially.
actual := update.Result.(*pbpeering.TrustBundleReadResponse)
require.Equal(t, i, actual.Index)
i++
if i > 3 {
return
}
}
}
}

View File

@ -7,11 +7,11 @@ import (
"context" "context"
"fmt" "fmt"
"strconv" "strconv"
"time"
"github.com/mitchellh/hashstructure" "github.com/mitchellh/hashstructure"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/status" "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
@ -23,6 +23,8 @@ import (
// Recommended name for registration. // Recommended name for registration.
const TrustBundleListName = "trust-bundles" const TrustBundleListName = "trust-bundles"
// TrustBundleListRequest represents the combination of request payload
// and options that would normally be sent over headers.
type TrustBundleListRequest struct { type TrustBundleListRequest struct {
Request *pbpeering.TrustBundleListByServiceRequest Request *pbpeering.TrustBundleListByServiceRequest
structs.QueryOptions structs.QueryOptions
@ -32,13 +34,10 @@ func (r *TrustBundleListRequest) CacheInfo() cache.RequestInfo {
info := cache.RequestInfo{ info := cache.RequestInfo{
Token: r.Token, Token: r.Token,
Datacenter: "", Datacenter: "",
MinIndex: 0, MinIndex: r.MinQueryIndex,
Timeout: 0, Timeout: r.MaxQueryTime,
MustRevalidate: false, MaxAge: r.MaxAge,
MustRevalidate: r.MustRevalidate,
// OPTIMIZE(peering): Cache.notifyPollingQuery polls at this interval. We need to revisit how that polling works.
// Using an exponential backoff when the result hasn't changed may be preferable.
MaxAge: 1 * time.Second,
} }
v, err := hashstructure.Hash([]interface{}{ v, err := hashstructure.Hash([]interface{}{
@ -60,7 +59,7 @@ func (r *TrustBundleListRequest) CacheInfo() cache.RequestInfo {
// TrustBundles supports fetching discovering service instances via prepared // TrustBundles supports fetching discovering service instances via prepared
// queries. // queries.
type TrustBundles struct { type TrustBundles struct {
RegisterOptionsNoRefresh RegisterOptionsBlockingRefresh
Client TrustBundleLister Client TrustBundleLister
} }
@ -71,7 +70,7 @@ type TrustBundleLister interface {
) (*pbpeering.TrustBundleListByServiceResponse, error) ) (*pbpeering.TrustBundleListByServiceResponse, error)
} }
func (t *TrustBundles) Fetch(_ cache.FetchOptions, req cache.Request) (cache.FetchResult, error) { func (t *TrustBundles) Fetch(opts cache.FetchOptions, req cache.Request) (cache.FetchResult, error) {
var result cache.FetchResult var result cache.FetchResult
// The request should be a TrustBundleListRequest. // The request should be a TrustBundleListRequest.
@ -83,6 +82,14 @@ func (t *TrustBundles) Fetch(_ cache.FetchOptions, req cache.Request) (cache.Fet
"Internal cache failure: request wrong type: %T", req) "Internal cache failure: request wrong type: %T", req)
} }
// Lightweight copy this object so that manipulating QueryOptions doesn't race.
dup := *reqReal
reqReal = &dup
// Set the minimum query index to our current index, so we block
reqReal.QueryOptions.MinQueryIndex = opts.MinIndex
reqReal.QueryOptions.MaxQueryTime = opts.Timeout
// Always allow stale - there's no point in hitting leader if the request is // Always allow stale - there's no point in hitting leader if the request is
// going to be served from cache and end up arbitrarily stale anyway. This // going to be served from cache and end up arbitrarily stale anyway. This
// allows cached service-discover to automatically read scale across all // allows cached service-discover to automatically read scale across all
@ -95,20 +102,32 @@ func (t *TrustBundles) Fetch(_ cache.FetchOptions, req cache.Request) (cache.Fet
return result, err return result, err
} }
reply, err := t.Client.TrustBundleListByService(ctx, reqReal.Request) var header metadata.MD
reply, err := t.Client.TrustBundleListByService(ctx, reqReal.Request, grpc.Header(&header))
if err != nil { if err != nil {
// Return an empty result if the error is due to peering being disabled. // Return an empty result if the error is due to peering being disabled.
// This allows mesh gateways to receive an update and confirm that the watch is set. // This allows mesh gateways to receive an update and confirm that the watch is set.
if e, ok := status.FromError(err); ok && e.Code() == codes.FailedPrecondition { if e, ok := status.FromError(err); ok && e.Code() == codes.FailedPrecondition {
result.Index = 1 result.Index = 1
result.Value = &pbpeering.TrustBundleListByServiceResponse{Index: 1} result.Value = &pbpeering.TrustBundleListByServiceResponse{OBSOLETE_Index: 1}
return result, nil return result, nil
} }
return result, err return result, err
} }
// This first case is using the legacy index field
// It should be removed in a future version in favor of the index from QueryMeta
if reply.OBSOLETE_Index != 0 {
result.Index = reply.OBSOLETE_Index
} else {
meta, err := external.QueryMetaFromGRPCMeta(header)
if err != nil {
return result, fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
}
result.Index = meta.GetIndex()
}
result.Value = reply result.Value = reply
result.Index = reply.Index
return result, nil return result, nil
} }

View File

@ -10,10 +10,14 @@ import (
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
grpcstatus "google.golang.org/grpc/status" grpcstatus "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/private/pbpeering" "github.com/hashicorp/consul/proto/private/pbpeering"
) )
@ -22,7 +26,6 @@ func TestTrustBundles(t *testing.T) {
typ := &TrustBundles{Client: client} typ := &TrustBundles{Client: client}
resp := &pbpeering.TrustBundleListByServiceResponse{ resp := &pbpeering.TrustBundleListByServiceResponse{
Index: 48,
Bundles: []*pbpeering.PeeringTrustBundle{ Bundles: []*pbpeering.PeeringTrustBundle{
{ {
PeerName: "peer1", PeerName: "peer1",
@ -33,15 +36,41 @@ func TestTrustBundles(t *testing.T) {
// Expect the proper call. // Expect the proper call.
// This also returns the canned response above. // This also returns the canned response above.
client.On("TrustBundleListByService", mock.Anything, mock.Anything). client.On("TrustBundleListByService", mock.Anything, mock.Anything, mock.Anything).
Run(func(args mock.Arguments) { Run(func(args mock.Arguments) {
// Validate Query Options
ctx := args.Get(0).(context.Context)
out, ok := metadata.FromOutgoingContext(ctx)
require.True(t, ok)
ctx = metadata.NewIncomingContext(ctx, out)
options, err := external.QueryOptionsFromContext(ctx)
require.NoError(t, err)
require.Equal(t, uint64(28), options.MinQueryIndex)
require.Equal(t, time.Duration(1100), options.MaxQueryTime)
require.True(t, options.AllowStale)
// Validate Request
req := args.Get(1).(*pbpeering.TrustBundleListByServiceRequest) req := args.Get(1).(*pbpeering.TrustBundleListByServiceRequest)
require.Equal(t, "foo", req.ServiceName) require.Equal(t, "foo", req.ServiceName)
// Send back Query Meta on pointer of header
header := args.Get(2).(grpc.HeaderCallOption)
qm := structs.QueryMeta{
Index: 48,
}
md, err := external.GRPCMetadataFromQueryMeta(qm)
require.NoError(t, err)
*header.HeaderAddr = md
}). }).
Return(resp, nil) Return(resp, nil)
// Fetch and assert against the result. // Fetch and assert against the result.
result, err := typ.Fetch(cache.FetchOptions{}, &TrustBundleListRequest{ result, err := typ.Fetch(cache.FetchOptions{
MinIndex: 28,
Timeout: time.Duration(1100),
}, &TrustBundleListRequest{
Request: &pbpeering.TrustBundleListByServiceRequest{ Request: &pbpeering.TrustBundleListByServiceRequest{
ServiceName: "foo", ServiceName: "foo",
}, },
@ -61,7 +90,7 @@ func TestTrustBundles_PeeringDisabled(t *testing.T) {
// Expect the proper call. // Expect the proper call.
// This also returns the canned response above. // This also returns the canned response above.
client.On("TrustBundleListByService", mock.Anything, mock.Anything). client.On("TrustBundleListByService", mock.Anything, mock.Anything, mock.Anything).
Return(resp, grpcstatus.Error(codes.FailedPrecondition, "peering must be enabled to use this endpoint")) Return(resp, grpcstatus.Error(codes.FailedPrecondition, "peering must be enabled to use this endpoint"))
// Fetch and assert against the result. // Fetch and assert against the result.
@ -86,55 +115,3 @@ func TestTrustBundles_badReqType(t *testing.T) {
require.Error(t, err) require.Error(t, err)
require.Contains(t, err.Error(), "wrong type") require.Contains(t, err.Error(), "wrong type")
} }
// This test asserts that we can continuously poll this cache type, given that it doesn't support blocking.
func TestTrustBundles_MultipleUpdates(t *testing.T) {
c := cache.New(cache.Options{})
client := NewMockTrustBundleLister(t)
// On each mock client call to TrustBundleList by service we will increment the index by 1
// to simulate new data arriving.
resp := &pbpeering.TrustBundleListByServiceResponse{
Index: uint64(0),
}
client.On("TrustBundleListByService", mock.Anything, mock.Anything).
Run(func(args mock.Arguments) {
req := args.Get(1).(*pbpeering.TrustBundleListByServiceRequest)
require.Equal(t, "foo", req.ServiceName)
// Increment on each call.
resp.Index++
}).
Return(resp, nil)
c.RegisterType(TrustBundleListName, &TrustBundles{Client: client})
ch := make(chan cache.UpdateEvent)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
t.Cleanup(cancel)
err := c.Notify(ctx, TrustBundleListName, &TrustBundleListRequest{
Request: &pbpeering.TrustBundleListByServiceRequest{ServiceName: "foo"},
}, "updates", ch)
require.NoError(t, err)
i := uint64(1)
for {
select {
case <-ctx.Done():
t.Fatal("context deadline exceeded")
return
case update := <-ch:
// Expect to receive updates for increasing indexes serially.
resp := update.Result.(*pbpeering.TrustBundleListByServiceResponse)
require.Equal(t, i, resp.Index)
i++
if i > 3 {
return
}
}
}
}

View File

@ -1113,7 +1113,7 @@ func (a *ACL) PolicyResolve(args *structs.ACLPolicyBatchGetRequest, reply *struc
} }
} }
a.srv.setQueryMeta(&reply.QueryMeta, args.Token) a.srv.SetQueryMeta(&reply.QueryMeta, args.Token)
return nil return nil
} }
@ -1520,7 +1520,7 @@ func (a *ACL) RoleResolve(args *structs.ACLRoleBatchGetRequest, reply *structs.A
} }
} }
a.srv.setQueryMeta(&reply.QueryMeta, args.Token) a.srv.SetQueryMeta(&reply.QueryMeta, args.Token)
return nil return nil
} }

View File

@ -14,6 +14,7 @@ import (
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
memdb "github.com/hashicorp/go-memdb" memdb "github.com/hashicorp/go-memdb"
"github.com/hashicorp/consul/agent/blockingquery"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
@ -273,7 +274,7 @@ func getRandomItem(items []string) string {
} }
type serverDelegate interface { type serverDelegate interface {
blockingQuery(queryOpts blockingQueryOptions, queryMeta blockingQueryResponseMeta, fn queryFn) error blockingQuery(requestOpts blockingquery.RequestOptions, responseMeta blockingquery.ResponseMeta, query blockingquery.QueryFn) error
IsLeader() bool IsLeader() bool
LeaderLastContact() time.Time LeaderLastContact() time.Time
setDatacenterSupportsFederationStates() setDatacenterSupportsFederationStates()

View File

@ -13,6 +13,7 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/blockingquery"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
@ -475,6 +476,8 @@ func TestGatewayLocator(t *testing.T) {
}) })
} }
var _ serverDelegate = (*testServerDelegate)(nil)
type testServerDelegate struct { type testServerDelegate struct {
dcSupportsFederationStates int32 // atomically accessed, at start to prevent alignment issues dcSupportsFederationStates int32 // atomically accessed, at start to prevent alignment issues
@ -496,9 +499,9 @@ func (d *testServerDelegate) datacenterSupportsFederationStates() bool {
// This is just enough to exercise the logic. // This is just enough to exercise the logic.
func (d *testServerDelegate) blockingQuery( func (d *testServerDelegate) blockingQuery(
queryOpts blockingQueryOptions, queryOpts blockingquery.RequestOptions,
queryMeta blockingQueryResponseMeta, queryMeta blockingquery.ResponseMeta,
fn queryFn, fn blockingquery.QueryFn,
) error { ) error {
minQueryIndex := queryOpts.GetMinQueryIndex() minQueryIndex := queryOpts.GetMinQueryIndex()

View File

@ -815,7 +815,7 @@ func (m *Internal) EventFire(args *structs.EventFireRequest,
} }
// Set the query meta data // Set the query meta data
m.srv.setQueryMeta(&reply.QueryMeta, args.Token) m.srv.SetQueryMeta(&reply.QueryMeta, args.Token)
// Add the consul prefix to the event name // Add the consul prefix to the event name
eventName := userEventName(args.Name) eventName := userEventName(args.Name)

View File

@ -312,9 +312,9 @@ func (p *PreparedQuery) Explain(args *structs.PreparedQueryExecuteRequest,
defer metrics.MeasureSince([]string{"prepared-query", "explain"}, time.Now()) defer metrics.MeasureSince([]string{"prepared-query", "explain"}, time.Now())
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta, args.Token) p.srv.SetQueryMeta(&reply.QueryMeta, args.Token)
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.ConsistentRead(); err != nil {
return err return err
} }
} }
@ -360,7 +360,7 @@ func (p *PreparedQuery) Execute(args *structs.PreparedQueryExecuteRequest,
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.ConsistentRead(); err != nil {
return err return err
} }
} }
@ -404,7 +404,7 @@ func (p *PreparedQuery) Execute(args *structs.PreparedQueryExecuteRequest,
// though, since this is essentially a misconfiguration. // though, since this is essentially a misconfiguration.
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta, token) p.srv.SetQueryMeta(&reply.QueryMeta, token)
// Shuffle the results in case coordinates are not available if they // Shuffle the results in case coordinates are not available if they
// requested an RTT sort. // requested an RTT sort.
@ -506,7 +506,7 @@ func (p *PreparedQuery) ExecuteRemote(args *structs.PreparedQueryExecuteRemoteRe
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.ConsistentRead(); err != nil {
return err return err
} }
} }
@ -527,7 +527,7 @@ func (p *PreparedQuery) ExecuteRemote(args *structs.PreparedQueryExecuteRemoteRe
} }
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta, token) p.srv.SetQueryMeta(&reply.QueryMeta, token)
// We don't bother trying to do an RTT sort here since we are by // We don't bother trying to do an RTT sort here since we are by
// definition in another DC. We just shuffle to make sure that we // definition in another DC. We just shuffle to make sure that we

View File

@ -13,7 +13,6 @@ import (
"math" "math"
"net" "net"
"strings" "strings"
"sync/atomic"
"time" "time"
"github.com/armon/go-metrics" "github.com/armon/go-metrics"
@ -30,6 +29,7 @@ import (
msgpackrpc "github.com/hashicorp/consul-net-rpc/net-rpc-msgpackrpc" msgpackrpc "github.com/hashicorp/consul-net-rpc/net-rpc-msgpackrpc"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/blockingquery"
"github.com/hashicorp/consul/agent/consul/rate" "github.com/hashicorp/consul/agent/consul/rate"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/consul/wanfed" "github.com/hashicorp/consul/agent/consul/wanfed"
@ -1018,167 +1018,26 @@ type blockingQueryResponseMeta interface {
SetResultsFilteredByACLs(bool) SetResultsFilteredByACLs(bool)
} }
// blockingQuery performs a blocking query if opts.GetMinQueryIndex is // blockingQuery is a passthrough to blockingquery.Query that keeps API
// greater than 0, otherwise performs a non-blocking query. Blocking queries will // compatibility with Server. That has RPC and FSM machinery mixed in the same consul
// block until responseMeta.Index is greater than opts.GetMinQueryIndex, // package.
// or opts.GetMaxQueryTime is reached. Non-blocking queries return immediately
// after performing the query.
//
// If opts.GetRequireConsistent is true, blockingQuery will first verify it is
// still the cluster leader before performing the query.
//
// The query function is expected to be a closure that has access to responseMeta
// so that it can set the Index. The actual result of the query is opaque to blockingQuery.
//
// The query function can return errNotFound, which is a sentinel error. Returning
// errNotFound indicates that the query found no results, which allows
// blockingQuery to keep blocking until the query returns a non-nil error.
// The query function must take care to set the actual result of the query to
// nil in these cases, otherwise when blockingQuery times out it may return
// a previous result. errNotFound will never be returned to the caller, it is
// converted to nil before returning.
//
// The query function can return errNotChanged, which is a sentinel error. This
// can only be returned on calls AFTER the first call, as it would not be
// possible to detect the absence of a change on the first call. Returning
// errNotChanged indicates that the query results are identical to the prior
// results which allows blockingQuery to keep blocking until the query returns
// a real changed result.
//
// The query function must take care to ensure the actual result of the query
// is either left unmodified or explicitly left in a good state before
// returning, otherwise when blockingQuery times out it may return an
// incomplete or unexpected result. errNotChanged will never be returned to the
// caller, it is converted to nil before returning.
//
// If query function returns any other error, the error is returned to the caller
// immediately.
//
// The query function must follow these rules:
//
// 1. to access data it must use the passed in state.Store.
// 2. it must set the responseMeta.Index to an index greater than
// opts.GetMinQueryIndex if the results return by the query have changed.
// 3. any channels added to the memdb.WatchSet must unblock when the results
// returned by the query have changed.
//
// To ensure optimal performance of the query, the query function should make a
// best-effort attempt to follow these guidelines:
//
// 1. only set responseMeta.Index to an index greater than
// opts.GetMinQueryIndex when the results returned by the query have changed.
// 2. any channels added to the memdb.WatchSet should only unblock when the
// results returned by the query have changed.
func (s *Server) blockingQuery( func (s *Server) blockingQuery(
opts blockingQueryOptions, requestOpts blockingquery.RequestOptions,
responseMeta blockingQueryResponseMeta, responseMeta blockingquery.ResponseMeta,
query queryFn, query blockingquery.QueryFn,
) error { ) error {
var ctx context.Context = &lib.StopChannelContext{StopCh: s.shutdownCh} return blockingquery.Query(s, requestOpts, responseMeta, query)
metrics.IncrCounter([]string{"rpc", "query"}, 1)
minQueryIndex := opts.GetMinQueryIndex()
// Perform a non-blocking query
if minQueryIndex == 0 {
if opts.GetRequireConsistent() {
if err := s.consistentRead(); err != nil {
return err
}
}
var ws memdb.WatchSet
err := query(ws, s.fsm.State())
s.setQueryMeta(responseMeta, opts.GetToken())
if errors.Is(err, errNotFound) || errors.Is(err, errNotChanged) {
return nil
}
return err
}
maxQueryTimeout, err := opts.GetMaxQueryTime()
if err != nil {
return err
}
timeout := s.rpcQueryTimeout(maxQueryTimeout)
ctx, cancel := context.WithTimeout(ctx, timeout)
defer cancel()
count := atomic.AddUint64(&s.queriesBlocking, 1)
metrics.SetGauge([]string{"rpc", "queries_blocking"}, float32(count))
// decrement the count when the function returns.
defer atomic.AddUint64(&s.queriesBlocking, ^uint64(0))
var (
notFound bool
ranOnce bool
)
for {
if opts.GetRequireConsistent() {
if err := s.consistentRead(); err != nil {
return err
}
}
// Operate on a consistent set of state. This makes sure that the
// abandon channel goes with the state that the caller is using to
// build watches.
state := s.fsm.State()
ws := memdb.NewWatchSet()
// This channel will be closed if a snapshot is restored and the
// whole state store is abandoned.
ws.Add(state.AbandonCh())
err := query(ws, state)
s.setQueryMeta(responseMeta, opts.GetToken())
switch {
case errors.Is(err, errNotFound):
if notFound {
// query result has not changed
minQueryIndex = responseMeta.GetIndex()
}
notFound = true
case errors.Is(err, errNotChanged):
if ranOnce {
// query result has not changed
minQueryIndex = responseMeta.GetIndex()
}
case err != nil:
return err
}
ranOnce = true
if responseMeta.GetIndex() > minQueryIndex {
return nil
}
// block until something changes, or the timeout
if err := ws.WatchCtx(ctx); err != nil {
// exit if we've reached the timeout, or other cancellation
return nil
}
// exit if the state store has been abandoned
select {
case <-state.AbandonCh():
return nil
default:
}
}
} }
var ( var (
errNotFound = fmt.Errorf("no data found for query") errNotFound = blockingquery.ErrNotFound
errNotChanged = fmt.Errorf("data did not change for query") errNotChanged = blockingquery.ErrNotChanged
) )
// setQueryMeta is used to populate the QueryMeta data for an RPC call // SetQueryMeta is used to populate the QueryMeta data for an RPC call
// //
// Note: This method must be called *after* filtering query results with ACLs. // Note: This method must be called *after* filtering query results with ACLs.
func (s *Server) setQueryMeta(m blockingQueryResponseMeta, token string) { func (s *Server) SetQueryMeta(m blockingquery.ResponseMeta, token string) {
if s.IsLeader() { if s.IsLeader() {
m.SetLastContact(0) m.SetLastContact(0)
m.SetKnownLeader(true) m.SetKnownLeader(true)
@ -1240,19 +1099,19 @@ func (s *Server) consistentReadWithContext(ctx context.Context) error {
} }
} }
// consistentRead is used to ensure we do not perform a stale // ConsistentRead is used to ensure we do not perform a stale
// read. This is done by verifying leadership before the read. // read. This is done by verifying leadership before the read.
func (s *Server) consistentRead() error { func (s *Server) ConsistentRead() error {
ctx, cancel := context.WithTimeout(context.Background(), s.config.RPCHoldTimeout) ctx, cancel := context.WithTimeout(context.Background(), s.config.RPCHoldTimeout)
defer cancel() defer cancel()
return s.consistentReadWithContext(ctx) return s.consistentReadWithContext(ctx)
} }
// rpcQueryTimeout calculates the timeout for the query, ensures it is // RPCQueryTimeout calculates the timeout for the query, ensures it is
// constrained to the configured limit, and adds jitter to prevent multiple // constrained to the configured limit, and adds jitter to prevent multiple
// blocking queries from all timing out at the same time. // blocking queries from all timing out at the same time.
func (s *Server) rpcQueryTimeout(queryTimeout time.Duration) time.Duration { func (s *Server) RPCQueryTimeout(queryTimeout time.Duration) time.Duration {
// Restrict the max query time, and ensure there is always one. // Restrict the max query time, and ensure there is always one.
if queryTimeout > s.config.MaxQueryTime { if queryTimeout > s.config.MaxQueryTime {
queryTimeout = s.config.MaxQueryTime queryTimeout = s.config.MaxQueryTime

View File

@ -502,7 +502,7 @@ func TestRPC_ReadyForConsistentReads(t *testing.T) {
} }
s.resetConsistentReadReady() s.resetConsistentReadReady()
err := s.consistentRead() err := s.ConsistentRead()
if err.Error() != "Not ready to serve consistent reads" { if err.Error() != "Not ready to serve consistent reads" {
t.Fatal("Server should NOT be ready for consistent reads") t.Fatal("Server should NOT be ready for consistent reads")
} }
@ -513,7 +513,7 @@ func TestRPC_ReadyForConsistentReads(t *testing.T) {
}() }()
retry.Run(t, func(r *retry.R) { retry.Run(t, func(r *retry.R) {
if err := s.consistentRead(); err != nil { if err := s.ConsistentRead(); err != nil {
r.Fatalf("Expected server to be ready for consistent reads, got error %v", err) r.Fatalf("Expected server to be ready for consistent reads, got error %v", err)
} }
}) })

View File

@ -39,6 +39,7 @@ import (
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/acl/resolver" "github.com/hashicorp/consul/acl/resolver"
"github.com/hashicorp/consul/agent/blockingquery"
"github.com/hashicorp/consul/agent/consul/authmethod" "github.com/hashicorp/consul/agent/consul/authmethod"
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth" "github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
"github.com/hashicorp/consul/agent/consul/fsm" "github.com/hashicorp/consul/agent/consul/fsm"
@ -173,6 +174,8 @@ type raftStore interface {
const requestLimitsBurstMultiplier = 10 const requestLimitsBurstMultiplier = 10
var _ blockingquery.FSMServer = (*Server)(nil)
// Server is Consul server which manages the service discovery, // Server is Consul server which manages the service discovery,
// health checking, DC forwarding, Raft, and multiple Serf pools. // health checking, DC forwarding, Raft, and multiple Serf pools.
type Server struct { type Server struct {
@ -447,6 +450,18 @@ type Server struct {
reportingManager *reporting.ReportingManager reportingManager *reporting.ReportingManager
} }
func (s *Server) DecrementBlockingQueries() uint64 {
return atomic.AddUint64(&s.queriesBlocking, ^uint64(0))
}
func (s *Server) GetShutdownChannel() chan struct{} {
return s.shutdownCh
}
func (s *Server) IncrementBlockingQueries() uint64 {
return atomic.AddUint64(&s.queriesBlocking, 1)
}
type connHandler interface { type connHandler interface {
Run() error Run() error
Handle(conn net.Conn) Handle(conn net.Conn)
@ -875,6 +890,7 @@ func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler
ConnectEnabled: config.ConnectEnabled, ConnectEnabled: config.ConnectEnabled,
PeeringEnabled: config.PeeringEnabled, PeeringEnabled: config.PeeringEnabled,
Locality: config.Locality, Locality: config.Locality,
FSMServer: s,
}) })
s.peeringServer = p s.peeringServer = p
o := operator.NewServer(operator.Config{ o := operator.NewServer(operator.Config{
@ -1819,6 +1835,13 @@ func (s *Server) FSM() *fsm.FSM {
return s.fsm return s.fsm
} }
func (s *Server) GetState() *state.Store {
if s == nil || s.FSM() == nil {
return nil
}
return s.FSM().State()
}
// Stats is used to return statistics for debugging and insight // Stats is used to return statistics for debugging and insight
// for various sub-systems // for various sub-systems
func (s *Server) Stats() map[string]map[string]string { func (s *Server) Stats() map[string]map[string]string {

View File

@ -71,14 +71,14 @@ func (s *Server) dispatchSnapshotRequest(args *structs.SnapshotRequest, in io.Re
switch args.Op { switch args.Op {
case structs.SnapshotSave: case structs.SnapshotSave:
if !args.AllowStale { if !args.AllowStale {
if err := s.consistentRead(); err != nil { if err := s.ConsistentRead(); err != nil {
return nil, err return nil, err
} }
} }
// Set the metadata here before we do anything; this should always be // Set the metadata here before we do anything; this should always be
// pessimistic if we get more data while the snapshot is being taken. // pessimistic if we get more data while the snapshot is being taken.
s.setQueryMeta(&reply.QueryMeta, args.Token) s.SetQueryMeta(&reply.QueryMeta, args.Token)
// Take the snapshot and capture the index. // Take the snapshot and capture the index.
snap, err := snapshot.New(s.logger, s.raft) snap, err := snapshot.New(s.logger, s.raft)

View File

@ -188,7 +188,7 @@ func (t *Txn) Read(args *structs.TxnReadRequest, reply *structs.TxnReadResponse)
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
if args.RequireConsistent { if args.RequireConsistent {
if err := t.srv.consistentRead(); err != nil { if err := t.srv.ConsistentRead(); err != nil {
return err return err
} }
} }
@ -220,7 +220,7 @@ func (t *Txn) Read(args *structs.TxnReadRequest, reply *structs.TxnReadResponse)
reply.QueryMeta.ResultsFilteredByACLs = total != len(reply.Results) reply.QueryMeta.ResultsFilteredByACLs = total != len(reply.Results)
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
t.srv.setQueryMeta(&reply.QueryMeta, args.Token) t.srv.SetQueryMeta(&reply.QueryMeta, args.Token)
return nil return nil
} }

View File

@ -0,0 +1,77 @@
package external
import (
"fmt"
"reflect"
"github.com/mitchellh/mapstructure"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/structs"
)
func StringToQueryBackendDecodeHookFunc(f reflect.Type, t reflect.Type, data any) (any, error) {
if f.Kind() != reflect.String {
return data, nil
}
if t != reflect.TypeOf(structs.QueryBackend(0)) {
return data, nil
}
name, ok := data.(string)
if !ok {
return data, fmt.Errorf("could not parse query backend as string")
}
return structs.QueryBackendFromString(name), nil
}
// QueryMetaFromGRPCMeta returns a structs.QueryMeta struct parsed from the metadata.MD,
// such as from a gRPC header or trailer.
func QueryMetaFromGRPCMeta(md metadata.MD) (structs.QueryMeta, error) {
var queryMeta structs.QueryMeta
m := map[string]string{}
for k, v := range md {
m[k] = v[0]
}
decodeHooks := mapstructure.ComposeDecodeHookFunc(
mapstructure.StringToTimeDurationHookFunc(),
StringToQueryBackendDecodeHookFunc,
)
config := &mapstructure.DecoderConfig{
Metadata: nil,
Result: &queryMeta,
WeaklyTypedInput: true,
DecodeHook: decodeHooks,
}
decoder, err := mapstructure.NewDecoder(config)
if err != nil {
return queryMeta, err
}
err = decoder.Decode(m)
if err != nil {
return queryMeta, err
}
return queryMeta, nil
}
// GRPCMetadataFromQueryMeta returns a metadata struct with fields from the structs.QueryMeta attached.
// The return value is suitable for attaching to a gRPC header/trailer.
func GRPCMetadataFromQueryMeta(queryMeta structs.QueryMeta) (metadata.MD, error) {
md := metadata.MD{}
m := map[string]any{}
err := mapstructure.Decode(queryMeta, &m)
if err != nil {
return nil, err
}
for k, v := range m {
md.Set(k, fmt.Sprintf("%v", v))
}
return md, nil
}

View File

@ -0,0 +1,35 @@
package external
import (
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/structs"
)
func TestQueryMetaFromGRPCMetaRoundTrip(t *testing.T) {
lastContact, err := time.ParseDuration("1s")
require.NoError(t, err)
expected := structs.QueryMeta{
Index: 42,
LastContact: lastContact,
KnownLeader: true,
ConsistencyLevel: "stale",
NotModified: true,
Backend: structs.QueryBackend(0),
ResultsFilteredByACLs: true,
}
md, err := GRPCMetadataFromQueryMeta(expected)
require.NoError(t, err)
actual, err := QueryMetaFromGRPCMeta(md)
if err != nil {
t.Fatal(err)
}
require.Equal(t, expected, actual)
}

View File

@ -8,6 +8,9 @@ import (
"net/http" "net/http"
"strings" "strings"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
external "github.com/hashicorp/consul/agent/grpc-external" external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
@ -49,12 +52,14 @@ func (s *HTTPHandlers) peeringRead(resp http.ResponseWriter, req *http.Request,
var dc string var dc string
options := structs.QueryOptions{} options := structs.QueryOptions{}
s.parse(resp, req, &dc, &options) s.parse(resp, req, &dc, &options)
options.AllowStale = false // To get all information on a peering, this request must be forward to a leader
ctx, err := external.ContextWithQueryOptions(req.Context(), options) ctx, err := external.ContextWithQueryOptions(req.Context(), options)
if err != nil { if err != nil {
return nil, err return nil, err
} }
result, err := s.agent.rpcClientPeering.PeeringRead(ctx, &args) var header metadata.MD
result, err := s.agent.rpcClientPeering.PeeringRead(ctx, &args, grpc.Header(&header))
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -62,6 +67,14 @@ func (s *HTTPHandlers) peeringRead(resp http.ResponseWriter, req *http.Request,
return nil, HTTPError{StatusCode: http.StatusNotFound, Reason: fmt.Sprintf("Peering not found for %q", name)} return nil, HTTPError{StatusCode: http.StatusNotFound, Reason: fmt.Sprintf("Peering not found for %q", name)}
} }
meta, err := external.QueryMetaFromGRPCMeta(header)
if err != nil {
return result.Peering.ToAPI(), fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
}
if err := setMeta(resp, &meta); err != nil {
return nil, err
}
return result.Peering.ToAPI(), nil return result.Peering.ToAPI(), nil
} }
@ -78,16 +91,26 @@ func (s *HTTPHandlers) PeeringList(resp http.ResponseWriter, req *http.Request)
var dc string var dc string
options := structs.QueryOptions{} options := structs.QueryOptions{}
s.parse(resp, req, &dc, &options) s.parse(resp, req, &dc, &options)
options.AllowStale = false // To get all information on a peering, this request must be forward to a leader
ctx, err := external.ContextWithQueryOptions(req.Context(), options) ctx, err := external.ContextWithQueryOptions(req.Context(), options)
if err != nil { if err != nil {
return nil, err return nil, err
} }
pbresp, err := s.agent.rpcClientPeering.PeeringList(ctx, &args) var header metadata.MD
pbresp, err := s.agent.rpcClientPeering.PeeringList(ctx, &args, grpc.Header(&header))
if err != nil { if err != nil {
return nil, err return nil, err
} }
meta, err := external.QueryMetaFromGRPCMeta(header)
if err != nil {
return pbresp.ToAPI(), fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
}
if err := setMeta(resp, &meta); err != nil {
return nil, err
}
return pbresp.ToAPI(), nil return pbresp.ToAPI(), nil
} }

View File

@ -563,6 +563,13 @@ func TestHTTP_Peering_Read(t *testing.T) {
a.srv.h.ServeHTTP(resp, req) a.srv.h.ServeHTTP(resp, req)
require.Equal(t, http.StatusOK, resp.Code) require.Equal(t, http.StatusOK, resp.Code)
httpResult := resp.Result()
_, ok := httpResult.Header["X-Consul-Index"]
require.True(t, ok)
idx, err := strconv.Atoi(httpResult.Header.Get("X-Consul-Index"))
require.NoError(t, err)
require.Greater(t, idx, 0) // the raft index is not deterministic at this point
var apiResp api.Peering var apiResp api.Peering
require.NoError(t, json.NewDecoder(resp.Body).Decode(&apiResp)) require.NoError(t, json.NewDecoder(resp.Body).Decode(&apiResp))
@ -692,6 +699,13 @@ func TestHTTP_Peering_List(t *testing.T) {
a.srv.h.ServeHTTP(resp, req) a.srv.h.ServeHTTP(resp, req)
require.Equal(t, http.StatusOK, resp.Code) require.Equal(t, http.StatusOK, resp.Code)
httpResult := resp.Result()
_, ok := httpResult.Header["X-Consul-Index"]
require.True(t, ok)
idx, err := strconv.Atoi(httpResult.Header.Get("X-Consul-Index"))
require.NoError(t, err)
require.Greater(t, idx, 0) // the raft index is not deterministic at this point
var apiResp []*api.Peering var apiResp []*api.Peering
require.NoError(t, json.NewDecoder(resp.Body).Decode(&apiResp)) require.NoError(t, json.NewDecoder(resp.Body).Decode(&apiResp))

View File

@ -52,7 +52,7 @@ func (s *serverPeeringList) Notify(ctx context.Context, req *cachetype.PeeringLi
return 0, nil, err return 0, nil, err
} }
return index, &pbpeering.PeeringListResponse{ return index, &pbpeering.PeeringListResponse{
Index: index, OBSOLETE_Index: index,
Peerings: peerings, Peerings: peerings,
}, nil }, nil
}, },

View File

@ -48,7 +48,7 @@ func TestServerPeeringList(t *testing.T) {
result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh) result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh)
require.Len(t, result.Peerings, 1) require.Len(t, result.Peerings, 1)
require.Equal(t, "peer-01", result.Peerings[0].Name) require.Equal(t, "peer-01", result.Peerings[0].Name)
require.Equal(t, index, result.Index) require.Equal(t, index, result.OBSOLETE_Index)
}) })
testutil.RunStep(t, "add peering", func(t *testing.T) { testutil.RunStep(t, "add peering", func(t *testing.T) {
@ -63,7 +63,7 @@ func TestServerPeeringList(t *testing.T) {
result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh) result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh)
require.Len(t, result.Peerings, 2) require.Len(t, result.Peerings, 2)
require.Equal(t, "peer-02", result.Peerings[1].Name) require.Equal(t, "peer-02", result.Peerings[1].Name)
require.Equal(t, index+1, result.Index) require.Equal(t, index+1, result.OBSOLETE_Index)
}) })
} }
@ -100,7 +100,7 @@ func TestServerPeeringList_ACLEnforcement(t *testing.T) {
result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh) result := getEventResult[*pbpeering.PeeringListResponse](t, eventCh)
require.Len(t, result.Peerings, 1) require.Len(t, result.Peerings, 1)
require.Equal(t, "peer-01", result.Peerings[0].Name) require.Equal(t, "peer-01", result.Peerings[0].Name)
require.Equal(t, index, result.Index) require.Equal(t, index, result.OBSOLETE_Index)
}) })
testutil.RunStep(t, "can't read", func(t *testing.T) { testutil.RunStep(t, "can't read", func(t *testing.T) {

View File

@ -59,7 +59,7 @@ func (s *serverTrustBundle) Notify(ctx context.Context, req *cachetype.TrustBund
return 0, nil, err return 0, nil, err
} }
return index, &pbpeering.TrustBundleReadResponse{ return index, &pbpeering.TrustBundleReadResponse{
Index: index, OBSOLETE_Index: index,
Bundle: bundle, Bundle: bundle,
}, nil }, nil
}, },
@ -116,7 +116,7 @@ func (s *serverTrustBundleList) Notify(ctx context.Context, req *cachetype.Trust
} }
return index, &pbpeering.TrustBundleListByServiceResponse{ return index, &pbpeering.TrustBundleListByServiceResponse{
Index: index, OBSOLETE_Index: index,
Bundles: bundles, Bundles: bundles,
}, nil }, nil
}, },

View File

@ -12,9 +12,10 @@ import (
"strings" "strings"
"time" "time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/consul/acl"
cachetype "github.com/hashicorp/consul/agent/cache-types" cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/hashicorp/consul/agent/proxycfg/internal/watch" "github.com/hashicorp/consul/agent/proxycfg/internal/watch"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
@ -615,9 +616,11 @@ func (s *handlerMeshGateway) handleUpdate(ctx context.Context, u UpdateEvent, sn
peerServers := make(map[string]PeerServersValue) peerServers := make(map[string]PeerServersValue)
for _, peering := range resp.Peerings { for _, peering := range resp.Peerings {
// We only need to keep track of outbound establish connections // We only need to keep track of outbound establish connections for mesh gateway.
// for mesh gateway. // We could also check for the peering status, but this requires a response from the leader
if !peering.ShouldDial() || !peering.IsActive() { // which holds the peerstream information. We want to allow stale reads so there could be peerings in
// a deleting or terminating state.
if !peering.ShouldDial() {
continue continue
} }

View File

@ -806,14 +806,13 @@ func TestConfigSnapshotPeeredMeshGateway(t testing.T, variant string, nsFn func(
CorrelationID: peerServersWatchID, CorrelationID: peerServersWatchID,
Result: &pbpeering.PeeringListResponse{ Result: &pbpeering.PeeringListResponse{
Peerings: []*pbpeering.Peering{ Peerings: []*pbpeering.Peering{
// Not active // Empty state should be included. This could result from a query being served by a follower.
{ {
Name: "peer-a", Name: "peer-a",
PeerServerName: connect.PeeringServerSAN("dc2", "f3f41279-001d-42bb-912e-f6103fb036b8"), PeerServerName: connect.PeeringServerSAN("dc2", "f3f41279-001d-42bb-912e-f6103fb036b8"),
PeerServerAddresses: []string{ PeerServerAddresses: []string{
"1.2.3.4:5200", "1.2.3.4:5200",
}, },
State: pbpeering.PeeringState_TERMINATED,
ModifyIndex: 2, ModifyIndex: 2,
}, },
// No server addresses, so this should only be accepting connections // No server addresses, so this should only be accepting connections

View File

@ -21,16 +21,16 @@ import (
"google.golang.org/protobuf/proto" "google.golang.org/protobuf/proto"
"google.golang.org/protobuf/types/known/timestamppb" "google.golang.org/protobuf/types/known/timestamppb"
"github.com/hashicorp/consul/lib/retry"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/acl/resolver" "github.com/hashicorp/consul/acl/resolver"
"github.com/hashicorp/consul/agent/blockingquery"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/consul/stream" "github.com/hashicorp/consul/agent/consul/stream"
external "github.com/hashicorp/consul/agent/grpc-external" external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream" "github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/lib/retry"
"github.com/hashicorp/consul/proto/private/pbcommon" "github.com/hashicorp/consul/proto/private/pbcommon"
"github.com/hashicorp/consul/proto/private/pbpeering" "github.com/hashicorp/consul/proto/private/pbpeering"
"github.com/hashicorp/consul/proto/private/pbpeerstream" "github.com/hashicorp/consul/proto/private/pbpeerstream"
@ -72,11 +72,13 @@ var writeRequest struct {
structs.DCSpecificRequest structs.DCSpecificRequest
} }
var readRequest struct { type readRequest struct {
structs.QueryOptions structs.QueryOptions
structs.DCSpecificRequest structs.DCSpecificRequest
} }
var emptyDCSpecificRequest structs.DCSpecificRequest
// Server implements pbpeering.PeeringService to provide RPC operations for // Server implements pbpeering.PeeringService to provide RPC operations for
// managing peering relationships. // managing peering relationships.
type Server struct { type Server struct {
@ -92,6 +94,9 @@ type Config struct {
ConnectEnabled bool ConnectEnabled bool
PeeringEnabled bool PeeringEnabled bool
Locality *structs.Locality Locality *structs.Locality
// Needed because the stateful components needed to handle blocking queries are mixed in with server goo
FSMServer blockingquery.FSMServer
} }
func NewServer(cfg Config) *Server { func NewServer(cfg Config) *Server {
@ -99,6 +104,7 @@ func NewServer(cfg Config) *Server {
requireNotNil(cfg.Tracker, "Tracker") requireNotNil(cfg.Tracker, "Tracker")
requireNotNil(cfg.Logger, "Logger") requireNotNil(cfg.Logger, "Logger")
requireNotNil(cfg.ForwardRPC, "ForwardRPC") requireNotNil(cfg.ForwardRPC, "ForwardRPC")
requireNotNil(cfg.FSMServer, "FSMServer")
if cfg.Datacenter == "" { if cfg.Datacenter == "" {
panic("Datacenter is required") panic("Datacenter is required")
} }
@ -595,7 +601,10 @@ func retryExchange(ctx context.Context, req *pbpeerstream.ExchangeSecretRequest,
return resp, dialErrors return resp, dialErrors
} }
// OPTIMIZE: Handle blocking queries // PeeringRead returns the peering of the requested name and partition (enterprise only).
// Note that for the purposes of the blocking query, changes are only observed as part of the
// storage Index, which does not include the hydrated state from reconcilePeering, including
// the Active state and the count of imported/exported services.
func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequest) (*pbpeering.PeeringReadResponse, error) { func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequest) (*pbpeering.PeeringReadResponse, error) {
if !s.Config.PeeringEnabled { if !s.Config.PeeringEnabled {
return nil, peeringNotEnabledErr return nil, peeringNotEnabledErr
@ -605,8 +614,13 @@ func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequ
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
var resp *pbpeering.PeeringReadResponse var resp *pbpeering.PeeringReadResponse
handled, err := s.ForwardRPC(&readRequest, func(conn *grpc.ClientConn) error { handled, err := s.ForwardRPC(&readRequest{options, emptyDCSpecificRequest}, func(conn *grpc.ClientConn) error {
ctx := external.ForwardMetadataContext(ctx) ctx := external.ForwardMetadataContext(ctx)
var err error var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringRead(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringRead(ctx, req)
@ -620,10 +634,7 @@ func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequ
var authzCtx acl.AuthorizerContext var authzCtx acl.AuthorizerContext
entMeta := structs.DefaultEnterpriseMetaInPartition(req.Partition) entMeta := structs.DefaultEnterpriseMetaInPartition(req.Partition)
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, entMeta, &authzCtx) authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, entMeta, &authzCtx)
if err != nil { if err != nil {
return nil, err return nil, err
@ -633,23 +644,45 @@ func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequ
return nil, err return nil, err
} }
res := &pbpeering.PeeringReadResponse{}
meta := structs.QueryMeta{}
err = blockingquery.Query(s.FSMServer, &options, &meta, func(ws memdb.WatchSet, store *state.Store) error {
q := state.Query{ q := state.Query{
Value: strings.ToLower(req.Name), Value: strings.ToLower(req.Name),
EnterpriseMeta: *entMeta, EnterpriseMeta: *entMeta,
} }
_, peering, err := s.Backend.Store().PeeringRead(nil, q) idx, peering, err := store.PeeringRead(ws, q)
if err != nil { if err != nil {
return nil, err return err
} }
meta.SetIndex(idx)
if peering == nil { if peering == nil {
return &pbpeering.PeeringReadResponse{Peering: nil}, nil return blockingquery.ErrNotFound
} }
cp := s.reconcilePeering(peering) res.Peering = s.reconcilePeering(peering)
return &pbpeering.PeeringReadResponse{Peering: cp}, nil return nil
})
if err != nil {
return nil, fmt.Errorf("error executing peering read blocking query: %w", err)
} }
// OPTIMIZE: Handle blocking queries header, err := external.GRPCMetadataFromQueryMeta(meta)
if err != nil {
return nil, fmt.Errorf("could not convert query metadata to gRPC header")
}
if err := grpc.SendHeader(ctx, header); err != nil {
return nil, fmt.Errorf("could not send gRPC header")
}
return res, nil
}
// PeeringList returns the list of peerings in the requested partition(s) (enterprise only).
// Note that for the purposes of the blocking query, changes are only observed as part of the
// storage Index, which does not include the hydrated state from reconcilePeering, including
// the Active state and the count of imported/exported services.
func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequest) (*pbpeering.PeeringListResponse, error) { func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequest) (*pbpeering.PeeringListResponse, error) {
if !s.Config.PeeringEnabled { if !s.Config.PeeringEnabled {
return nil, peeringNotEnabledErr return nil, peeringNotEnabledErr
@ -659,8 +692,13 @@ func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequ
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
var resp *pbpeering.PeeringListResponse var resp *pbpeering.PeeringListResponse
handled, err := s.ForwardRPC(&readRequest, func(conn *grpc.ClientConn) error { handled, err := s.ForwardRPC(&readRequest{options, emptyDCSpecificRequest}, func(conn *grpc.ClientConn) error {
ctx := external.ForwardMetadataContext(ctx) ctx := external.ForwardMetadataContext(ctx)
var err error var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringList(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringList(ctx, req)
@ -672,10 +710,6 @@ func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequ
var authzCtx acl.AuthorizerContext var authzCtx acl.AuthorizerContext
entMeta := structs.DefaultEnterpriseMetaInPartition(req.Partition) entMeta := structs.DefaultEnterpriseMetaInPartition(req.Partition)
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, entMeta, &authzCtx) authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, entMeta, &authzCtx)
if err != nil { if err != nil {
@ -688,9 +722,12 @@ func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequ
defer metrics.MeasureSince([]string{"peering", "list"}, time.Now()) defer metrics.MeasureSince([]string{"peering", "list"}, time.Now())
idx, peerings, err := s.Backend.Store().PeeringList(nil, *entMeta) res := &pbpeering.PeeringListResponse{}
meta := structs.QueryMeta{}
err = blockingquery.Query(s.FSMServer, &options, &meta, func(ws memdb.WatchSet, store *state.Store) error {
idx, peerings, err := store.PeeringList(ws, *entMeta)
if err != nil { if err != nil {
return nil, err return err
} }
// reconcile the actual peering state; need to copy over the ds for peering // reconcile the actual peering state; need to copy over the ds for peering
@ -700,7 +737,25 @@ func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequ
cPeerings = append(cPeerings, cp) cPeerings = append(cPeerings, cp)
} }
return &pbpeering.PeeringListResponse{Peerings: cPeerings, Index: idx}, nil res.Peerings = cPeerings
meta.SetIndex(idx)
res.OBSOLETE_Index = idx // Compatibility with 1.14 API, deprecate in future release
return nil
})
if err != nil {
return nil, fmt.Errorf("error executing peering list blocking query: %w", err)
}
header, err := external.GRPCMetadataFromQueryMeta(meta)
if err != nil {
return nil, fmt.Errorf("could not convert query metadata to gRPC header")
}
if err := grpc.SendHeader(ctx, header); err != nil {
return nil, fmt.Errorf("could not send gRPC header")
}
return res, nil
} }
// TODO(peering): Get rid of this func when we stop using the stream tracker for imported/ exported services and the peering state // TODO(peering): Get rid of this func when we stop using the stream tracker for imported/ exported services and the peering state
@ -899,7 +954,6 @@ func (s *Server) PeeringDelete(ctx context.Context, req *pbpeering.PeeringDelete
return &pbpeering.PeeringDeleteResponse{}, nil return &pbpeering.PeeringDeleteResponse{}, nil
} }
// OPTIMIZE: Handle blocking queries
func (s *Server) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundleReadRequest) (*pbpeering.TrustBundleReadResponse, error) { func (s *Server) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundleReadRequest) (*pbpeering.TrustBundleReadResponse, error) {
if !s.Config.PeeringEnabled { if !s.Config.PeeringEnabled {
return nil, peeringNotEnabledErr return nil, peeringNotEnabledErr
@ -915,7 +969,7 @@ func (s *Server) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundle
} }
var resp *pbpeering.TrustBundleReadResponse var resp *pbpeering.TrustBundleReadResponse
handled, err := s.ForwardRPC(&readRequest, func(conn *grpc.ClientConn) error { handled, err := s.ForwardRPC(&readRequest{options, emptyDCSpecificRequest}, func(conn *grpc.ClientConn) error {
ctx := external.ForwardMetadataContext(ctx) ctx := external.ForwardMetadataContext(ctx)
var err error var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleRead(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleRead(ctx, req)
@ -941,22 +995,43 @@ func (s *Server) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundle
return nil, err return nil, err
} }
idx, trustBundle, err := s.Backend.Store().PeeringTrustBundleRead(nil, state.Query{ res := &pbpeering.TrustBundleReadResponse{}
meta := structs.QueryMeta{}
err = blockingquery.Query(s.FSMServer, &options, &meta, func(ws memdb.WatchSet, store *state.Store) error {
idx, trustBundle, err := store.PeeringTrustBundleRead(ws, state.Query{
Value: req.Name, Value: req.Name,
EnterpriseMeta: entMeta, EnterpriseMeta: entMeta,
}) })
if err != nil { if err != nil {
return nil, fmt.Errorf("failed to read trust bundle for peer %s: %w", req.Name, err) return fmt.Errorf("failed to read trust bundle for peer %s: %w", req.Name, err)
} }
return &pbpeering.TrustBundleReadResponse{ meta.SetIndex(idx)
Index: idx, if trustBundle == nil {
Bundle: trustBundle, return blockingquery.ErrNotFound
}, nil }
res.Bundle = trustBundle
res.OBSOLETE_Index = idx // Compatibility with 1.14 API, deprecate in future release
return nil
})
if err != nil {
return nil, fmt.Errorf("error executing trust bundle read blocking query: %w", err)
}
header, err := external.GRPCMetadataFromQueryMeta(meta)
if err != nil {
return nil, fmt.Errorf("could not convert query metadata to gRPC header")
}
if err := grpc.SendHeader(ctx, header); err != nil {
return nil, fmt.Errorf("could not send gRPC header")
}
return res, nil
} }
// TODO(peering): rename rpc & request/response to drop the "service" part // TODO(peering): rename rpc & request/response to drop the "service" part
// OPTIMIZE: Handle blocking queries
func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.TrustBundleListByServiceRequest) (*pbpeering.TrustBundleListByServiceResponse, error) { func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.TrustBundleListByServiceRequest) (*pbpeering.TrustBundleListByServiceResponse, error) {
if !s.Config.PeeringEnabled { if !s.Config.PeeringEnabled {
return nil, peeringNotEnabledErr return nil, peeringNotEnabledErr
@ -972,8 +1047,13 @@ func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.Tr
return nil, errors.New("missing service name") return nil, errors.New("missing service name")
} }
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
var resp *pbpeering.TrustBundleListByServiceResponse var resp *pbpeering.TrustBundleListByServiceResponse
handled, err := s.ForwardRPC(&readRequest, func(conn *grpc.ClientConn) error { handled, err := s.ForwardRPC(&readRequest{options, emptyDCSpecificRequest}, func(conn *grpc.ClientConn) error {
ctx := external.ForwardMetadataContext(ctx) ctx := external.ForwardMetadataContext(ctx)
var err error var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleListByService(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleListByService(ctx, req)
@ -987,10 +1067,6 @@ func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.Tr
var authzCtx acl.AuthorizerContext var authzCtx acl.AuthorizerContext
entMeta := acl.NewEnterpriseMetaWithPartition(req.Partition, req.Namespace) entMeta := acl.NewEnterpriseMetaWithPartition(req.Partition, req.Namespace)
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, err
}
authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, &entMeta, &authzCtx) authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, &entMeta, &authzCtx)
if err != nil { if err != nil {
@ -1001,26 +1077,47 @@ func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.Tr
return nil, err return nil, err
} }
res := &pbpeering.TrustBundleListByServiceResponse{}
meta := structs.QueryMeta{}
err = blockingquery.Query(s.FSMServer, &options, &meta, func(ws memdb.WatchSet, store *state.Store) error {
var ( var (
idx uint64 idx uint64
bundles []*pbpeering.PeeringTrustBundle bundles []*pbpeering.PeeringTrustBundle
) )
switch { switch {
case req.Kind == string(structs.ServiceKindMeshGateway): case req.Kind == string(structs.ServiceKindMeshGateway):
idx, bundles, err = s.Backend.Store().PeeringTrustBundleList(nil, entMeta) idx, bundles, err = store.PeeringTrustBundleList(ws, entMeta)
case req.ServiceName != "": case req.ServiceName != "":
idx, bundles, err = s.Backend.Store().TrustBundleListByService(nil, req.ServiceName, s.Datacenter, entMeta) idx, bundles, err = store.TrustBundleListByService(ws, req.ServiceName, s.Datacenter, entMeta)
case req.Kind != "": case req.Kind != "":
return nil, grpcstatus.Error(codes.InvalidArgument, "kind must be mesh-gateway if set") return grpcstatus.Error(codes.InvalidArgument, "kind must be mesh-gateway if set")
default: default:
return nil, grpcstatus.Error(codes.InvalidArgument, "one of service or kind is required") return grpcstatus.Error(codes.InvalidArgument, "one of service or kind is required")
} }
if err != nil { if err != nil {
return nil, err return fmt.Errorf("error listing trust bundles from store: %w", err)
} }
return &pbpeering.TrustBundleListByServiceResponse{Index: idx, Bundles: bundles}, nil
res.Bundles = bundles
meta.SetIndex(idx)
res.OBSOLETE_Index = idx // Compatibility with 1.14 API, deprecate in future release
return nil
})
if err != nil {
return nil, fmt.Errorf("error executing trust bundle list blocking query: %w", err)
}
header, err := external.GRPCMetadataFromQueryMeta(meta)
if err != nil {
return nil, fmt.Errorf("could not convert query metadata to gRPC header")
}
if err := grpc.SendHeader(ctx, header); err != nil {
return nil, fmt.Errorf("could not send gRPC header")
}
return res, nil
} }
func (s *Server) getExistingPeering(peerName, partition string) (*pbpeering.Peering, error) { func (s *Server) getExistingPeering(peerName, partition string) (*pbpeering.Peering, error) {

View File

@ -21,7 +21,9 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
gogrpc "google.golang.org/grpc" gogrpc "google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
grpcstatus "google.golang.org/grpc/status" grpcstatus "google.golang.org/grpc/status"
"google.golang.org/protobuf/proto"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
@ -861,6 +863,64 @@ func TestPeeringService_Read_ACLEnforcement(t *testing.T) {
} }
} }
func TestPeeringService_Read_Blocking(t *testing.T) {
// TODO(peering): see note on newTestServer, refactor to not use this
s := newTestServer(t, nil)
// insert peering directly to state store
lastIdx := uint64(10)
p := &pbpeering.Peering{
ID: testUUID(t),
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerCAPems: nil,
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
}
err := s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: p})
require.NoError(t, err)
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
// Setup blocking query
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
options := structs.QueryOptions{
MinQueryIndex: lastIdx,
MaxQueryTime: 1 * time.Second,
}
ctx, err = external.ContextWithQueryOptions(ctx, options)
require.NoError(t, err)
// Mutate the original peering
p = proto.Clone(p).(*pbpeering.Peering)
p.PeerServerAddresses = append(p.PeerServerAddresses, "addr2")
// Async change to trigger update
marker := time.Now()
go func() {
time.Sleep(100 * time.Millisecond)
lastIdx++
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: p}))
}()
var header metadata.MD
resp, err := client.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "foo"}, gogrpc.Header(&header))
require.NoError(t, err)
// The query should return after the async change, but before the timeout
require.True(t, time.Since(marker) >= 100*time.Millisecond)
require.True(t, time.Since(marker) < 1*time.Second)
// Verify query results
meta, err := external.QueryMetaFromGRPCMeta(header)
require.NoError(t, err)
require.Equal(t, lastIdx, meta.Index)
prototest.AssertDeepEqual(t, p, resp.Peering)
}
func TestPeeringService_Delete(t *testing.T) { func TestPeeringService_Delete(t *testing.T) {
tt := map[string]pbpeering.PeeringState{ tt := map[string]pbpeering.PeeringState{
"active peering": pbpeering.PeeringState_ACTIVE, "active peering": pbpeering.PeeringState_ACTIVE,
@ -993,6 +1053,7 @@ func TestPeeringService_List(t *testing.T) {
// Insert peerings directly to state store. // Insert peerings directly to state store.
// Note that the state store holds reference to the underlying // Note that the state store holds reference to the underlying
// variables; do not modify them after writing. // variables; do not modify them after writing.
lastIdx := uint64(10)
foo := &pbpeering.Peering{ foo := &pbpeering.Peering{
ID: testUUID(t), ID: testUUID(t),
Name: "foo", Name: "foo",
@ -1001,7 +1062,9 @@ func TestPeeringService_List(t *testing.T) {
PeerServerName: "fooservername", PeerServerName: "fooservername",
PeerServerAddresses: []string{"addr1"}, PeerServerAddresses: []string{"addr1"},
} }
require.NoError(t, s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: foo})) require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: foo}))
lastIdx++
bar := &pbpeering.Peering{ bar := &pbpeering.Peering{
ID: testUUID(t), ID: testUUID(t),
Name: "bar", Name: "bar",
@ -1010,21 +1073,78 @@ func TestPeeringService_List(t *testing.T) {
PeerServerName: "barservername", PeerServerName: "barservername",
PeerServerAddresses: []string{"addr1"}, PeerServerAddresses: []string{"addr1"},
} }
require.NoError(t, s.Server.FSM().State().PeeringWrite(15, &pbpeering.PeeringWriteRequest{Peering: bar})) require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: bar}))
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t)) client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
t.Run("non-blocking query", func(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel) t.Cleanup(cancel)
resp, err := client.PeeringList(ctx, &pbpeering.PeeringListRequest{}) var header metadata.MD
resp, err := client.PeeringList(ctx, &pbpeering.PeeringListRequest{}, gogrpc.Header(&header))
require.NoError(t, err) require.NoError(t, err)
meta, err := external.QueryMetaFromGRPCMeta(header)
require.NoError(t, err)
require.Equal(t, lastIdx, meta.Index)
expect := &pbpeering.PeeringListResponse{ expect := &pbpeering.PeeringListResponse{
Peerings: []*pbpeering.Peering{bar, foo}, Peerings: []*pbpeering.Peering{bar, foo},
Index: 15, OBSOLETE_Index: lastIdx,
} }
prototest.AssertDeepEqual(t, expect, resp) prototest.AssertDeepEqual(t, expect, resp)
})
t.Run("blocking query", func(t *testing.T) {
// Setup blocking query
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
marker := time.Now()
options := structs.QueryOptions{
MinQueryIndex: lastIdx,
MaxQueryTime: 1 * time.Second,
}
ctx, err := external.ContextWithQueryOptions(ctx, options)
require.NoError(t, err)
// Async change to trigger update
baz := &pbpeering.Peering{
ID: testUUID(t),
Name: "baz",
State: pbpeering.PeeringState_ACTIVE,
PeerCAPems: nil,
PeerServerName: "bazservername",
PeerServerAddresses: []string{"addr1"},
}
go func() {
time.Sleep(100 * time.Millisecond)
lastIdx++
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: baz}))
}()
// Make the blocking query
var header metadata.MD
resp, err := client.PeeringList(ctx, &pbpeering.PeeringListRequest{}, gogrpc.Header(&header))
require.NoError(t, err)
// The query should return after the async change, but before the timeout
require.True(t, time.Since(marker) >= 100*time.Millisecond)
require.True(t, time.Since(marker) < 1*time.Second)
// Verify query results
meta, err := external.QueryMetaFromGRPCMeta(header)
require.NoError(t, err)
require.Equal(t, lastIdx, meta.Index)
expect := &pbpeering.PeeringListResponse{
Peerings: []*pbpeering.Peering{bar, baz, foo},
OBSOLETE_Index: lastIdx,
}
prototest.AssertDeepEqual(t, expect, resp)
})
} }
func TestPeeringService_List_ACLEnforcement(t *testing.T) { func TestPeeringService_List_ACLEnforcement(t *testing.T) {
@ -1088,7 +1208,7 @@ func TestPeeringService_List_ACLEnforcement(t *testing.T) {
token: testTokenPeeringReadSecret, token: testTokenPeeringReadSecret,
expect: &pbpeering.PeeringListResponse{ expect: &pbpeering.PeeringListResponse{
Peerings: []*pbpeering.Peering{bar, foo}, Peerings: []*pbpeering.Peering{bar, foo},
Index: 15, OBSOLETE_Index: 15,
}, },
}, },
} }
@ -1115,16 +1235,65 @@ func TestPeeringService_TrustBundleRead(t *testing.T) {
lastIdx++ lastIdx++
require.NoError(t, store.PeeringTrustBundleWrite(lastIdx, bundle)) require.NoError(t, store.PeeringTrustBundleWrite(lastIdx, bundle))
ctx, cancel := context.WithCancel(context.Background()) t.Run("non-blocking query", func(t *testing.T) {
defer cancel() ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
resp, err := client.TrustBundleRead(ctx, &pbpeering.TrustBundleReadRequest{ resp, err := client.TrustBundleRead(ctx, &pbpeering.TrustBundleReadRequest{
Name: "my-peering", Name: "my-peering",
}) })
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, lastIdx, resp.Index) require.Equal(t, lastIdx, resp.OBSOLETE_Index)
require.NotNil(t, resp.Bundle) require.NotNil(t, resp.Bundle)
prototest.AssertDeepEqual(t, bundle, resp.Bundle) prototest.AssertDeepEqual(t, bundle, resp.Bundle)
})
t.Run("blocking query", func(t *testing.T) {
// Set up the blocking query
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
marker := time.Now()
options := structs.QueryOptions{
MinQueryIndex: lastIdx,
MaxQueryTime: 1 * time.Second,
}
ctx, err := external.ContextWithQueryOptions(ctx, options)
require.NoError(t, err)
updatedBundle := &pbpeering.PeeringTrustBundle{
TrustDomain: "peer1.com",
PeerName: "my-peering",
RootPEMs: []string{"peer1-root-1", "peer1-root-2"}, // Adding a CA here
}
// Async change to trigger update
go func() {
time.Sleep(100 * time.Millisecond)
lastIdx++
require.NoError(t, store.PeeringTrustBundleWrite(lastIdx, updatedBundle))
}()
// Make the blocking query
var header metadata.MD
resp, err := client.TrustBundleRead(ctx, &pbpeering.TrustBundleReadRequest{
Name: "my-peering",
}, gogrpc.Header(&header))
require.NoError(t, err)
// The query should return after the async change, but before the timeout
require.True(t, time.Since(marker) >= 100*time.Millisecond)
require.True(t, time.Since(marker) < 1*time.Second)
// Verify query results
meta, err := external.QueryMetaFromGRPCMeta(header)
require.NoError(t, err)
require.Equal(t, lastIdx, meta.Index)
require.Equal(t, lastIdx, resp.OBSOLETE_Index)
require.NotNil(t, resp.Bundle)
prototest.AssertDeepEqual(t, updatedBundle, resp.Bundle)
})
} }
func TestPeeringService_TrustBundleRead_ACLEnforcement(t *testing.T) { func TestPeeringService_TrustBundleRead_ACLEnforcement(t *testing.T) {
@ -1290,6 +1459,7 @@ func TestPeeringService_TrustBundleListByService(t *testing.T) {
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t)) client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
t.Run("non-blocking query", func(t *testing.T) {
req := pbpeering.TrustBundleListByServiceRequest{ req := pbpeering.TrustBundleListByServiceRequest{
ServiceName: "api", ServiceName: "api",
} }
@ -1298,6 +1468,55 @@ func TestPeeringService_TrustBundleListByService(t *testing.T) {
require.Len(t, resp.Bundles, 2) require.Len(t, resp.Bundles, 2)
require.Equal(t, []string{"bar-root-1"}, resp.Bundles[0].RootPEMs) require.Equal(t, []string{"bar-root-1"}, resp.Bundles[0].RootPEMs)
require.Equal(t, []string{"foo-root-1"}, resp.Bundles[1].RootPEMs) require.Equal(t, []string{"foo-root-1"}, resp.Bundles[1].RootPEMs)
require.Equal(t, uint64(17), resp.OBSOLETE_Index)
})
t.Run("blocking query", func(t *testing.T) {
// Setup blocking query
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
options := structs.QueryOptions{
MinQueryIndex: lastIdx,
MaxQueryTime: 1 * time.Second,
}
ctx, err := external.ContextWithQueryOptions(ctx, options)
require.NoError(t, err)
// Async change to trigger update
marker := time.Now()
go func() {
time.Sleep(100 * time.Millisecond)
lastIdx++
require.NoError(t, store.PeeringTrustBundleWrite(lastIdx, &pbpeering.PeeringTrustBundle{
TrustDomain: "bar.com",
PeerName: "bar",
RootPEMs: []string{"bar-root-1", "bar-root-2"}, // Appending new cert
}))
}()
// Make the blocking query
req := pbpeering.TrustBundleListByServiceRequest{
ServiceName: "api",
}
var header metadata.MD
resp, err := client.TrustBundleListByService(ctx, &req, gogrpc.Header(&header))
require.NoError(t, err)
// The query should return after the async change, but before the timeout
require.True(t, time.Since(marker) >= 100*time.Millisecond)
require.True(t, time.Since(marker) < 1*time.Second)
// Verify query results
meta, err := external.QueryMetaFromGRPCMeta(header)
require.NoError(t, err)
require.Equal(t, uint64(18), meta.Index)
require.Len(t, resp.Bundles, 2)
require.Equal(t, []string{"bar-root-1", "bar-root-2"}, resp.Bundles[0].RootPEMs)
require.Equal(t, []string{"foo-root-1"}, resp.Bundles[1].RootPEMs)
require.Equal(t, uint64(18), resp.OBSOLETE_Index)
})
} }
func TestPeeringService_validatePeer(t *testing.T) { func TestPeeringService_validatePeer(t *testing.T) {

View File

@ -420,6 +420,17 @@ func (q QueryBackend) String() string {
} }
} }
func QueryBackendFromString(s string) QueryBackend {
switch s {
case "blocking-query":
return QueryBackendBlocking
case "streaming":
return QueryBackendStreaming
default:
return QueryBackendBlocking
}
}
// QueryMeta allows a query response to include potentially // QueryMeta allows a query response to include potentially
// useful metadata about a query // useful metadata about a query
type QueryMeta struct { type QueryMeta struct {

View File

@ -7,17 +7,26 @@
"type": "EDS", "type": "EDS",
"edsClusterConfig": { "edsClusterConfig": {
"edsConfig": { "edsConfig": {
"ads": { "ads": {},
},
"resourceApiVersion": "V3" "resourceApiVersion": "V3"
} }
}, },
"connectTimeout": "5s", "connectTimeout": "5s",
"outlierDetection": { "outlierDetection": {}
},
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "server.dc2.peering.f3f41279-001d-42bb-912e-f6103fb036b8",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {},
"resourceApiVersion": "V3"
} }
}, },
"connectTimeout": "5s",
"outlierDetection": {}
},
{ {
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster", "@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "server.dc3.peering.f622dc37-7238-4485-ab58-0f53864a9ae5", "name": "server.dc3.peering.f622dc37-7238-4485-ab58-0f53864a9ae5",
@ -44,9 +53,7 @@
}, },
"dnsRefreshRate": "10s", "dnsRefreshRate": "10s",
"dnsLookupFamily": "V4_ONLY", "dnsLookupFamily": "V4_ONLY",
"outlierDetection": { "outlierDetection": {}
}
} }
], ],
"typeUrl": "type.googleapis.com/envoy.config.cluster.v3.Cluster", "typeUrl": "type.googleapis.com/envoy.config.cluster.v3.Cluster",

View File

@ -30,6 +30,26 @@
] ]
} }
] ]
},
{
"@type": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
"clusterName": "server.dc2.peering.f3f41279-001d-42bb-912e-f6103fb036b8",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "1.2.3.4",
"portValue": 5200
}
}
}
}
]
}
]
} }
], ],
"typeUrl": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", "typeUrl": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",

View File

@ -28,6 +28,23 @@
} }
] ]
}, },
{
"filterChainMatch": {
"serverNames": [
"server.dc2.peering.f3f41279-001d-42bb-912e-f6103fb036b8"
]
},
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"statPrefix": "mesh_gateway_remote_peering_servers.server.dc2.peering.f3f41279-001d-42bb-912e-f6103fb036b8",
"cluster": "server.dc2.peering.f3f41279-001d-42bb-912e-f6103fb036b8"
}
}
]
},
{ {
"filterChainMatch": { "filterChainMatch": {
"serverNames": [ "serverNames": [

View File

@ -41,21 +41,6 @@ type ExampleDatacenter struct {
Datacenter string Datacenter string
} }
// @consul-rpc-glue: ReadTODO
type ExampleReadTODO struct {
Value string
}
// @consul-rpc-glue: LeaderReadTODO
type ExampleLeaderReadTODO struct {
Value string
}
// @consul-rpc-glue: WriteTODO
type ExampleWriteTODO struct {
Value string
}
// @consul-rpc-glue: WriteRequest=AltWriteRequest // @consul-rpc-glue: WriteRequest=AltWriteRequest
type AltExampleWriteRequest struct { type AltExampleWriteRequest struct {
Value int Value int

View File

@ -1,6 +1,3 @@
//go:build example
// +build example
// Code generated by proto-gen-rpc-glue. DO NOT EDIT. // Code generated by proto-gen-rpc-glue. DO NOT EDIT.
package e2e package e2e
@ -28,14 +25,6 @@ func (msg *ExampleWriteRequest) HasTimedOut(start time.Time, rpcHoldTimeout time
return msg.WriteRequest.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.WriteRequest.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *ExampleWriteRequest) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.WriteRequest == nil {
return 0
}
return msg.WriteRequest.Timeout(rpcHoldTimeout, a, b)
}
// IsRead implements structs.RPCInfo // IsRead implements structs.RPCInfo
func (msg *ExampleWriteRequest) IsRead() bool { func (msg *ExampleWriteRequest) IsRead() bool {
return false return false
@ -90,14 +79,6 @@ func (msg *ExampleReadRequest) HasTimedOut(start time.Time, rpcHoldTimeout time.
return msg.ReadRequest.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.ReadRequest.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *ExampleReadRequest) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.ReadRequest == nil {
return 0
}
return msg.ReadRequest.Timeout(rpcHoldTimeout, a, b)
}
// SetTokenSecret implements structs.RPCInfo // SetTokenSecret implements structs.RPCInfo
func (msg *ExampleReadRequest) SetTokenSecret(s string) { func (msg *ExampleReadRequest) SetTokenSecret(s string) {
// TODO: initialize if nil // TODO: initialize if nil
@ -146,6 +127,16 @@ func (msg *ExampleQueryOptions) AllowStaleRead() bool {
return msg.QueryOptions.AllowStaleRead() return msg.QueryOptions.AllowStaleRead()
} }
// BlockingTimeout implements pool.BlockableQuery
func (msg *ExampleQueryOptions) BlockingTimeout(maxQueryTime, defaultQueryTime time.Duration) time.Duration {
maxTime := structs.DurationFromProto(msg.QueryOptions.GetMaxQueryTime())
o := structs.QueryOptions{
MaxQueryTime: maxTime,
MinQueryIndex: msg.QueryOptions.GetMinQueryIndex(),
}
return o.BlockingTimeout(maxQueryTime, defaultQueryTime)
}
// HasTimedOut implements structs.RPCInfo // HasTimedOut implements structs.RPCInfo
func (msg *ExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) { func (msg *ExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
if msg == nil || msg.QueryOptions == nil { if msg == nil || msg.QueryOptions == nil {
@ -154,14 +145,6 @@ func (msg *ExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout time
return msg.QueryOptions.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.QueryOptions.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *ExampleQueryOptions) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.QueryOptions == nil {
return 0
}
return msg.QueryOptions.Timeout(rpcHoldTimeout, a, b)
}
// SetTokenSecret implements structs.RPCInfo // SetTokenSecret implements structs.RPCInfo
func (msg *ExampleQueryOptions) SetTokenSecret(s string) { func (msg *ExampleQueryOptions) SetTokenSecret(s string) {
// TODO: initialize if nil // TODO: initialize if nil
@ -267,126 +250,6 @@ func (msg *ExampleDatacenter) RequestDatacenter() string {
return msg.Datacenter return msg.Datacenter
} }
// IsRead implements structs.RPCInfo
func (msg *ExampleReadTODO) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *ExampleReadTODO) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *ExampleReadTODO) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// Timeout implements structs.RPCInfo
func (msg *ExampleReadTODO) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
// TODO(peering): figure out read semantics here
return rpcHoldTimeout
}
// SetTokenSecret implements structs.RPCInfo
func (msg *ExampleReadTODO) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *ExampleReadTODO) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *ExampleReadTODO) Token() string {
// TODO(peering): figure out read semantics here
return ""
}
// IsRead implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
// TODO(peering): this needs to stay false for calls to head to the leader until we sync stream tracker information
// like ImportedServicesCount, ExportedServicesCount, as well as general Status fields thru raft to make available
// to followers as well
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// Timeout implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
// TODO(peering): figure out read semantics here
return rpcHoldTimeout
}
// SetTokenSecret implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *ExampleLeaderReadTODO) Token() string {
// TODO(peering): figure out read semantics here
return ""
}
// IsRead implements structs.RPCInfo
func (msg *ExampleWriteTODO) IsRead() bool {
// TODO(peering): figure out write semantics here
return false
}
// AllowStaleRead implements structs.RPCInfo
func (msg *ExampleWriteTODO) AllowStaleRead() bool {
// TODO(peering): figure out write semantics here
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *ExampleWriteTODO) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out write semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// Timeout implements structs.RPCInfo
func (msg *ExampleWriteTODO) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
// TODO(peering): figure out write semantics here
return rpcHoldTimeout
}
// SetTokenSecret implements structs.RPCInfo
func (msg *ExampleWriteTODO) SetTokenSecret(s string) {
// TODO(peering): figure out write semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *ExampleWriteTODO) TokenSecret() string {
// TODO(peering): figure out write semantics here
return ""
}
// AllowStaleRead implements structs.RPCInfo // AllowStaleRead implements structs.RPCInfo
func (msg *AltExampleWriteRequest) AllowStaleRead() bool { func (msg *AltExampleWriteRequest) AllowStaleRead() bool {
return false return false
@ -400,14 +263,6 @@ func (msg *AltExampleWriteRequest) HasTimedOut(start time.Time, rpcHoldTimeout t
return msg.AltWriteRequest.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.AltWriteRequest.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *AltExampleWriteRequest) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.AltWriteRequest == nil {
return 0
}
return msg.AltWriteRequest.Timeout(rpcHoldTimeout, a, b)
}
// IsRead implements structs.RPCInfo // IsRead implements structs.RPCInfo
func (msg *AltExampleWriteRequest) IsRead() bool { func (msg *AltExampleWriteRequest) IsRead() bool {
return false return false
@ -454,14 +309,6 @@ func (msg *AltExampleReadRequest) HasTimedOut(start time.Time, rpcHoldTimeout ti
return msg.AltReadRequest.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.AltReadRequest.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *AltExampleReadRequest) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.AltReadRequest == nil {
return 0
}
return msg.AltReadRequest.Timeout(rpcHoldTimeout, a, b)
}
// SetTokenSecret implements structs.RPCInfo // SetTokenSecret implements structs.RPCInfo
func (msg *AltExampleReadRequest) SetTokenSecret(s string) { func (msg *AltExampleReadRequest) SetTokenSecret(s string) {
// TODO: initialize if nil // TODO: initialize if nil
@ -494,6 +341,16 @@ func (msg *AltExampleQueryOptions) AllowStaleRead() bool {
return msg.AltQueryOptions.AllowStaleRead() return msg.AltQueryOptions.AllowStaleRead()
} }
// BlockingTimeout implements pool.BlockableQuery
func (msg *AltExampleQueryOptions) BlockingTimeout(maxQueryTime, defaultQueryTime time.Duration) time.Duration {
maxTime := structs.DurationFromProto(msg.AltQueryOptions.GetMaxQueryTime())
o := structs.QueryOptions{
MaxQueryTime: maxTime,
MinQueryIndex: msg.AltQueryOptions.GetMinQueryIndex(),
}
return o.BlockingTimeout(maxQueryTime, defaultQueryTime)
}
// HasTimedOut implements structs.RPCInfo // HasTimedOut implements structs.RPCInfo
func (msg *AltExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) { func (msg *AltExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
if msg == nil || msg.AltQueryOptions == nil { if msg == nil || msg.AltQueryOptions == nil {
@ -502,14 +359,6 @@ func (msg *AltExampleQueryOptions) HasTimedOut(start time.Time, rpcHoldTimeout t
return msg.AltQueryOptions.HasTimedOut(start, rpcHoldTimeout, a, b) return msg.AltQueryOptions.HasTimedOut(start, rpcHoldTimeout, a, b)
} }
// Timeout implements structs.RPCInfo
func (msg *AltExampleQueryOptions) Timeout(rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) time.Duration {
if msg == nil || msg.AltQueryOptions == nil {
return 0
}
return msg.AltQueryOptions.Timeout(rpcHoldTimeout, a, b)
}
// SetTokenSecret implements structs.RPCInfo // SetTokenSecret implements structs.RPCInfo
func (msg *AltExampleQueryOptions) SetTokenSecret(s string) { func (msg *AltExampleQueryOptions) SetTokenSecret(s string) {
// TODO: initialize if nil // TODO: initialize if nil

View File

@ -108,15 +108,6 @@ func processFile(path string) error {
if ann.Datacenter != "" { if ann.Datacenter != "" {
log.Printf(" Datacenter from %s", ann.Datacenter) log.Printf(" Datacenter from %s", ann.Datacenter)
} }
if ann.ReadTODO != "" {
log.Printf(" ReadTODO from %s", ann.ReadTODO)
}
if ann.LeaderReadTODO != "" {
log.Printf(" LeaderReadTODO from %s", ann.LeaderReadTODO)
}
if ann.WriteTODO != "" {
log.Printf(" WriteTODO from %s", ann.WriteTODO)
}
} }
} }
@ -163,15 +154,6 @@ var _ time.Month
if typ.Annotation.Datacenter != "" { if typ.Annotation.Datacenter != "" {
buf.WriteString(fmt.Sprintf(tmplDatacenter, typ.Name, typ.Annotation.Datacenter)) buf.WriteString(fmt.Sprintf(tmplDatacenter, typ.Name, typ.Annotation.Datacenter))
} }
if typ.Annotation.LeaderReadTODO != "" {
buf.WriteString(fmt.Sprintf(tmplLeaderOnlyReadTODO, typ.Name, typ.Annotation.LeaderReadTODO))
}
if typ.Annotation.ReadTODO != "" {
buf.WriteString(fmt.Sprintf(tmplReadTODO, typ.Name, typ.Annotation.ReadTODO))
}
if typ.Annotation.WriteTODO != "" {
buf.WriteString(fmt.Sprintf(tmplWriteTODO, typ.Name, typ.Annotation.WriteTODO))
}
} }
// write to disk // write to disk
@ -325,13 +307,6 @@ func getAnnotation(doc []*ast.Comment) (Annotation, error) {
case strings.HasPrefix(part, "Datacenter="): case strings.HasPrefix(part, "Datacenter="):
ann.Datacenter = strings.TrimPrefix(part, "Datacenter=") ann.Datacenter = strings.TrimPrefix(part, "Datacenter=")
case part == "ReadTODO":
ann.ReadTODO = "ReadTODO"
case part == "WriteTODO":
ann.WriteTODO = "WriteTODO"
case part == "LeaderReadTODO":
ann.LeaderReadTODO = "LeaderReadTODO"
default: default:
return Annotation{}, fmt.Errorf("unexpected annotation part: %s", part) return Annotation{}, fmt.Errorf("unexpected annotation part: %s", part)
} }
@ -459,114 +434,6 @@ func (msg *%[1]s) Token() string {
} }
` `
const tmplLeaderOnlyReadTODO = `
// IsRead implements structs.RPCInfo
func (msg *%[1]s) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *%[1]s) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
// TODO(peering): this needs to stay false for calls to head to the leader until we sync stream tracker information
// like ImportedServicesCount, ExportedServicesCount, as well as general Status fields thru raft to make available
// to followers as well
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *%[1]s) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// SetTokenSecret implements structs.RPCInfo
func (msg *%[1]s) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *%[1]s) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *%[1]s) Token() string {
// TODO(peering): figure out read semantics here
return ""
}
`
const tmplReadTODO = `
// IsRead implements structs.RPCInfo
func (msg *%[1]s) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *%[1]s) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *%[1]s) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// SetTokenSecret implements structs.RPCInfo
func (msg *%[1]s) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *%[1]s) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *%[1]s) Token() string {
// TODO(peering): figure out read semantics here
return ""
}
`
const tmplWriteTODO = `
// IsRead implements structs.RPCInfo
func (msg *%[1]s) IsRead() bool {
// TODO(peering): figure out write semantics here
return false
}
// AllowStaleRead implements structs.RPCInfo
func (msg *%[1]s) AllowStaleRead() bool {
// TODO(peering): figure out write semantics here
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *%[1]s) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out write semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// SetTokenSecret implements structs.RPCInfo
func (msg *%[1]s) SetTokenSecret(s string) {
// TODO(peering): figure out write semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *%[1]s) TokenSecret() string {
// TODO(peering): figure out write semantics here
return ""
}
`
const tmplTargetDatacenter = ` const tmplTargetDatacenter = `
// RequestDatacenter implements structs.RPCInfo // RequestDatacenter implements structs.RPCInfo
func (msg *%[1]s) RequestDatacenter() string { func (msg *%[1]s) RequestDatacenter() string {

View File

@ -792,7 +792,6 @@ func (x *PeeringServerAddresses) GetAddresses() []string {
return nil return nil
} }
// @consul-rpc-glue: LeaderReadTODO
type PeeringReadRequest struct { type PeeringReadRequest struct {
state protoimpl.MessageState state protoimpl.MessageState
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
@ -895,7 +894,6 @@ func (x *PeeringReadResponse) GetPeering() *Peering {
return nil return nil
} }
// @consul-rpc-glue: LeaderReadTODO
type PeeringListRequest struct { type PeeringListRequest struct {
state protoimpl.MessageState state protoimpl.MessageState
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
@ -949,7 +947,7 @@ type PeeringListResponse struct {
unknownFields protoimpl.UnknownFields unknownFields protoimpl.UnknownFields
Peerings []*Peering `protobuf:"bytes,1,rep,name=Peerings,proto3" json:"Peerings,omitempty"` Peerings []*Peering `protobuf:"bytes,1,rep,name=Peerings,proto3" json:"Peerings,omitempty"`
Index uint64 `protobuf:"varint,2,opt,name=Index,proto3" json:"Index,omitempty"` OBSOLETE_Index uint64 `protobuf:"varint,2,opt,name=OBSOLETE_Index,json=OBSOLETEIndex,proto3" json:"OBSOLETE_Index,omitempty"` // Deprecated in favor of gRPC metadata
} }
func (x *PeeringListResponse) Reset() { func (x *PeeringListResponse) Reset() {
@ -991,9 +989,9 @@ func (x *PeeringListResponse) GetPeerings() []*Peering {
return nil return nil
} }
func (x *PeeringListResponse) GetIndex() uint64 { func (x *PeeringListResponse) GetOBSOLETE_Index() uint64 {
if x != nil { if x != nil {
return x.Index return x.OBSOLETE_Index
} }
return 0 return 0
} }
@ -1273,7 +1271,7 @@ type TrustBundleListByServiceResponse struct {
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields unknownFields protoimpl.UnknownFields
Index uint64 `protobuf:"varint,1,opt,name=Index,proto3" json:"Index,omitempty"` OBSOLETE_Index uint64 `protobuf:"varint,1,opt,name=OBSOLETE_Index,json=OBSOLETEIndex,proto3" json:"OBSOLETE_Index,omitempty"` // Deprecated in favor of gRPC metadata
Bundles []*PeeringTrustBundle `protobuf:"bytes,2,rep,name=Bundles,proto3" json:"Bundles,omitempty"` Bundles []*PeeringTrustBundle `protobuf:"bytes,2,rep,name=Bundles,proto3" json:"Bundles,omitempty"`
} }
@ -1309,9 +1307,9 @@ func (*TrustBundleListByServiceResponse) Descriptor() ([]byte, []int) {
return file_private_pbpeering_peering_proto_rawDescGZIP(), []int{16} return file_private_pbpeering_peering_proto_rawDescGZIP(), []int{16}
} }
func (x *TrustBundleListByServiceResponse) GetIndex() uint64 { func (x *TrustBundleListByServiceResponse) GetOBSOLETE_Index() uint64 {
if x != nil { if x != nil {
return x.Index return x.OBSOLETE_Index
} }
return 0 return 0
} }
@ -1383,7 +1381,7 @@ type TrustBundleReadResponse struct {
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields unknownFields protoimpl.UnknownFields
Index uint64 `protobuf:"varint,1,opt,name=Index,proto3" json:"Index,omitempty"` OBSOLETE_Index uint64 `protobuf:"varint,1,opt,name=OBSOLETE_Index,json=OBSOLETEIndex,proto3" json:"OBSOLETE_Index,omitempty"` // Deprecated in favor of gRPC metadata
Bundle *PeeringTrustBundle `protobuf:"bytes,2,opt,name=Bundle,proto3" json:"Bundle,omitempty"` Bundle *PeeringTrustBundle `protobuf:"bytes,2,opt,name=Bundle,proto3" json:"Bundle,omitempty"`
} }
@ -1419,9 +1417,9 @@ func (*TrustBundleReadResponse) Descriptor() ([]byte, []int) {
return file_private_pbpeering_peering_proto_rawDescGZIP(), []int{18} return file_private_pbpeering_peering_proto_rawDescGZIP(), []int{18}
} }
func (x *TrustBundleReadResponse) GetIndex() uint64 { func (x *TrustBundleReadResponse) GetOBSOLETE_Index() uint64 {
if x != nil { if x != nil {
return x.Index return x.OBSOLETE_Index
} }
return 0 return 0
} }
@ -2485,234 +2483,237 @@ var file_private_pbpeering_peering_proto_rawDesc = []byte{
0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
0x6f, 0x6e, 0x22, 0x73, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x6f, 0x6e, 0x22, 0x84, 0x01, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69,
0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x08, 0x50, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x68, 0x61, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x68,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x73, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x67, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x4f, 0x42, 0x53, 0x4f, 0x4c, 0x45, 0x54, 0x45, 0x5f, 0x49,
0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xca, 0x02, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x4f, 0x42, 0x53, 0x4f,
0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4c, 0x45, 0x54, 0x45, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xca, 0x02, 0x0a, 0x13, 0x50, 0x65,
0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x32, 0x2a, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x12, 0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01,
0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63,
0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, 0x50, 0x65, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70,
0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x5e, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x5e, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65,
0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x67, 0x2e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72,
0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x52, 0x65, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73,
0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18,
0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65,
0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a,
0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,
0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05,
0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c,
0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x48,
0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01,
0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x17, 0x0a, 0x15, 0x50, 0x65, 0x65, 0x72,
0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x93, 0x01, 0x0a, 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c,
0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x53, 0x65, 0x72, 0x76,
0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73,
0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65,
0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28,
0x09, 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x22, 0x9a, 0x01, 0x0a, 0x20, 0x54, 0x72, 0x75, 0x73,
0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72,
0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e,
0x4f, 0x42, 0x53, 0x4f, 0x4c, 0x45, 0x54, 0x45, 0x5f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x4f, 0x42, 0x53, 0x4f, 0x4c, 0x45, 0x54, 0x45, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x12, 0x4f, 0x0a, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x18, 0x02,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70,
0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, 0x75, 0x6e,
0x64, 0x6c, 0x65, 0x73, 0x22, 0x4a, 0x0a, 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e,
0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12,
0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61,
0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18,
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
0x22, 0x8f, 0x01, 0x0a, 0x17, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e,
0x4f, 0x42, 0x53, 0x4f, 0x4c, 0x45, 0x54, 0x45, 0x5f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x4f, 0x42, 0x53, 0x4f, 0x4c, 0x45, 0x54, 0x45, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e,
0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e,
0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54,
0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x06, 0x42, 0x75, 0x6e, 0x64,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x6c, 0x65, 0x22, 0x2d, 0x0a, 0x1b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72,
0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49,
0x44, 0x22, 0x1e, 0x0a, 0x1c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d,
0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x87, 0x01, 0x0a, 0x1e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75,
0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x65, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54,
0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e,
0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73,
0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x21, 0x0a, 0x1f, 0x50,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c,
0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x53,
0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75,
0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x22, 0x22, 0x0a, 0x20, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72,
0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x6e, 0x65,
0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09,
0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x04, 0x4d, 0x65,
0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69,
0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e,
0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74,
0x61, 0x12, 0x38, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03,
0x28, 0x09, 0x52, 0x17, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x4d,
0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
0x3a, 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x15, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65,
0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20,
0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72,
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x17, 0x0a, 0x15, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x93, 0x01, 0x0a, 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c,
0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61,
0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61,
0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70,
0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
0x04, 0x4b, 0x69, 0x6e, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x20, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69,
0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78,
0x12, 0x4f, 0x0a, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,
0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f,
0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65,
0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75,
0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
0x73, 0x22, 0x4a, 0x0a, 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e,
0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12,
0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01,
0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x7e, 0x0a,
0x17, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65,
0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d,
0x0a, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35,
0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75,
0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69,
0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x2d, 0x0a,
0x1b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74,
0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02,
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x22, 0x1e, 0x0a, 0x1c,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65,
0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x87, 0x01, 0x0a,
0x1e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e,
0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x65, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64,
0x6c, 0x65, 0x52, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74,
0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x21, 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74,
0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x53, 0x0a, 0x1f, 0x50, 0x65, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44,
0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04,
0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65,
0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,
0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x22,
0x0a, 0x20, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75,
0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x9a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54,
0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50,
0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50,
0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69,
0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x05, 0x20,
0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e,
0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e,
0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65,
0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74,
0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x38, 0x0a, 0x17,
0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x41, 0x64,
0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x17, 0x53,
0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x41, 0x64, 0x64,
0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,
0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22,
0x3b, 0x0a, 0x15, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72,
0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0xfc, 0x01, 0x0a,
0x10, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a,
0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20,
0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65,
0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x6e, 0x22, 0xfc, 0x01, 0x0a, 0x10, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52,
0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61,
0x51, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61,
0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b,
0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69,
0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x51, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03,
0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63,
0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70,
0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x13, 0x0a, 0x11, 0x45, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68,
0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72,
0x2a, 0x73, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45,
0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0b, 0x0a, 0x07, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
0x45, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x49, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0a, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x22, 0x13, 0x0a, 0x11, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73,
0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x73, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x49, 0x4e, 0x47, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e,
0x54, 0x45, 0x44, 0x10, 0x06, 0x32, 0x94, 0x09, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10,
0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x8c, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x45, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x49, 0x53, 0x48, 0x49, 0x4e,
0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x47, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x12,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x45,
0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, 0x10, 0x06, 0x32, 0x94, 0x09, 0x0a, 0x0e, 0x50,
0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x8c, 0x01,
0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12,
0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72,
0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0x80, 0x01, 0x0a, 0x09, 0x45, 0x73, 0x74, 0x61, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65,
0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x33, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69,
0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65,
0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e,
0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0x80, 0x01, 0x0a,
0x09, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x33, 0x2e, 0x68, 0x61, 0x73,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45,
0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x50, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x12, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x86, 0x01, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x12,
0x74, 0x1a, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72,
0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52,
0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f,
0x02, 0x10, 0x05, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x69, 0x73, 0x74, 0x12, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69,
0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08,
0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x05, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x50, 0x65, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69,
0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72,
0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10,
0x05, 0x12, 0x8c, 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c,
0x65, 0x74, 0x65, 0x12, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e,
0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e,
0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44,
0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x68,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x05, 0x12, 0x8c, 0x01, 0x0a,
0x0d, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x37,
0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75,
0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69,
0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63,
0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72,
0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0x89, 0x01, 0x0a, 0x0c,
0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x12, 0x36, 0x2e, 0x68,
0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e,
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05,
0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x89, 0x01, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74,
0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x65, 0x12, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f,
0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65,
0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0xad, 0x01, 0x0a, 0x18, 0x54, 0x72, 0x75, 0x73, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69,
0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x68,
0x76, 0x69, 0x63, 0x65, 0x12, 0x42, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74,
0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65,
0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x03, 0x10, 0x05, 0x12, 0xad, 0x01, 0x0a,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x18, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74,
0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x42, 0x2e, 0x68, 0x61, 0x73, 0x68,
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74,
0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72,
0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53,
0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x05, 0x12, 0x92, 0x01, 0x0a, 0x0f, 0x54, 0x72, 0x75, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x43, 0x2e,
0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x12, 0x39, 0x2e, 0x68, 0x61, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c,
0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73,
0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x05, 0x12, 0x92, 0x01, 0x0a,
0x0f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64,
0x12, 0x39, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e,
0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65,
0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x68, 0x61,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e,
0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10,
0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x05, 0x42, 0x92, 0x02, 0x0a, 0x25, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63,
0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x0c, 0x50, 0x65, 0x65,
0x73, 0x65, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x05, 0x42, 0x92, 0x02, 0x0a, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x33, 0x67, 0x69, 0x74,
0x25, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67,
0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x50, 0xaa, 0x02, 0x21, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63,
0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72,
0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x6e, 0x61, 0x6c, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xca, 0x02, 0x21, 0x48, 0x61,
0x65, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xa2, 0x02, 0x04, 0x48, 0x43, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49,
0x49, 0x50, 0xaa, 0x02, 0x21, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xe2,
0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x50, 0x02, 0x2d, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73,
0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xca, 0x02, 0x21, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72,
0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x69, 0x6e, 0x67, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea,
0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xe2, 0x02, 0x2d, 0x48, 0x61, 0x73, 0x02, 0x24, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x43, 0x6f, 0x6e,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x73, 0x75, 0x6c, 0x3a, 0x3a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x3a, 0x3a, 0x50,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5c, 0x47, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x24, 0x48, 0x61, 0x73,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x3a, 0x3a,
0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x3a, 0x3a, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e,
0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
} }
var ( var (

View File

@ -322,7 +322,6 @@ message PeeringServerAddresses {
repeated string Addresses = 1; repeated string Addresses = 1;
} }
// @consul-rpc-glue: LeaderReadTODO
message PeeringReadRequest { message PeeringReadRequest {
string Name = 1; string Name = 1;
string Partition = 2; string Partition = 2;
@ -332,14 +331,13 @@ message PeeringReadResponse {
Peering Peering = 1; Peering Peering = 1;
} }
// @consul-rpc-glue: LeaderReadTODO
message PeeringListRequest { message PeeringListRequest {
string Partition = 1; string Partition = 1;
} }
message PeeringListResponse { message PeeringListResponse {
repeated Peering Peerings = 1; repeated Peering Peerings = 1;
uint64 Index = 2; uint64 OBSOLETE_Index = 2; // Deprecated in favor of gRPC metadata
} }
message PeeringWriteRequest { message PeeringWriteRequest {
@ -373,7 +371,7 @@ message TrustBundleListByServiceRequest {
} }
message TrustBundleListByServiceResponse { message TrustBundleListByServiceResponse {
uint64 Index = 1; uint64 OBSOLETE_Index = 1; // Deprecated in favor of gRPC metadata
repeated PeeringTrustBundle Bundles = 2; repeated PeeringTrustBundle Bundles = 2;
} }
@ -383,7 +381,7 @@ message TrustBundleReadRequest {
} }
message TrustBundleReadResponse { message TrustBundleReadResponse {
uint64 Index = 1; uint64 OBSOLETE_Index = 1; // Deprecated in favor of gRPC metadata
PeeringTrustBundle Bundle = 2; PeeringTrustBundle Bundle = 2;
} }

View File

@ -1,89 +0,0 @@
// Code generated by proto-gen-rpc-glue. DO NOT EDIT.
package pbpeering
import (
"time"
"github.com/hashicorp/consul/agent/structs"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ structs.RPCInfo
var _ time.Month
// IsRead implements structs.RPCInfo
func (msg *PeeringReadRequest) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *PeeringReadRequest) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
// TODO(peering): this needs to stay false for calls to head to the leader until we sync stream tracker information
// like ImportedServicesCount, ExportedServicesCount, as well as general Status fields thru raft to make available
// to followers as well
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *PeeringReadRequest) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// SetTokenSecret implements structs.RPCInfo
func (msg *PeeringReadRequest) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *PeeringReadRequest) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *PeeringReadRequest) Token() string {
// TODO(peering): figure out read semantics here
return ""
}
// IsRead implements structs.RPCInfo
func (msg *PeeringListRequest) IsRead() bool {
// TODO(peering): figure out read semantics here
return true
}
// AllowStaleRead implements structs.RPCInfo
func (msg *PeeringListRequest) AllowStaleRead() bool {
// TODO(peering): figure out read semantics here
// TODO(peering): this needs to stay false for calls to head to the leader until we sync stream tracker information
// like ImportedServicesCount, ExportedServicesCount, as well as general Status fields thru raft to make available
// to followers as well
return false
}
// HasTimedOut implements structs.RPCInfo
func (msg *PeeringListRequest) HasTimedOut(start time.Time, rpcHoldTimeout time.Duration, a time.Duration, b time.Duration) (bool, error) {
// TODO(peering): figure out read semantics here
return time.Since(start) > rpcHoldTimeout, nil
}
// SetTokenSecret implements structs.RPCInfo
func (msg *PeeringListRequest) SetTokenSecret(s string) {
// TODO(peering): figure out read semantics here
}
// TokenSecret implements structs.RPCInfo
func (msg *PeeringListRequest) TokenSecret() string {
// TODO(peering): figure out read semantics here
return ""
}
// Token implements structs.RPCInfo
func (msg *PeeringListRequest) Token() string {
// TODO(peering): figure out read semantics here
return ""
}

View File

@ -154,7 +154,7 @@ The table below shows this endpoint's support for
| Blocking Queries | Consistency Modes | Agent Caching | ACL Required | | Blocking Queries | Consistency Modes | Agent Caching | ACL Required |
| ---------------- | ----------------- | ------------- | -------------- | | ---------------- | ----------------- | ------------- | -------------- |
| `NO` | `consistent` | `none` | `peering:read` | | `YES` | `consistent` | `none` | `peering:read` |
### Path Parameters ### Path Parameters
@ -264,8 +264,8 @@ The table below shows this endpoint's support for
[required ACLs](/consul/api-docs/api-structure#authentication). [required ACLs](/consul/api-docs/api-structure#authentication).
| Blocking Queries | Consistency Modes | Agent Caching | ACL Required | | Blocking Queries | Consistency Modes | Agent Caching | ACL Required |
| ---------------- | ----------------- | ------------- | -------------- | | ---------------- | ----------------- | -------------------- | -------------- |
| `NO` | `consistent` | `none` | `peering:read` | | `YES` | `consistent` | `background refresh` | `peering:read` |
### Query Parameters ### Query Parameters