mirror of https://github.com/status-im/consul.git
NET-5824 Exported services api (#20015)
* Exported services api implemented * Tests added, refactored code * Adding server tests * changelog added * Proto gen added * Adding codegen changes * changing url, response object * Fixing lint error by having namespace and partition directly * Tests changes * refactoring tests * Simplified uniqueness logic for exported services, sorted the response in order of service name * Fix lint errors, refactored code
This commit is contained in:
parent
528147e5ad
commit
5d294b26d3
|
@ -0,0 +1,3 @@
|
||||||
|
```release-note:improvement
|
||||||
|
api: add a new api(/v1/exported-services) to list all the exported service and their consumers.
|
||||||
|
```
|
|
@ -78,6 +78,7 @@ import (
|
||||||
"github.com/hashicorp/consul/lib/routine"
|
"github.com/hashicorp/consul/lib/routine"
|
||||||
"github.com/hashicorp/consul/logging"
|
"github.com/hashicorp/consul/logging"
|
||||||
"github.com/hashicorp/consul/proto-public/pbresource"
|
"github.com/hashicorp/consul/proto-public/pbresource"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
"github.com/hashicorp/consul/proto/private/pboperator"
|
"github.com/hashicorp/consul/proto/private/pboperator"
|
||||||
"github.com/hashicorp/consul/proto/private/pbpeering"
|
"github.com/hashicorp/consul/proto/private/pbpeering"
|
||||||
"github.com/hashicorp/consul/tlsutil"
|
"github.com/hashicorp/consul/tlsutil"
|
||||||
|
@ -417,8 +418,9 @@ type Agent struct {
|
||||||
|
|
||||||
// TODO: pass directly to HTTPHandlers and dnsServer once those are passed
|
// TODO: pass directly to HTTPHandlers and dnsServer once those are passed
|
||||||
// into Agent, which will allow us to remove this field.
|
// into Agent, which will allow us to remove this field.
|
||||||
rpcClientHealth *health.Client
|
rpcClientHealth *health.Client
|
||||||
rpcClientConfigEntry *configentry.Client
|
rpcClientConfigEntry *configentry.Client
|
||||||
|
grpcClientConfigEntry pbconfigentry.ConfigEntryServiceClient
|
||||||
|
|
||||||
rpcClientPeering pbpeering.PeeringServiceClient
|
rpcClientPeering pbpeering.PeeringServiceClient
|
||||||
|
|
||||||
|
@ -521,6 +523,7 @@ func New(bd BaseDeps) (*Agent, error) {
|
||||||
|
|
||||||
a.rpcClientPeering = pbpeering.NewPeeringServiceClient(conn)
|
a.rpcClientPeering = pbpeering.NewPeeringServiceClient(conn)
|
||||||
a.rpcClientOperator = pboperator.NewOperatorServiceClient(conn)
|
a.rpcClientOperator = pboperator.NewOperatorServiceClient(conn)
|
||||||
|
a.grpcClientConfigEntry = pbconfigentry.NewConfigEntryServiceClient(conn)
|
||||||
|
|
||||||
a.serviceManager = NewServiceManager(&a)
|
a.serviceManager = NewServiceManager(&a)
|
||||||
a.rpcClientConfigEntry = &configentry.Client{
|
a.rpcClientConfigEntry = &configentry.Client{
|
||||||
|
|
|
@ -10,7 +10,12 @@ import (
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
|
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||||
"github.com/hashicorp/consul/agent/structs"
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/api"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
"google.golang.org/grpc/metadata"
|
||||||
)
|
)
|
||||||
|
|
||||||
const ConfigEntryNotFoundErr string = "Config entry not found"
|
const ConfigEntryNotFoundErr string = "Config entry not found"
|
||||||
|
@ -176,3 +181,45 @@ func (s *HTTPHandlers) parseEntMetaForConfigEntryKind(kind string, req *http.Req
|
||||||
}
|
}
|
||||||
return s.parseEntMetaNoWildcard(req, entMeta)
|
return s.parseEntMetaNoWildcard(req, entMeta)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ExportedServices returns all the exported services by resolving wildcards and sameness groups
|
||||||
|
// in the exported services configuration entry
|
||||||
|
func (s *HTTPHandlers) ExportedServices(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
|
||||||
|
var entMeta acl.EnterpriseMeta
|
||||||
|
if err := s.parseEntMetaPartition(req, &entMeta); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
args := pbconfigentry.GetResolvedExportedServicesRequest{
|
||||||
|
Partition: entMeta.PartitionOrEmpty(),
|
||||||
|
}
|
||||||
|
|
||||||
|
var dc string
|
||||||
|
options := structs.QueryOptions{}
|
||||||
|
s.parse(resp, req, &dc, &options)
|
||||||
|
ctx, err := external.ContextWithQueryOptions(req.Context(), options)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
var header metadata.MD
|
||||||
|
result, err := s.agent.grpcClientConfigEntry.GetResolvedExportedServices(ctx, &args, grpc.Header(&header))
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
meta, err := external.QueryMetaFromGRPCMeta(header)
|
||||||
|
if err != nil {
|
||||||
|
return result.Services, fmt.Errorf("could not convert gRPC metadata to query meta: %w", err)
|
||||||
|
}
|
||||||
|
if err := setMeta(resp, &meta); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
svcs := make([]api.ResolvedExportedService, len(result.Services))
|
||||||
|
|
||||||
|
for idx, svc := range result.Services {
|
||||||
|
svcs[idx] = *svc.ToAPI()
|
||||||
|
}
|
||||||
|
|
||||||
|
return svcs, nil
|
||||||
|
}
|
||||||
|
|
|
@ -16,6 +16,7 @@ import (
|
||||||
|
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
"github.com/hashicorp/consul/agent/structs"
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/api"
|
||||||
"github.com/hashicorp/consul/testrpc"
|
"github.com/hashicorp/consul/testrpc"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -736,3 +737,84 @@ func TestConfig_Apply_ProxyDefaultsExpose(t *testing.T) {
|
||||||
require.Equal(t, expose, entry.Expose)
|
require.Equal(t, expose, entry.Expose)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestConfig_Exported_Services(t *testing.T) {
|
||||||
|
if testing.Short() {
|
||||||
|
t.Skip("too slow for testing.Short")
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Parallel()
|
||||||
|
a := NewTestAgent(t, "")
|
||||||
|
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
|
||||||
|
defer a.Shutdown()
|
||||||
|
|
||||||
|
{
|
||||||
|
// Register exported services
|
||||||
|
args := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "api",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
req := structs.ConfigEntryRequest{
|
||||||
|
Datacenter: "dc1",
|
||||||
|
Entry: args,
|
||||||
|
}
|
||||||
|
var configOutput bool
|
||||||
|
require.NoError(t, a.RPC(context.Background(), "ConfigEntry.Apply", &req, &configOutput))
|
||||||
|
require.True(t, configOutput)
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Run("exported services", func(t *testing.T) {
|
||||||
|
req, _ := http.NewRequest("GET", "/v1/exported-services", nil)
|
||||||
|
resp := httptest.NewRecorder()
|
||||||
|
raw, err := a.srv.ExportedServices(resp, req)
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, http.StatusOK, resp.Code)
|
||||||
|
|
||||||
|
services, ok := raw.([]api.ResolvedExportedService)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Len(t, services, 2)
|
||||||
|
assertIndex(t, resp)
|
||||||
|
|
||||||
|
entMeta := acl.DefaultEnterpriseMeta()
|
||||||
|
|
||||||
|
expected := []api.ResolvedExportedService{
|
||||||
|
{
|
||||||
|
Service: "api",
|
||||||
|
Partition: entMeta.PartitionOrEmpty(),
|
||||||
|
Namespace: entMeta.NamespaceOrEmpty(),
|
||||||
|
Consumers: api.ResolvedConsumers{
|
||||||
|
Peers: []string{"east", "west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "db",
|
||||||
|
Partition: entMeta.PartitionOrEmpty(),
|
||||||
|
Namespace: entMeta.NamespaceOrEmpty(),
|
||||||
|
Consumers: api.ResolvedConsumers{
|
||||||
|
Peers: []string{"east"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
require.Equal(t, expected, services)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,31 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package consul
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/hashicorp/consul/acl"
|
||||||
|
"github.com/hashicorp/consul/acl/resolver"
|
||||||
|
"github.com/hashicorp/consul/agent/grpc-external/services/configentry"
|
||||||
|
)
|
||||||
|
|
||||||
|
type ConfigEntryBackend struct {
|
||||||
|
srv *Server
|
||||||
|
}
|
||||||
|
|
||||||
|
var _ configentry.Backend = (*ConfigEntryBackend)(nil)
|
||||||
|
|
||||||
|
// NewConfigEntryBackend returns a configentry.Backend implementation that is bound to the given server.
|
||||||
|
func NewConfigEntryBackend(srv *Server) *ConfigEntryBackend {
|
||||||
|
return &ConfigEntryBackend{
|
||||||
|
srv: srv,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *ConfigEntryBackend) EnterpriseCheckPartitions(partition string) error {
|
||||||
|
return b.enterpriseCheckPartitions(partition)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *ConfigEntryBackend) ResolveTokenAndDefaultMeta(token string, entMeta *acl.EnterpriseMeta, authzCtx *acl.AuthorizerContext) (resolver.Result, error) {
|
||||||
|
return b.srv.ResolveTokenAndDefaultMeta(token, entMeta, authzCtx)
|
||||||
|
}
|
|
@ -0,0 +1,18 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
//go:build !consulent
|
||||||
|
|
||||||
|
package consul
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"strings"
|
||||||
|
)
|
||||||
|
|
||||||
|
func (b *ConfigEntryBackend) enterpriseCheckPartitions(partition string) error {
|
||||||
|
if partition == "" || strings.EqualFold(partition, "default") {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return fmt.Errorf("Partitions are a Consul Enterprise feature")
|
||||||
|
}
|
|
@ -0,0 +1,87 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
//go:build !consulent
|
||||||
|
|
||||||
|
package consul
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
gogrpc "google.golang.org/grpc"
|
||||||
|
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"github.com/hashicorp/consul/sdk/freeport"
|
||||||
|
"github.com/hashicorp/consul/testrpc"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestConfigEntryBackend_RejectsPartition(t *testing.T) {
|
||||||
|
if testing.Short() {
|
||||||
|
t.Skip("too slow for testing.Short")
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Parallel()
|
||||||
|
|
||||||
|
_, s1 := testServerWithConfig(t, func(c *Config) {
|
||||||
|
c.GRPCTLSPort = freeport.GetOne(t)
|
||||||
|
})
|
||||||
|
testrpc.WaitForLeader(t, s1.RPC, "dc1")
|
||||||
|
|
||||||
|
// make a grpc client to dial s1 directly
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
|
||||||
|
t.Cleanup(cancel)
|
||||||
|
|
||||||
|
conn, err := gogrpc.DialContext(ctx, s1.config.RPCAddr.String(),
|
||||||
|
gogrpc.WithContextDialer(newServerDialer(s1.config.RPCAddr.String())),
|
||||||
|
//nolint:staticcheck
|
||||||
|
gogrpc.WithInsecure(),
|
||||||
|
gogrpc.WithBlock())
|
||||||
|
require.NoError(t, err)
|
||||||
|
t.Cleanup(func() { conn.Close() })
|
||||||
|
|
||||||
|
configEntryClient := pbconfigentry.NewConfigEntryServiceClient(conn)
|
||||||
|
|
||||||
|
req := pbconfigentry.GetResolvedExportedServicesRequest{
|
||||||
|
Partition: "test",
|
||||||
|
}
|
||||||
|
_, err = configEntryClient.GetResolvedExportedServices(ctx, &req)
|
||||||
|
require.Error(t, err)
|
||||||
|
require.Contains(t, err.Error(), "Partitions are a Consul Enterprise feature")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestConfigEntryBackend_IgnoresDefaultPartition(t *testing.T) {
|
||||||
|
if testing.Short() {
|
||||||
|
t.Skip("too slow for testing.Short")
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Parallel()
|
||||||
|
|
||||||
|
_, s1 := testServerWithConfig(t, func(c *Config) {
|
||||||
|
c.GRPCTLSPort = freeport.GetOne(t)
|
||||||
|
})
|
||||||
|
|
||||||
|
testrpc.WaitForLeader(t, s1.RPC, "dc1")
|
||||||
|
|
||||||
|
// make a grpc client to dial s1 directly
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
|
||||||
|
t.Cleanup(cancel)
|
||||||
|
|
||||||
|
conn, err := gogrpc.DialContext(ctx, s1.config.RPCAddr.String(),
|
||||||
|
gogrpc.WithContextDialer(newServerDialer(s1.config.RPCAddr.String())),
|
||||||
|
//nolint:staticcheck
|
||||||
|
gogrpc.WithInsecure(),
|
||||||
|
gogrpc.WithBlock())
|
||||||
|
require.NoError(t, err)
|
||||||
|
t.Cleanup(func() { conn.Close() })
|
||||||
|
|
||||||
|
configEntryClient := pbconfigentry.NewConfigEntryServiceClient(conn)
|
||||||
|
|
||||||
|
req := pbconfigentry.GetResolvedExportedServicesRequest{
|
||||||
|
Partition: "DeFaUlT",
|
||||||
|
}
|
||||||
|
_, err = configEntryClient.GetResolvedExportedServices(ctx, &req)
|
||||||
|
require.NoError(t, err)
|
||||||
|
}
|
|
@ -0,0 +1,49 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package consul
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"github.com/hashicorp/consul/sdk/freeport"
|
||||||
|
"github.com/hashicorp/consul/testrpc"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
gogrpc "google.golang.org/grpc"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestConfigEntryBackend_EmptyPartition(t *testing.T) {
|
||||||
|
if testing.Short() {
|
||||||
|
t.Skip("too slow for testing.Short")
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Parallel()
|
||||||
|
|
||||||
|
_, s1 := testServerWithConfig(t, func(c *Config) {
|
||||||
|
c.GRPCTLSPort = freeport.GetOne(t)
|
||||||
|
})
|
||||||
|
testrpc.WaitForLeader(t, s1.RPC, "dc1")
|
||||||
|
|
||||||
|
// make a grpc client to dial s1 directly
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
|
||||||
|
t.Cleanup(cancel)
|
||||||
|
|
||||||
|
conn, err := gogrpc.DialContext(ctx, s1.config.RPCAddr.String(),
|
||||||
|
gogrpc.WithContextDialer(newServerDialer(s1.config.RPCAddr.String())),
|
||||||
|
//nolint:staticcheck
|
||||||
|
gogrpc.WithInsecure(),
|
||||||
|
gogrpc.WithBlock())
|
||||||
|
require.NoError(t, err)
|
||||||
|
t.Cleanup(func() { conn.Close() })
|
||||||
|
|
||||||
|
configEntryClient := pbconfigentry.NewConfigEntryServiceClient(conn)
|
||||||
|
|
||||||
|
req := pbconfigentry.GetResolvedExportedServicesRequest{
|
||||||
|
Partition: "",
|
||||||
|
}
|
||||||
|
_, err = configEntryClient.GetResolvedExportedServices(ctx, &req)
|
||||||
|
require.NoError(t, err)
|
||||||
|
}
|
|
@ -16,6 +16,7 @@ import (
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
"github.com/hashicorp/consul/acl/resolver"
|
"github.com/hashicorp/consul/acl/resolver"
|
||||||
aclgrpc "github.com/hashicorp/consul/agent/grpc-external/services/acl"
|
aclgrpc "github.com/hashicorp/consul/agent/grpc-external/services/acl"
|
||||||
|
"github.com/hashicorp/consul/agent/grpc-external/services/configentry"
|
||||||
"github.com/hashicorp/consul/agent/grpc-external/services/connectca"
|
"github.com/hashicorp/consul/agent/grpc-external/services/connectca"
|
||||||
"github.com/hashicorp/consul/agent/grpc-external/services/dataplane"
|
"github.com/hashicorp/consul/agent/grpc-external/services/dataplane"
|
||||||
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
|
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
|
||||||
|
@ -322,6 +323,16 @@ func (s *Server) setupGRPCServices(config *Config, deps Deps) error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// register the configEntry service on the internal interface only. As
|
||||||
|
// it is only accessed via the internalGRPCHandler with an actual network
|
||||||
|
// conn managed by the Agents GRPCConnPool.
|
||||||
|
err = s.registerConfigEntryServer(
|
||||||
|
s.internalGRPCHandler,
|
||||||
|
)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
// enable grpc server reflection for the external gRPC interface only
|
// enable grpc server reflection for the external gRPC interface only
|
||||||
reflection.Register(s.externalGRPCServer)
|
reflection.Register(s.externalGRPCServer)
|
||||||
|
|
||||||
|
@ -536,3 +547,21 @@ func (s *Server) registerServerDiscoveryServer(resolver serverdiscovery.ACLResol
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *Server) registerConfigEntryServer(registrars ...grpc.ServiceRegistrar) error {
|
||||||
|
|
||||||
|
srv := configentry.NewServer(configentry.Config{
|
||||||
|
Backend: NewConfigEntryBackend(s),
|
||||||
|
Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.ConfigEntry),
|
||||||
|
ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) {
|
||||||
|
return s.ForwardGRPC(s.grpcConnPool, info, fn)
|
||||||
|
},
|
||||||
|
FSMServer: s,
|
||||||
|
})
|
||||||
|
|
||||||
|
for _, reg := range registrars {
|
||||||
|
srv.Register(reg)
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
|
@ -5,10 +5,13 @@ package state
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"sort"
|
||||||
|
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
"github.com/hashicorp/consul/agent/configentry"
|
"github.com/hashicorp/consul/agent/configentry"
|
||||||
"github.com/hashicorp/consul/agent/structs"
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/lib"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
"github.com/hashicorp/go-memdb"
|
"github.com/hashicorp/go-memdb"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -64,3 +67,108 @@ func getExportedServicesConfigEntryTxn(
|
||||||
}
|
}
|
||||||
return idx, export, nil
|
return idx, export, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ResolvedExportedServices returns the list of exported services along with consumers.
|
||||||
|
// Sameness Groups and wild card entries are resolved.
|
||||||
|
func (s *Store) ResolvedExportedServices(ws memdb.WatchSet, entMeta *acl.EnterpriseMeta) (uint64, []*pbconfigentry.ResolvedExportedService, error) {
|
||||||
|
tx := s.db.ReadTxn()
|
||||||
|
defer tx.Abort()
|
||||||
|
|
||||||
|
return resolvedExportedServicesTxn(tx, ws, entMeta)
|
||||||
|
}
|
||||||
|
|
||||||
|
func resolvedExportedServicesTxn(tx ReadTxn, ws memdb.WatchSet, entMeta *acl.EnterpriseMeta) (uint64, []*pbconfigentry.ResolvedExportedService, error) {
|
||||||
|
var resp []*pbconfigentry.ResolvedExportedService
|
||||||
|
|
||||||
|
// getSimplifiedExportedServices resolves the sameness group information to partitions and peers.
|
||||||
|
maxIdx, exports, err := getSimplifiedExportedServices(tx, ws, nil, *entMeta)
|
||||||
|
if err != nil {
|
||||||
|
return 0, nil, err
|
||||||
|
}
|
||||||
|
if exports == nil {
|
||||||
|
return maxIdx, nil, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
var exportedServices []structs.ExportedService
|
||||||
|
|
||||||
|
for _, svc := range exports.Services {
|
||||||
|
// Prevent exporting the "consul" service.
|
||||||
|
if svc.Name == structs.ConsulServiceName {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
// If this isn't a wildcard, we can simply add it to the list of exportedServices and move to the next entry.
|
||||||
|
if svc.Name != structs.WildcardSpecifier {
|
||||||
|
exportedServices = append(exportedServices, svc)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
svcEntMeta := acl.NewEnterpriseMetaWithPartition(entMeta.PartitionOrDefault(), svc.Namespace)
|
||||||
|
|
||||||
|
// If all services in the namespace are exported by the wildcard, query those service names.
|
||||||
|
idx, typicalServices, err := serviceNamesOfKindTxn(tx, ws, structs.ServiceKindTypical, svcEntMeta)
|
||||||
|
if err != nil {
|
||||||
|
return 0, nil, fmt.Errorf("failed to get typical service names: %w", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
maxIdx = lib.MaxUint64(maxIdx, idx)
|
||||||
|
|
||||||
|
for _, sn := range typicalServices {
|
||||||
|
// Prevent exporting the "consul" service.
|
||||||
|
if sn.Service.Name != structs.ConsulServiceName {
|
||||||
|
exportedServices = append(exportedServices, structs.ExportedService{
|
||||||
|
Name: sn.Service.Name,
|
||||||
|
Namespace: sn.Service.NamespaceOrDefault(),
|
||||||
|
Consumers: svc.Consumers,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
uniqueExportedServices := getUniqueExportedServices(exportedServices, entMeta)
|
||||||
|
resp = prepareExportedServicesResponse(uniqueExportedServices, entMeta)
|
||||||
|
|
||||||
|
return maxIdx, resp, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// getUniqueExportedServices removes duplicate services and consumers. Services are also sorted in ascending order
|
||||||
|
func getUniqueExportedServices(exportedServices []structs.ExportedService, entMeta *acl.EnterpriseMeta) []structs.ExportedService {
|
||||||
|
// Services -> ServiceConsumers
|
||||||
|
var exportedServicesMapper = make(map[structs.ServiceName]map[structs.ServiceConsumer]struct{})
|
||||||
|
for _, svc := range exportedServices {
|
||||||
|
svcEntMeta := acl.NewEnterpriseMetaWithPartition(entMeta.PartitionOrDefault(), svc.Namespace)
|
||||||
|
svcName := structs.NewServiceName(svc.Name, &svcEntMeta)
|
||||||
|
|
||||||
|
for _, c := range svc.Consumers {
|
||||||
|
cons, ok := exportedServicesMapper[svcName]
|
||||||
|
if !ok {
|
||||||
|
cons = make(map[structs.ServiceConsumer]struct{})
|
||||||
|
exportedServicesMapper[svcName] = cons
|
||||||
|
}
|
||||||
|
cons[c] = struct{}{}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
uniqueExportedServices := make([]structs.ExportedService, 0, len(exportedServicesMapper))
|
||||||
|
|
||||||
|
for svc, cons := range exportedServicesMapper {
|
||||||
|
consumers := make([]structs.ServiceConsumer, 0, len(cons))
|
||||||
|
for con := range cons {
|
||||||
|
consumers = append(consumers, con)
|
||||||
|
}
|
||||||
|
|
||||||
|
uniqueExportedServices = append(uniqueExportedServices, structs.ExportedService{
|
||||||
|
Name: svc.Name,
|
||||||
|
Namespace: svc.NamespaceOrDefault(),
|
||||||
|
Consumers: consumers,
|
||||||
|
})
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
sort.Slice(uniqueExportedServices, func(i, j int) bool {
|
||||||
|
return (uniqueExportedServices[i].Name < uniqueExportedServices[j].Name) ||
|
||||||
|
(uniqueExportedServices[i].Name == uniqueExportedServices[j].Name && uniqueExportedServices[i].Namespace < uniqueExportedServices[j].Namespace)
|
||||||
|
})
|
||||||
|
|
||||||
|
return uniqueExportedServices
|
||||||
|
}
|
||||||
|
|
|
@ -6,9 +6,12 @@
|
||||||
package state
|
package state
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"sort"
|
||||||
|
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
"github.com/hashicorp/consul/agent/configentry"
|
"github.com/hashicorp/consul/agent/configentry"
|
||||||
"github.com/hashicorp/consul/agent/structs"
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
"github.com/hashicorp/go-memdb"
|
"github.com/hashicorp/go-memdb"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -31,3 +34,29 @@ func (s *Store) GetSimplifiedExportedServices(ws memdb.WatchSet, entMeta acl.Ent
|
||||||
defer tx.Abort()
|
defer tx.Abort()
|
||||||
return getSimplifiedExportedServices(tx, ws, nil, entMeta)
|
return getSimplifiedExportedServices(tx, ws, nil, entMeta)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func prepareExportedServicesResponse(exportedServices []structs.ExportedService, entMeta *acl.EnterpriseMeta) []*pbconfigentry.ResolvedExportedService {
|
||||||
|
|
||||||
|
resp := make([]*pbconfigentry.ResolvedExportedService, len(exportedServices))
|
||||||
|
|
||||||
|
for idx, exportedService := range exportedServices {
|
||||||
|
consumerPeers := []string{}
|
||||||
|
|
||||||
|
for _, consumer := range exportedService.Consumers {
|
||||||
|
if consumer.Peer != "" {
|
||||||
|
consumerPeers = append(consumerPeers, consumer.Peer)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
sort.Strings(consumerPeers)
|
||||||
|
|
||||||
|
resp[idx] = &pbconfigentry.ResolvedExportedService{
|
||||||
|
Service: exportedService.Name,
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: consumerPeers,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return resp
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,317 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
//go:build !consulent
|
||||||
|
|
||||||
|
package state
|
||||||
|
|
||||||
|
import (
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"github.com/hashicorp/go-memdb"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestStore_prepareExportedServicesResponse(t *testing.T) {
|
||||||
|
|
||||||
|
exportedServices := []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Partition: "part",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "web",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "peer-a",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "peer-b",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
resp := prepareExportedServicesResponse(exportedServices, nil)
|
||||||
|
|
||||||
|
expected := []*pbconfigentry.ResolvedExportedService{
|
||||||
|
{
|
||||||
|
Service: "db",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"east", "west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "web",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"peer-a", "peer-b"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
require.Equal(t, expected, resp)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestStore_ResolvedExportingServices(t *testing.T) {
|
||||||
|
s := NewStateStore(nil)
|
||||||
|
var c indexCounter
|
||||||
|
|
||||||
|
{
|
||||||
|
require.NoError(t, s.EnsureNode(c.Next(), &structs.Node{
|
||||||
|
Node: "foo", Address: "127.0.0.1",
|
||||||
|
}))
|
||||||
|
|
||||||
|
require.NoError(t, s.EnsureService(c.Next(), "foo", &structs.NodeService{
|
||||||
|
ID: "db", Service: "db", Port: 5000,
|
||||||
|
}))
|
||||||
|
|
||||||
|
require.NoError(t, s.EnsureService(c.Next(), "foo", &structs.NodeService{
|
||||||
|
ID: "cache", Service: "cache", Port: 5000,
|
||||||
|
}))
|
||||||
|
|
||||||
|
entry := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "cache",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
err := s.EnsureConfigEntry(c.Next(), entry)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
// Adding services to check wildcard config later on
|
||||||
|
|
||||||
|
require.NoError(t, s.EnsureService(c.Next(), "foo", &structs.NodeService{
|
||||||
|
ID: "frontend", Service: "frontend", Port: 5000,
|
||||||
|
}))
|
||||||
|
|
||||||
|
require.NoError(t, s.EnsureService(c.Next(), "foo", &structs.NodeService{
|
||||||
|
ID: "backend", Service: "backend", Port: 5000,
|
||||||
|
}))
|
||||||
|
|
||||||
|
// The consul service should never be exported.
|
||||||
|
require.NoError(t, s.EnsureService(c.Next(), "foo", &structs.NodeService{
|
||||||
|
ID: structs.ConsulServiceID, Service: structs.ConsulServiceName, Port: 8000,
|
||||||
|
}))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
type testCase struct {
|
||||||
|
expect []*pbconfigentry.ResolvedExportedService
|
||||||
|
idx uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
run := func(t *testing.T, tc testCase) {
|
||||||
|
ws := memdb.NewWatchSet()
|
||||||
|
defaultMeta := structs.DefaultEnterpriseMetaInDefaultPartition()
|
||||||
|
idx, services, err := s.ResolvedExportedServices(ws, defaultMeta)
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, tc.idx, idx)
|
||||||
|
require.Equal(t, tc.expect, services)
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Run("only exported services are included", func(t *testing.T) {
|
||||||
|
tc := testCase{
|
||||||
|
expect: []*pbconfigentry.ResolvedExportedService{
|
||||||
|
{
|
||||||
|
Service: "cache",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"east"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "db",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"east", "west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
idx: 4,
|
||||||
|
}
|
||||||
|
|
||||||
|
run(t, tc)
|
||||||
|
})
|
||||||
|
|
||||||
|
t.Run("wild card includes all services", func(t *testing.T) {
|
||||||
|
entry := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "*",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{Peer: "west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
err := s.EnsureConfigEntry(c.Next(), entry)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
tc := testCase{
|
||||||
|
expect: []*pbconfigentry.ResolvedExportedService{
|
||||||
|
{
|
||||||
|
Service: "backend",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "cache",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "db",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
|
||||||
|
{
|
||||||
|
Service: "frontend",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
idx: c.Last(),
|
||||||
|
}
|
||||||
|
|
||||||
|
run(t, tc)
|
||||||
|
})
|
||||||
|
|
||||||
|
t.Run("deleting the config entry clears the services", func(t *testing.T) {
|
||||||
|
defaultMeta := structs.DefaultEnterpriseMetaInDefaultPartition()
|
||||||
|
err := s.DeleteConfigEntry(c.Next(), structs.ExportedServices, "default", nil)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
idx, result, err := s.ResolvedExportedServices(nil, defaultMeta)
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, c.Last(), idx)
|
||||||
|
require.Nil(t, result)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestStore_getUniqueExportedServices(t *testing.T) {
|
||||||
|
|
||||||
|
exportedServices := []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Partition: "part",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "web",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "peer-a",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "peer-b",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west-2",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west-2",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
resp := getUniqueExportedServices(exportedServices, nil)
|
||||||
|
|
||||||
|
expected := []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Partition: "part",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west-2",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "web",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "peer-a",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "peer-b",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
require.Equal(t, 2, len(resp))
|
||||||
|
|
||||||
|
for idx, expSvc := range expected {
|
||||||
|
require.Equal(t, expSvc.Name, resp[idx].Name)
|
||||||
|
require.ElementsMatch(t, expSvc.Consumers, resp[idx].Consumers)
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,133 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package configentry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/armon/go-metrics"
|
||||||
|
"github.com/hashicorp/go-hclog"
|
||||||
|
"github.com/hashicorp/go-memdb"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
"google.golang.org/grpc/codes"
|
||||||
|
grpcstatus "google.golang.org/grpc/status"
|
||||||
|
|
||||||
|
"github.com/hashicorp/consul/acl"
|
||||||
|
"github.com/hashicorp/consul/acl/resolver"
|
||||||
|
"github.com/hashicorp/consul/agent/blockingquery"
|
||||||
|
"github.com/hashicorp/consul/agent/consul/state"
|
||||||
|
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||||
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Server implements pbconfigentry.ConfigEntryService to provide RPC operations related to
|
||||||
|
// configentries
|
||||||
|
type Server struct {
|
||||||
|
Config
|
||||||
|
}
|
||||||
|
|
||||||
|
type Config struct {
|
||||||
|
Backend Backend
|
||||||
|
Logger hclog.Logger
|
||||||
|
ForwardRPC func(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error)
|
||||||
|
FSMServer blockingquery.FSMServer
|
||||||
|
}
|
||||||
|
|
||||||
|
type Backend interface {
|
||||||
|
EnterpriseCheckPartitions(partition string) error
|
||||||
|
|
||||||
|
ResolveTokenAndDefaultMeta(token string, entMeta *acl.EnterpriseMeta, authzCtx *acl.AuthorizerContext) (resolver.Result, error)
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewServer(cfg Config) *Server {
|
||||||
|
external.RequireNotNil(cfg.Backend, "Backend")
|
||||||
|
external.RequireNotNil(cfg.Logger, "Logger")
|
||||||
|
external.RequireNotNil(cfg.FSMServer, "FSMServer")
|
||||||
|
|
||||||
|
return &Server{
|
||||||
|
Config: cfg,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
var _ pbconfigentry.ConfigEntryServiceServer = (*Server)(nil)
|
||||||
|
|
||||||
|
type readRequest struct {
|
||||||
|
structs.QueryOptions
|
||||||
|
structs.DCSpecificRequest
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *Server) Register(grpcServer grpc.ServiceRegistrar) {
|
||||||
|
pbconfigentry.RegisterConfigEntryServiceServer(grpcServer, s)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *Server) GetResolvedExportedServices(
|
||||||
|
ctx context.Context,
|
||||||
|
req *pbconfigentry.GetResolvedExportedServicesRequest,
|
||||||
|
) (*pbconfigentry.GetResolvedExportedServicesResponse, error) {
|
||||||
|
|
||||||
|
if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
|
||||||
|
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
|
||||||
|
}
|
||||||
|
|
||||||
|
options, err := external.QueryOptionsFromContext(ctx)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
var resp *pbconfigentry.GetResolvedExportedServicesResponse
|
||||||
|
var emptyDCSpecificRequest structs.DCSpecificRequest
|
||||||
|
|
||||||
|
handled, err := s.ForwardRPC(&readRequest{options, emptyDCSpecificRequest}, func(conn *grpc.ClientConn) error {
|
||||||
|
var err error
|
||||||
|
resp, err = pbconfigentry.NewConfigEntryServiceClient(conn).GetResolvedExportedServices(ctx, req)
|
||||||
|
return err
|
||||||
|
})
|
||||||
|
if handled || err != nil {
|
||||||
|
return resp, err
|
||||||
|
}
|
||||||
|
|
||||||
|
defer metrics.MeasureSince([]string{"configentry", "get_resolved_exported_services"}, time.Now())
|
||||||
|
|
||||||
|
var authzCtx acl.AuthorizerContext
|
||||||
|
entMeta := structs.DefaultEnterpriseMetaInPartition(req.Partition)
|
||||||
|
|
||||||
|
authz, err := s.Backend.ResolveTokenAndDefaultMeta(options.Token, entMeta, &authzCtx)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := authz.ToAllowAuthorizer().MeshReadAllowed(&authzCtx); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
res := &pbconfigentry.GetResolvedExportedServicesResponse{}
|
||||||
|
meta := structs.QueryMeta{}
|
||||||
|
err = blockingquery.Query(s.FSMServer, &options, &meta, func(ws memdb.WatchSet, store *state.Store) error {
|
||||||
|
idx, exportedSvcs, err := store.ResolvedExportedServices(ws, entMeta)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
meta.SetIndex(idx)
|
||||||
|
|
||||||
|
res.Services = exportedSvcs
|
||||||
|
return nil
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("error executing exported services 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
|
||||||
|
}
|
|
@ -0,0 +1,83 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package configentry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/hashicorp/consul/acl"
|
||||||
|
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||||
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"github.com/hashicorp/go-hclog"
|
||||||
|
"github.com/stretchr/testify/mock"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices(t *testing.T) {
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Allow)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(nil)
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
// Add config entry
|
||||||
|
entry := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "cache",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
fakeFSM.GetState().EnsureConfigEntry(1, entry)
|
||||||
|
|
||||||
|
expected := []*pbconfigentry.ResolvedExportedService{
|
||||||
|
{
|
||||||
|
Service: "cache",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"east"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Service: "db",
|
||||||
|
Consumers: &pbconfigentry.Consumers{
|
||||||
|
Peers: []string{"east", "west"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
ctx := grpc.NewContextWithServerTransportStream(context.Background(), &testutils.MockServerTransportStream{})
|
||||||
|
resp, err := server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, expected, resp.Services)
|
||||||
|
}
|
|
@ -0,0 +1,236 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package configentry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/armon/go-metrics"
|
||||||
|
"github.com/hashicorp/consul/acl"
|
||||||
|
"github.com/hashicorp/consul/acl/resolver"
|
||||||
|
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||||
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/proto/private/pbconfigentry"
|
||||||
|
"github.com/hashicorp/go-hclog"
|
||||||
|
"github.com/stretchr/testify/mock"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
)
|
||||||
|
|
||||||
|
type MockBackend struct {
|
||||||
|
mock.Mock
|
||||||
|
authorizer acl.Authorizer
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockBackend) ResolveTokenAndDefaultMeta(string, *acl.EnterpriseMeta, *acl.AuthorizerContext) (resolver.Result, error) {
|
||||||
|
return resolver.Result{Authorizer: m.authorizer}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockBackend) EnterpriseCheckPartitions(partition string) error {
|
||||||
|
called := m.Called(partition)
|
||||||
|
ret := called.Get(0)
|
||||||
|
|
||||||
|
if ret == nil {
|
||||||
|
return nil
|
||||||
|
} else {
|
||||||
|
return ret.(error)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices_ACL_Deny(t *testing.T) {
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Deny)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(nil)
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
_, err := server.GetResolvedExportedServices(context.Background(), &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.Error(t, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices_AC_Allow(t *testing.T) {
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Allow)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(nil)
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
ctx := grpc.NewContextWithServerTransportStream(context.Background(), &testutils.MockServerTransportStream{})
|
||||||
|
_, err := server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.NoError(t, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices_PartitionCheck(t *testing.T) {
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Allow)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(fmt.Errorf("partition not supported"))
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
ctx := grpc.NewContextWithServerTransportStream(context.Background(), &testutils.MockServerTransportStream{})
|
||||||
|
|
||||||
|
resp, err := server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.EqualError(t, err, "rpc error: code = InvalidArgument desc = partition not supported")
|
||||||
|
require.Nil(t, resp)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices_Index(t *testing.T) {
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Allow)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(nil)
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
// Add config entry
|
||||||
|
entry := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "cache",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
fakeFSM.GetState().EnsureConfigEntry(1, entry)
|
||||||
|
|
||||||
|
headerStream := &testutils.MockServerTransportStream{}
|
||||||
|
|
||||||
|
ctx := grpc.NewContextWithServerTransportStream(context.Background(), headerStream)
|
||||||
|
resp, err := server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, 2, len(resp.Services))
|
||||||
|
require.Equal(t, []string{"1"}, headerStream.MD.Get("index"))
|
||||||
|
|
||||||
|
// Updating the index
|
||||||
|
fakeFSM.GetState().EnsureConfigEntry(2, entry)
|
||||||
|
|
||||||
|
headerStream = &testutils.MockServerTransportStream{}
|
||||||
|
|
||||||
|
ctx = grpc.NewContextWithServerTransportStream(context.Background(), headerStream)
|
||||||
|
resp, err = server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, 2, len(resp.Services))
|
||||||
|
require.Equal(t, []string{"2"}, headerStream.MD.Get("index"))
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetResolvedExportedServices_Metrics(t *testing.T) {
|
||||||
|
sink := metrics.NewInmemSink(5*time.Second, time.Minute)
|
||||||
|
cfg := metrics.DefaultConfig("consul")
|
||||||
|
metrics.NewGlobal(cfg, sink)
|
||||||
|
|
||||||
|
authorizer := acl.MockAuthorizer{}
|
||||||
|
authorizer.On("MeshRead", mock.Anything).Return(acl.Allow)
|
||||||
|
|
||||||
|
backend := &MockBackend{authorizer: &authorizer}
|
||||||
|
backend.On("EnterpriseCheckPartitions", mock.Anything).Return(nil)
|
||||||
|
|
||||||
|
fakeFSM := testutils.NewFakeBlockingFSM(t)
|
||||||
|
|
||||||
|
c := Config{
|
||||||
|
Backend: backend,
|
||||||
|
Logger: hclog.New(nil),
|
||||||
|
ForwardRPC: doForwardRPC,
|
||||||
|
FSMServer: fakeFSM,
|
||||||
|
}
|
||||||
|
server := NewServer(c)
|
||||||
|
|
||||||
|
// Add config entry
|
||||||
|
entry := &structs.ExportedServicesConfigEntry{
|
||||||
|
Name: "default",
|
||||||
|
Services: []structs.ExportedService{
|
||||||
|
{
|
||||||
|
Name: "db",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Peer: "west",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "cache",
|
||||||
|
Consumers: []structs.ServiceConsumer{
|
||||||
|
{
|
||||||
|
Peer: "east",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
fakeFSM.GetState().EnsureConfigEntry(1, entry)
|
||||||
|
|
||||||
|
ctx := grpc.NewContextWithServerTransportStream(context.Background(), &testutils.MockServerTransportStream{})
|
||||||
|
resp, err := server.GetResolvedExportedServices(ctx, &pbconfigentry.GetResolvedExportedServicesRequest{})
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.Equal(t, 2, len(resp.Services))
|
||||||
|
|
||||||
|
// Checking if metrics were added
|
||||||
|
require.NotNil(t, sink.Data()[0].Samples[`consul.configentry.get_resolved_exported_services`])
|
||||||
|
}
|
||||||
|
|
||||||
|
func doForwardRPC(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error) {
|
||||||
|
return false, nil
|
||||||
|
}
|
|
@ -9,6 +9,7 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"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"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
@ -70,6 +71,47 @@ func (f *FakeFSM) ReplaceStore(store *state.Store) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type FakeBlockingFSM struct {
|
||||||
|
store *state.Store
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewFakeBlockingFSM(t *testing.T) *FakeBlockingFSM {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
store := TestStateStore(t, nil)
|
||||||
|
|
||||||
|
fsm := &FakeBlockingFSM{store: store}
|
||||||
|
|
||||||
|
return fsm
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) GetState() *state.Store {
|
||||||
|
return f.store
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) ConsistentRead() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) DecrementBlockingQueries() uint64 {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) IncrementBlockingQueries() uint64 {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) GetShutdownChannel() chan struct{} {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) RPCQueryTimeout(queryTimeout time.Duration) time.Duration {
|
||||||
|
return queryTimeout
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *FakeBlockingFSM) SetQueryMeta(blockingquery.ResponseMeta, string) {
|
||||||
|
}
|
||||||
|
|
||||||
func SetupFSMAndPublisher(t *testing.T, config FakeFSMConfig) (*FakeFSM, state.EventPublisher) {
|
func SetupFSMAndPublisher(t *testing.T, config FakeFSMConfig) (*FakeFSM, state.EventPublisher) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
config.publisher = stream.NewEventPublisher(10 * time.Second)
|
config.publisher = stream.NewEventPublisher(10 * time.Second)
|
||||||
|
|
|
@ -0,0 +1,27 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: BUSL-1.1
|
||||||
|
|
||||||
|
package testutils
|
||||||
|
|
||||||
|
import "google.golang.org/grpc/metadata"
|
||||||
|
|
||||||
|
type MockServerTransportStream struct {
|
||||||
|
MD metadata.MD
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockServerTransportStream) Method() string {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockServerTransportStream) SetHeader(md metadata.MD) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockServerTransportStream) SendHeader(md metadata.MD) error {
|
||||||
|
m.MD = metadata.Join(m.MD, md)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *MockServerTransportStream) SetTrailer(md metadata.MD) error {
|
||||||
|
return nil
|
||||||
|
}
|
|
@ -44,3 +44,9 @@ func RequireAnyValidACLToken(resolver ACLResolver, token string) error {
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func RequireNotNil(v interface{}, name string) {
|
||||||
|
if v == nil {
|
||||||
|
panic(name + " is required")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -4,35 +4,36 @@ package middleware
|
||||||
import "github.com/hashicorp/consul/agent/consul/rate"
|
import "github.com/hashicorp/consul/agent/consul/rate"
|
||||||
|
|
||||||
var rpcRateLimitSpecs = map[string]rate.OperationSpec{
|
var rpcRateLimitSpecs = map[string]rate.OperationSpec{
|
||||||
"/hashicorp.consul.acl.ACLService/Login": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryACL},
|
"/hashicorp.consul.acl.ACLService/Login": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryACL},
|
||||||
"/hashicorp.consul.acl.ACLService/Logout": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryACL},
|
"/hashicorp.consul.acl.ACLService/Logout": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryACL},
|
||||||
"/hashicorp.consul.connectca.ConnectCAService/Sign": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryConnectCA},
|
"/hashicorp.consul.connectca.ConnectCAService/Sign": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryConnectCA},
|
||||||
"/hashicorp.consul.connectca.ConnectCAService/WatchRoots": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryConnectCA},
|
"/hashicorp.consul.connectca.ConnectCAService/WatchRoots": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryConnectCA},
|
||||||
"/hashicorp.consul.dataplane.DataplaneService/GetEnvoyBootstrapParams": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDataPlane},
|
"/hashicorp.consul.dataplane.DataplaneService/GetEnvoyBootstrapParams": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDataPlane},
|
||||||
"/hashicorp.consul.dataplane.DataplaneService/GetSupportedDataplaneFeatures": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDataPlane},
|
"/hashicorp.consul.dataplane.DataplaneService/GetSupportedDataplaneFeatures": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDataPlane},
|
||||||
"/hashicorp.consul.dns.DNSService/Query": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDNS},
|
"/hashicorp.consul.dns.DNSService/Query": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryDNS},
|
||||||
"/hashicorp.consul.internal.operator.OperatorService/TransferLeader": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryOperator},
|
"/hashicorp.consul.internal.configentry.ConfigEntryService/GetResolvedExportedServices": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryConfigEntry},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/Establish": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.operator.OperatorService/TransferLeader": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryOperator},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/GenerateToken": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/Establish": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/PeeringDelete": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/GenerateToken": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/PeeringList": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/PeeringDelete": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/PeeringRead": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/PeeringList": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/PeeringWrite": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/PeeringRead": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/TrustBundleListByService": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/PeeringWrite": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peering.PeeringService/TrustBundleRead": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
"/hashicorp.consul.internal.peering.PeeringService/TrustBundleListByService": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peerstream.PeerStreamService/ExchangeSecret": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeerStream},
|
"/hashicorp.consul.internal.peering.PeeringService/TrustBundleRead": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeering},
|
||||||
"/hashicorp.consul.internal.peerstream.PeerStreamService/StreamResources": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeerStream},
|
"/hashicorp.consul.internal.peerstream.PeerStreamService/ExchangeSecret": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryPeerStream},
|
||||||
"/hashicorp.consul.internal.storage.raft.ForwardingService/Delete": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.internal.peerstream.PeerStreamService/StreamResources": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryPeerStream},
|
||||||
"/hashicorp.consul.internal.storage.raft.ForwardingService/List": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.internal.storage.raft.ForwardingService/Delete": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.internal.storage.raft.ForwardingService/Read": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.internal.storage.raft.ForwardingService/List": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.internal.storage.raft.ForwardingService/Write": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.internal.storage.raft.ForwardingService/Read": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/Delete": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.internal.storage.raft.ForwardingService/Write": {Type: rate.OperationTypeExempt, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/List": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/Delete": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/ListByOwner": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/List": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/Read": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/ListByOwner": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/WatchList": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/Read": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/Write": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/WatchList": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.resource.ResourceService/WriteStatus": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
"/hashicorp.consul.resource.ResourceService/Write": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
||||||
"/hashicorp.consul.serverdiscovery.ServerDiscoveryService/WatchServers": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryServerDiscovery},
|
"/hashicorp.consul.resource.ResourceService/WriteStatus": {Type: rate.OperationTypeWrite, Category: rate.OperationCategoryResource},
|
||||||
"/subscribe.StateChangeSubscription/Subscribe": {Type: rate.OperationTypeRead, Category: rate.OperationCategorySubscribe},
|
"/hashicorp.consul.serverdiscovery.ServerDiscoveryService/WatchServers": {Type: rate.OperationTypeRead, Category: rate.OperationCategoryServerDiscovery},
|
||||||
|
"/subscribe.StateChangeSubscription/Subscribe": {Type: rate.OperationTypeRead, Category: rate.OperationCategorySubscribe},
|
||||||
}
|
}
|
||||||
|
|
|
@ -86,6 +86,7 @@ func init() {
|
||||||
registerEndpoint("/v1/internal/federation-states/mesh-gateways", []string{"GET"}, (*HTTPHandlers).FederationStateListMeshGateways)
|
registerEndpoint("/v1/internal/federation-states/mesh-gateways", []string{"GET"}, (*HTTPHandlers).FederationStateListMeshGateways)
|
||||||
registerEndpoint("/v1/internal/federation-state/", []string{"GET"}, (*HTTPHandlers).FederationStateGet)
|
registerEndpoint("/v1/internal/federation-state/", []string{"GET"}, (*HTTPHandlers).FederationStateGet)
|
||||||
registerEndpoint("/v1/discovery-chain/", []string{"GET", "POST"}, (*HTTPHandlers).DiscoveryChainRead)
|
registerEndpoint("/v1/discovery-chain/", []string{"GET", "POST"}, (*HTTPHandlers).DiscoveryChainRead)
|
||||||
|
registerEndpoint("/v1/exported-services", []string{"GET"}, (*HTTPHandlers).ExportedServices)
|
||||||
registerEndpoint("/v1/event/fire/", []string{"PUT"}, (*HTTPHandlers).EventFire)
|
registerEndpoint("/v1/event/fire/", []string{"PUT"}, (*HTTPHandlers).EventFire)
|
||||||
registerEndpoint("/v1/event/list", []string{"GET"}, (*HTTPHandlers).EventList)
|
registerEndpoint("/v1/event/list", []string{"GET"}, (*HTTPHandlers).EventList)
|
||||||
registerEndpoint("/v1/health/node/", []string{"GET"}, (*HTTPHandlers).HealthNodeChecks)
|
registerEndpoint("/v1/health/node/", []string{"GET"}, (*HTTPHandlers).HealthNodeChecks)
|
||||||
|
|
|
@ -3,7 +3,9 @@
|
||||||
|
|
||||||
package api
|
package api
|
||||||
|
|
||||||
import "encoding/json"
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
)
|
||||||
|
|
||||||
// ExportedServicesConfigEntry manages the exported services for a single admin partition.
|
// ExportedServicesConfigEntry manages the exported services for a single admin partition.
|
||||||
// Admin Partitions are a Consul Enterprise feature.
|
// Admin Partitions are a Consul Enterprise feature.
|
||||||
|
|
|
@ -0,0 +1,49 @@
|
||||||
|
// Copyright (c) HashiCorp, Inc.
|
||||||
|
// SPDX-License-Identifier: MPL-2.0
|
||||||
|
|
||||||
|
package api
|
||||||
|
|
||||||
|
type ResolvedExportedService struct {
|
||||||
|
// Service is the name of the service which is exported.
|
||||||
|
Service string
|
||||||
|
|
||||||
|
// Partition of the service
|
||||||
|
Partition string `json:",omitempty"`
|
||||||
|
|
||||||
|
// Namespace of the service
|
||||||
|
Namespace string `json:",omitempty"`
|
||||||
|
|
||||||
|
// Consumers is a list of downstream consumers of the service.
|
||||||
|
Consumers ResolvedConsumers
|
||||||
|
}
|
||||||
|
|
||||||
|
type ResolvedConsumers struct {
|
||||||
|
Peers []string `json:",omitempty"`
|
||||||
|
Partitions []string `json:",omitempty"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *Client) ExportedServices(q *QueryOptions) ([]ResolvedExportedService, *QueryMeta, error) {
|
||||||
|
|
||||||
|
r := c.newRequest("GET", "/v1/exported-services")
|
||||||
|
r.setQueryOptions(q)
|
||||||
|
rtt, resp, err := c.doRequest(r)
|
||||||
|
if err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
defer closeResponseBody(resp)
|
||||||
|
if err := requireOK(resp); err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
qm := &QueryMeta{}
|
||||||
|
parseQueryMeta(resp, qm)
|
||||||
|
qm.RequestTime = rtt
|
||||||
|
|
||||||
|
var expSvcs []ResolvedExportedService
|
||||||
|
|
||||||
|
if err := decodeBody(resp, &expSvcs); err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
return expSvcs, qm, nil
|
||||||
|
}
|
|
@ -62,7 +62,7 @@ const (
|
||||||
rpc %s(...) returns (...) {
|
rpc %s(...) returns (...) {
|
||||||
option (hashicorp.consul.internal.ratelimit.spec) = {
|
option (hashicorp.consul.internal.ratelimit.spec) = {
|
||||||
operation_type: OPERATION_TYPE_READ | OPERATION_TYPE_WRITE | OPERATION_TYPE_EXEMPT,
|
operation_type: OPERATION_TYPE_READ | OPERATION_TYPE_WRITE | OPERATION_TYPE_EXEMPT,
|
||||||
operation_category: OPERATION_CATEGORY_ACL | OPERATION_CATEGORY_PEER_STREAM | OPERATION_CATEGORY_CONNECT_CA | OPERATION_CATEGORY_PARTITION | OPERATION_CATEGORY_PEERING | OPERATION_CATEGORY_SERVER_DISCOVERY | OPERATION_CATEGORY_DATAPLANE | OPERATION_CATEGORY_DNS | OPERATION_CATEGORY_SUBSCRIBE | OPERATION_CATEGORY_OPERATOR | OPERATION_CATEGORY_RESOURCE,
|
operation_category: OPERATION_CATEGORY_ACL | OPERATION_CATEGORY_PEER_STREAM | OPERATION_CATEGORY_CONNECT_CA | OPERATION_CATEGORY_PARTITION | OPERATION_CATEGORY_PEERING | OPERATION_CATEGORY_SERVER_DISCOVERY | OPERATION_CATEGORY_DATAPLANE | OPERATION_CATEGORY_DNS | OPERATION_CATEGORY_SUBSCRIBE | OPERATION_CATEGORY_OPERATOR | OPERATION_CATEGORY_RESOURCE | OPERATION_CATEGORY_CONFIGENTRY,
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -128,6 +128,8 @@ func (s spec) GoOperationCategory() string {
|
||||||
return "rate.OperationCategoryPartition"
|
return "rate.OperationCategoryPartition"
|
||||||
case "OPERATION_CATEGORY_PEERING":
|
case "OPERATION_CATEGORY_PEERING":
|
||||||
return "rate.OperationCategoryPeering"
|
return "rate.OperationCategoryPeering"
|
||||||
|
case "OPERATION_CATEGORY_CONFIGENTRY":
|
||||||
|
return "rate.OperationCategoryConfigEntry"
|
||||||
case "OPERATION_CATEGORY_SERVER_DISCOVERY":
|
case "OPERATION_CATEGORY_SERVER_DISCOVERY":
|
||||||
return "rate.OperationCategoryServerDiscovery"
|
return "rate.OperationCategoryServerDiscovery"
|
||||||
case "OPERATION_CATEGORY_DATAPLANE":
|
case "OPERATION_CATEGORY_DATAPLANE":
|
||||||
|
|
|
@ -93,6 +93,7 @@ const (
|
||||||
OperationCategory_OPERATION_CATEGORY_SUBSCRIBE OperationCategory = 9
|
OperationCategory_OPERATION_CATEGORY_SUBSCRIBE OperationCategory = 9
|
||||||
OperationCategory_OPERATION_CATEGORY_OPERATOR OperationCategory = 10
|
OperationCategory_OPERATION_CATEGORY_OPERATOR OperationCategory = 10
|
||||||
OperationCategory_OPERATION_CATEGORY_RESOURCE OperationCategory = 11
|
OperationCategory_OPERATION_CATEGORY_RESOURCE OperationCategory = 11
|
||||||
|
OperationCategory_OPERATION_CATEGORY_CONFIGENTRY OperationCategory = 12
|
||||||
)
|
)
|
||||||
|
|
||||||
// Enum value maps for OperationCategory.
|
// Enum value maps for OperationCategory.
|
||||||
|
@ -110,6 +111,7 @@ var (
|
||||||
9: "OPERATION_CATEGORY_SUBSCRIBE",
|
9: "OPERATION_CATEGORY_SUBSCRIBE",
|
||||||
10: "OPERATION_CATEGORY_OPERATOR",
|
10: "OPERATION_CATEGORY_OPERATOR",
|
||||||
11: "OPERATION_CATEGORY_RESOURCE",
|
11: "OPERATION_CATEGORY_RESOURCE",
|
||||||
|
12: "OPERATION_CATEGORY_CONFIGENTRY",
|
||||||
}
|
}
|
||||||
OperationCategory_value = map[string]int32{
|
OperationCategory_value = map[string]int32{
|
||||||
"OPERATION_CATEGORY_UNSPECIFIED": 0,
|
"OPERATION_CATEGORY_UNSPECIFIED": 0,
|
||||||
|
@ -124,6 +126,7 @@ var (
|
||||||
"OPERATION_CATEGORY_SUBSCRIBE": 9,
|
"OPERATION_CATEGORY_SUBSCRIBE": 9,
|
||||||
"OPERATION_CATEGORY_OPERATOR": 10,
|
"OPERATION_CATEGORY_OPERATOR": 10,
|
||||||
"OPERATION_CATEGORY_RESOURCE": 11,
|
"OPERATION_CATEGORY_RESOURCE": 11,
|
||||||
|
"OPERATION_CATEGORY_CONFIGENTRY": 12,
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -257,7 +260,7 @@ var file_annotations_ratelimit_ratelimit_proto_rawDesc = []byte{
|
||||||
0x4d, 0x50, 0x54, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49,
|
0x4d, 0x50, 0x54, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49,
|
||||||
0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x02, 0x12, 0x18,
|
0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x02, 0x12, 0x18,
|
||||||
0x0a, 0x14, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45,
|
0x0a, 0x14, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45,
|
||||||
0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x03, 0x2a, 0xa7, 0x03, 0x0a, 0x11, 0x4f, 0x70, 0x65,
|
0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x03, 0x2a, 0xcb, 0x03, 0x0a, 0x11, 0x4f, 0x70, 0x65,
|
||||||
0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x12, 0x22,
|
0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x12, 0x22,
|
||||||
0x0a, 0x1e, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x54, 0x45,
|
0x0a, 0x1e, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x54, 0x45,
|
||||||
0x47, 0x4f, 0x52, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44,
|
0x47, 0x4f, 0x52, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44,
|
||||||
|
@ -284,32 +287,34 @@ var file_annotations_ratelimit_ratelimit_proto_rawDesc = []byte{
|
||||||
0x54, 0x45, 0x47, 0x4f, 0x52, 0x59, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x10,
|
0x54, 0x45, 0x47, 0x4f, 0x52, 0x59, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x10,
|
||||||
0x0a, 0x12, 0x1f, 0x0a, 0x1b, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43,
|
0x0a, 0x12, 0x1f, 0x0a, 0x1b, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43,
|
||||||
0x41, 0x54, 0x45, 0x47, 0x4f, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45,
|
0x41, 0x54, 0x45, 0x47, 0x4f, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45,
|
||||||
0x10, 0x0b, 0x3a, 0x5e, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f,
|
0x10, 0x0b, 0x12, 0x22, 0x0a, 0x1e, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f,
|
||||||
0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74,
|
0x43, 0x41, 0x54, 0x45, 0x47, 0x4f, 0x52, 0x59, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x45,
|
||||||
0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xec, 0x40, 0x20, 0x01, 0x28,
|
0x4e, 0x54, 0x52, 0x59, 0x10, 0x0c, 0x3a, 0x5e, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x1e,
|
||||||
0x0b, 0x32, 0x29, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f,
|
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
|
||||||
0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x72, 0x61,
|
0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xec,
|
||||||
0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x2e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70,
|
0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
|
||||||
0x65, 0x63, 0x42, 0xa9, 0x02, 0x0a, 0x27, 0x63, 0x6f, 0x6d, 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, 0x72, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x2e, 0x53, 0x70, 0x65, 0x63,
|
||||||
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x72, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x42, 0x0e,
|
0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0xa9, 0x02, 0x0a, 0x27, 0x63, 0x6f, 0x6d, 0x2e, 0x68,
|
||||||
0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01,
|
0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e,
|
||||||
0x5a, 0x3e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73,
|
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x72, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d,
|
||||||
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72,
|
0x69, 0x74, 0x42, 0x0e, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x50, 0x72, 0x6f,
|
||||||
0x6f, 0x74, 0x6f, 0x2d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74,
|
0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
|
||||||
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x72, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74,
|
0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75,
|
||||||
0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x52, 0xaa, 0x02, 0x23, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63,
|
0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x2f, 0x61,
|
||||||
0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72,
|
0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x72, 0x61, 0x74, 0x65, 0x6c,
|
||||||
0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0xca, 0x02, 0x23,
|
0x69, 0x6d, 0x69, 0x74, 0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x52, 0xaa, 0x02, 0x23, 0x48, 0x61,
|
||||||
0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c,
|
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49,
|
||||||
0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69,
|
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69,
|
||||||
0x6d, 0x69, 0x74, 0xe2, 0x02, 0x2f, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c,
|
0x74, 0xca, 0x02, 0x23, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f,
|
||||||
0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c,
|
0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x52, 0x61,
|
||||||
0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74,
|
0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0xe2, 0x02, 0x2f, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63,
|
||||||
0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x26, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
|
0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72,
|
||||||
0x70, 0x3a, 0x3a, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x3a, 0x3a, 0x49, 0x6e, 0x74, 0x65, 0x72,
|
0x6e, 0x61, 0x6c, 0x5c, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x5c, 0x47, 0x50,
|
||||||
0x6e, 0x61, 0x6c, 0x3a, 0x3a, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x62, 0x06,
|
0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x26, 0x48, 0x61, 0x73, 0x68,
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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, 0x52, 0x61, 0x74, 0x65, 0x6c, 0x69, 0x6d,
|
||||||
|
0x69, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
|
|
@ -29,6 +29,7 @@ enum OperationCategory {
|
||||||
OPERATION_CATEGORY_SUBSCRIBE = 9;
|
OPERATION_CATEGORY_SUBSCRIBE = 9;
|
||||||
OPERATION_CATEGORY_OPERATOR = 10;
|
OPERATION_CATEGORY_OPERATOR = 10;
|
||||||
OPERATION_CATEGORY_RESOURCE = 11;
|
OPERATION_CATEGORY_RESOURCE = 11;
|
||||||
|
OPERATION_CATEGORY_CONFIGENTRY = 12;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Spec describes the kind of rate limit that will be applied to this RPC.
|
// Spec describes the kind of rate limit that will be applied to this RPC.
|
||||||
|
|
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
"github.com/hashicorp/consul/agent/structs"
|
"github.com/hashicorp/consul/agent/structs"
|
||||||
|
"github.com/hashicorp/consul/api"
|
||||||
"github.com/hashicorp/consul/proto/private/pbcommon"
|
"github.com/hashicorp/consul/proto/private/pbcommon"
|
||||||
"github.com/hashicorp/consul/types"
|
"github.com/hashicorp/consul/types"
|
||||||
)
|
)
|
||||||
|
@ -603,3 +604,18 @@ func serviceRefFromStructs(a structs.ServiceRouteReferences) map[string]*ListOfR
|
||||||
}
|
}
|
||||||
return m
|
return m
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (r *ResolvedExportedService) ToAPI() *api.ResolvedExportedService {
|
||||||
|
var t api.ResolvedExportedService
|
||||||
|
|
||||||
|
t.Service = r.Service
|
||||||
|
if r.EnterpriseMeta != nil {
|
||||||
|
t.Namespace = r.EnterpriseMeta.Namespace
|
||||||
|
t.Partition = r.EnterpriseMeta.Partition
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Consumers.Peers = r.Consumers.Peers
|
||||||
|
t.Consumers.Partitions = r.Consumers.Partitions
|
||||||
|
|
||||||
|
return &t
|
||||||
|
}
|
||||||
|
|
|
@ -7,6 +7,46 @@ import (
|
||||||
"google.golang.org/protobuf/proto"
|
"google.golang.org/protobuf/proto"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// MarshalBinary implements encoding.BinaryMarshaler
|
||||||
|
func (msg *GetResolvedExportedServicesRequest) MarshalBinary() ([]byte, error) {
|
||||||
|
return proto.Marshal(msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnmarshalBinary implements encoding.BinaryUnmarshaler
|
||||||
|
func (msg *GetResolvedExportedServicesRequest) UnmarshalBinary(b []byte) error {
|
||||||
|
return proto.Unmarshal(b, msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalBinary implements encoding.BinaryMarshaler
|
||||||
|
func (msg *GetResolvedExportedServicesResponse) MarshalBinary() ([]byte, error) {
|
||||||
|
return proto.Marshal(msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnmarshalBinary implements encoding.BinaryUnmarshaler
|
||||||
|
func (msg *GetResolvedExportedServicesResponse) UnmarshalBinary(b []byte) error {
|
||||||
|
return proto.Unmarshal(b, msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalBinary implements encoding.BinaryMarshaler
|
||||||
|
func (msg *ResolvedExportedService) MarshalBinary() ([]byte, error) {
|
||||||
|
return proto.Marshal(msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnmarshalBinary implements encoding.BinaryUnmarshaler
|
||||||
|
func (msg *ResolvedExportedService) UnmarshalBinary(b []byte) error {
|
||||||
|
return proto.Unmarshal(b, msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalBinary implements encoding.BinaryMarshaler
|
||||||
|
func (msg *Consumers) MarshalBinary() ([]byte, error) {
|
||||||
|
return proto.Marshal(msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnmarshalBinary implements encoding.BinaryUnmarshaler
|
||||||
|
func (msg *Consumers) UnmarshalBinary(b []byte) error {
|
||||||
|
return proto.Unmarshal(b, msg)
|
||||||
|
}
|
||||||
|
|
||||||
// MarshalBinary implements encoding.BinaryMarshaler
|
// MarshalBinary implements encoding.BinaryMarshaler
|
||||||
func (msg *ConfigEntry) MarshalBinary() ([]byte, error) {
|
func (msg *ConfigEntry) MarshalBinary() ([]byte, error) {
|
||||||
return proto.Marshal(msg)
|
return proto.Marshal(msg)
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -5,10 +5,40 @@ syntax = "proto3";
|
||||||
|
|
||||||
package hashicorp.consul.internal.configentry;
|
package hashicorp.consul.internal.configentry;
|
||||||
|
|
||||||
|
import "annotations/ratelimit/ratelimit.proto";
|
||||||
import "google/protobuf/duration.proto";
|
import "google/protobuf/duration.proto";
|
||||||
import "google/protobuf/timestamp.proto";
|
import "google/protobuf/timestamp.proto";
|
||||||
import "private/pbcommon/common.proto";
|
import "private/pbcommon/common.proto";
|
||||||
|
|
||||||
|
// ConfigEntryService handles operations related to config entries
|
||||||
|
service ConfigEntryService {
|
||||||
|
rpc GetResolvedExportedServices(GetResolvedExportedServicesRequest) returns (GetResolvedExportedServicesResponse) {
|
||||||
|
option (hashicorp.consul.internal.ratelimit.spec) = {
|
||||||
|
operation_type: OPERATION_TYPE_READ,
|
||||||
|
operation_category: OPERATION_CATEGORY_CONFIGENTRY
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
message GetResolvedExportedServicesRequest {
|
||||||
|
string Partition = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
message GetResolvedExportedServicesResponse {
|
||||||
|
repeated ResolvedExportedService services = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
message ResolvedExportedService {
|
||||||
|
string Service = 1;
|
||||||
|
common.EnterpriseMeta EnterpriseMeta = 2;
|
||||||
|
Consumers Consumers = 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
message Consumers {
|
||||||
|
repeated string Peers = 1;
|
||||||
|
repeated string Partitions = 2;
|
||||||
|
}
|
||||||
|
|
||||||
enum Kind {
|
enum Kind {
|
||||||
KindUnknown = 0;
|
KindUnknown = 0;
|
||||||
KindMeshConfig = 1;
|
KindMeshConfig = 1;
|
||||||
|
|
|
@ -0,0 +1,103 @@
|
||||||
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
|
// versions:
|
||||||
|
// - protoc-gen-go-grpc v1.2.0
|
||||||
|
// - protoc (unknown)
|
||||||
|
// source: private/pbconfigentry/config_entry.proto
|
||||||
|
|
||||||
|
package pbconfigentry
|
||||||
|
|
||||||
|
import (
|
||||||
|
context "context"
|
||||||
|
grpc "google.golang.org/grpc"
|
||||||
|
codes "google.golang.org/grpc/codes"
|
||||||
|
status "google.golang.org/grpc/status"
|
||||||
|
)
|
||||||
|
|
||||||
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
|
// ConfigEntryServiceClient is the client API for ConfigEntryService service.
|
||||||
|
//
|
||||||
|
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
|
||||||
|
type ConfigEntryServiceClient interface {
|
||||||
|
GetResolvedExportedServices(ctx context.Context, in *GetResolvedExportedServicesRequest, opts ...grpc.CallOption) (*GetResolvedExportedServicesResponse, error)
|
||||||
|
}
|
||||||
|
|
||||||
|
type configEntryServiceClient struct {
|
||||||
|
cc grpc.ClientConnInterface
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewConfigEntryServiceClient(cc grpc.ClientConnInterface) ConfigEntryServiceClient {
|
||||||
|
return &configEntryServiceClient{cc}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *configEntryServiceClient) GetResolvedExportedServices(ctx context.Context, in *GetResolvedExportedServicesRequest, opts ...grpc.CallOption) (*GetResolvedExportedServicesResponse, error) {
|
||||||
|
out := new(GetResolvedExportedServicesResponse)
|
||||||
|
err := c.cc.Invoke(ctx, "/hashicorp.consul.internal.configentry.ConfigEntryService/GetResolvedExportedServices", in, out, opts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// ConfigEntryServiceServer is the server API for ConfigEntryService service.
|
||||||
|
// All implementations should embed UnimplementedConfigEntryServiceServer
|
||||||
|
// for forward compatibility
|
||||||
|
type ConfigEntryServiceServer interface {
|
||||||
|
GetResolvedExportedServices(context.Context, *GetResolvedExportedServicesRequest) (*GetResolvedExportedServicesResponse, error)
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnimplementedConfigEntryServiceServer should be embedded to have forward compatible implementations.
|
||||||
|
type UnimplementedConfigEntryServiceServer struct {
|
||||||
|
}
|
||||||
|
|
||||||
|
func (UnimplementedConfigEntryServiceServer) GetResolvedExportedServices(context.Context, *GetResolvedExportedServicesRequest) (*GetResolvedExportedServicesResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method GetResolvedExportedServices not implemented")
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnsafeConfigEntryServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||||
|
// Use of this interface is not recommended, as added methods to ConfigEntryServiceServer will
|
||||||
|
// result in compilation errors.
|
||||||
|
type UnsafeConfigEntryServiceServer interface {
|
||||||
|
mustEmbedUnimplementedConfigEntryServiceServer()
|
||||||
|
}
|
||||||
|
|
||||||
|
func RegisterConfigEntryServiceServer(s grpc.ServiceRegistrar, srv ConfigEntryServiceServer) {
|
||||||
|
s.RegisterService(&ConfigEntryService_ServiceDesc, srv)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _ConfigEntryService_GetResolvedExportedServices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(GetResolvedExportedServicesRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(ConfigEntryServiceServer).GetResolvedExportedServices(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: "/hashicorp.consul.internal.configentry.ConfigEntryService/GetResolvedExportedServices",
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(ConfigEntryServiceServer).GetResolvedExportedServices(ctx, req.(*GetResolvedExportedServicesRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ConfigEntryService_ServiceDesc is the grpc.ServiceDesc for ConfigEntryService service.
|
||||||
|
// It's only intended for direct use with grpc.RegisterService,
|
||||||
|
// and not to be introspected or modified (even as a copy)
|
||||||
|
var ConfigEntryService_ServiceDesc = grpc.ServiceDesc{
|
||||||
|
ServiceName: "hashicorp.consul.internal.configentry.ConfigEntryService",
|
||||||
|
HandlerType: (*ConfigEntryServiceServer)(nil),
|
||||||
|
Methods: []grpc.MethodDesc{
|
||||||
|
{
|
||||||
|
MethodName: "GetResolvedExportedServices",
|
||||||
|
Handler: _ConfigEntryService_GetResolvedExportedServices_Handler,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
Streams: []grpc.StreamDesc{},
|
||||||
|
Metadata: "private/pbconfigentry/config_entry.proto",
|
||||||
|
}
|
Loading…
Reference in New Issue