mirror of https://github.com/status-im/consul.git
[OSS] feat: add experiments flag for v2 dns and skeleton interfaces (#20115)
feat: add experiments flag for v2 dns and skeleton interfaces
This commit is contained in:
parent
fa6eb61f70
commit
d52e80b619
136
agent/agent.go
136
agent/agent.go
|
@ -47,6 +47,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul"
|
||||
rpcRate "github.com/hashicorp/consul/agent/consul/rate"
|
||||
"github.com/hashicorp/consul/agent/consul/servercert"
|
||||
"github.com/hashicorp/consul/agent/discovery"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
grpcDNS "github.com/hashicorp/consul/agent/grpc-external/services/dns"
|
||||
|
@ -67,6 +68,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/xds"
|
||||
"github.com/hashicorp/consul/api"
|
||||
"github.com/hashicorp/consul/api/watch"
|
||||
libdns "github.com/hashicorp/consul/internal/dnsutil"
|
||||
proxytracker "github.com/hashicorp/consul/internal/mesh/proxy-tracker"
|
||||
"github.com/hashicorp/consul/ipaddr"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
|
@ -217,6 +219,14 @@ type notifier interface {
|
|||
Notify(string) error
|
||||
}
|
||||
|
||||
// dnsServer abstracts the V1 and V2 implementations of the DNS server.
|
||||
type dnsServer interface {
|
||||
GetAddr() string
|
||||
ListenAndServe(string, string, func()) error
|
||||
ReloadConfig(*config.RuntimeConfig) error
|
||||
Shutdown()
|
||||
}
|
||||
|
||||
// Agent is the long running process that is run on every machine.
|
||||
// It exposes an RPC interface that is used by the CLI to control the
|
||||
// agent. The agent runs the query interfaces like HTTP, DNS, and RPC.
|
||||
|
@ -336,7 +346,11 @@ type Agent struct {
|
|||
endpointsLock sync.RWMutex
|
||||
|
||||
// dnsServer provides the DNS API
|
||||
dnsServers []*DNSServer
|
||||
dnsServers []dnsServer
|
||||
|
||||
// catalogDataFetcher is used as an interface to the catalog for service discovery
|
||||
// (aka DNS). Only applicable to the V2 DNS server (agent/dns).
|
||||
catalogDataFetcher discovery.CatalogDataFetcher
|
||||
|
||||
// apiServers listening for connections. If any of these server goroutines
|
||||
// fail, the agent will be shutdown.
|
||||
|
@ -397,7 +411,7 @@ type Agent struct {
|
|||
// they can update their internal state.
|
||||
configReloaders []ConfigReloader
|
||||
|
||||
// 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.
|
||||
rpcClientHealth *health.Client
|
||||
rpcClientConfigEntry *configentry.Client
|
||||
|
@ -729,6 +743,10 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
if a.baseDeps.UseV2Resources() {
|
||||
return fmt.Errorf("can't start agent: client agents are not supported with v2 resources")
|
||||
}
|
||||
|
||||
a.externalGRPCServer = external.NewServer(
|
||||
a.logger.Named("grpc.external"),
|
||||
metrics.Default(),
|
||||
|
@ -836,9 +854,16 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
}
|
||||
|
||||
// start DNS servers
|
||||
if err := a.listenAndServeDNS(); err != nil {
|
||||
if a.baseDeps.UseV2DNS() {
|
||||
a.logger.Warn("DNS v2 is under construction")
|
||||
if err := a.listenAndServeV2DNS(); err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
if err := a.listenAndServeV1DNS(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Configure the http connection limiter.
|
||||
a.httpConnLimiter.SetConfig(connlimit.Config{
|
||||
|
@ -1016,7 +1041,7 @@ func (a *Agent) listenAndServeGRPC(proxyTracker *proxytracker.ProxyTracker, serv
|
|||
return nil
|
||||
}
|
||||
|
||||
func (a *Agent) listenAndServeDNS() error {
|
||||
func (a *Agent) listenAndServeV1DNS() error {
|
||||
notif := make(chan net.Addr, len(a.config.DNSAddrs))
|
||||
errCh := make(chan error, len(a.config.DNSAddrs))
|
||||
for _, addr := range a.config.DNSAddrs {
|
||||
|
@ -1068,6 +1093,80 @@ func (a *Agent) listenAndServeDNS() error {
|
|||
return merr.ErrorOrNil()
|
||||
}
|
||||
|
||||
func (a *Agent) listenAndServeV2DNS() error {
|
||||
|
||||
// Check the catalog version and decide which implementation of the data fetcher to implement
|
||||
if a.baseDeps.UseV2Resources() {
|
||||
a.catalogDataFetcher = discovery.NewV2DataFetcher(a.config)
|
||||
} else {
|
||||
a.catalogDataFetcher = discovery.NewV1DataFetcher(a.config)
|
||||
}
|
||||
|
||||
// Generate a Query Processor with the appropriate data fetcher
|
||||
processor := discovery.NewQueryProcessor(a.catalogDataFetcher)
|
||||
|
||||
notif := make(chan net.Addr, len(a.config.DNSAddrs))
|
||||
errCh := make(chan error, len(a.config.DNSAddrs))
|
||||
|
||||
// create server
|
||||
cfg := dns.Config{
|
||||
AgentConfig: a.config,
|
||||
EntMeta: a.AgentEnterpriseMeta(), // TODO (v2-dns): does this even work for v2 tenancy?
|
||||
Logger: a.logger,
|
||||
Processor: processor,
|
||||
TokenFunc: a.getTokenFunc(),
|
||||
}
|
||||
|
||||
for _, addr := range a.config.DNSAddrs {
|
||||
s, err := dns.NewServer(cfg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
a.dnsServers = append(a.dnsServers, s)
|
||||
|
||||
// start server
|
||||
a.wgServers.Add(1)
|
||||
go func(addr net.Addr) {
|
||||
defer a.wgServers.Done()
|
||||
err := s.ListenAndServe(addr.Network(), addr.String(), func() { notif <- addr })
|
||||
if err != nil && !strings.Contains(err.Error(), "accept") {
|
||||
errCh <- err
|
||||
}
|
||||
}(addr)
|
||||
}
|
||||
|
||||
// TODO(v2-dns): implement a new grpcDNS proxy that takes in the new Router object.
|
||||
//s, _ := dns.NewServer(cfg)
|
||||
//
|
||||
//grpcDNS.NewServer(grpcDNS.Config{
|
||||
// Logger: a.logger.Named("grpc-api.dns"),
|
||||
// DNSServeMux: s.mux,
|
||||
// LocalAddr: grpcDNS.LocalAddr{IP: net.IPv4(127, 0, 0, 1), Port: a.config.GRPCPort},
|
||||
//}).Register(a.externalGRPCServer)
|
||||
//
|
||||
//a.dnsServers = append(a.dnsServers, s)
|
||||
|
||||
// wait for servers to be up
|
||||
timeout := time.After(time.Second)
|
||||
var merr *multierror.Error
|
||||
for range a.config.DNSAddrs {
|
||||
select {
|
||||
case addr := <-notif:
|
||||
a.logger.Info("Started DNS server",
|
||||
"address", addr.String(),
|
||||
"network", addr.Network(),
|
||||
)
|
||||
|
||||
case err := <-errCh:
|
||||
merr = multierror.Append(merr, err)
|
||||
case <-timeout:
|
||||
merr = multierror.Append(merr, fmt.Errorf("agent: timeout starting DNS servers"))
|
||||
return merr.ErrorOrNil()
|
||||
}
|
||||
}
|
||||
return merr.ErrorOrNil()
|
||||
}
|
||||
|
||||
// startListeners will return a net.Listener for every address unless an
|
||||
// error is encountered, in which case it will close all previously opened
|
||||
// listeners and return the error.
|
||||
|
@ -1803,15 +1902,8 @@ func (a *Agent) ShutdownEndpoints() {
|
|||
ctx := context.TODO()
|
||||
|
||||
for _, srv := range a.dnsServers {
|
||||
if srv.Server != nil {
|
||||
a.logger.Info("Stopping server",
|
||||
"protocol", "DNS",
|
||||
"address", srv.Server.Addr,
|
||||
"network", srv.Server.Net,
|
||||
)
|
||||
srv.Shutdown()
|
||||
}
|
||||
}
|
||||
a.dnsServers = nil
|
||||
|
||||
a.apiServers.Shutdown(ctx)
|
||||
|
@ -2645,13 +2737,13 @@ func (a *Agent) validateService(service *structs.NodeService, chkTypes []*struct
|
|||
}
|
||||
|
||||
// Warn if the service name is incompatible with DNS
|
||||
if dns.InvalidNameRe.MatchString(service.Service) {
|
||||
if libdns.InvalidNameRe.MatchString(service.Service) {
|
||||
a.logger.Warn("Service name will not be discoverable "+
|
||||
"via DNS due to invalid characters. Valid characters include "+
|
||||
"all alpha-numerics and dashes.",
|
||||
"service", service.Service,
|
||||
)
|
||||
} else if len(service.Service) > dns.MaxLabelLength {
|
||||
} else if len(service.Service) > libdns.MaxLabelLength {
|
||||
a.logger.Warn("Service name will not be discoverable "+
|
||||
"via DNS due to it being too long. Valid lengths are between "+
|
||||
"1 and 63 bytes.",
|
||||
|
@ -2661,13 +2753,13 @@ func (a *Agent) validateService(service *structs.NodeService, chkTypes []*struct
|
|||
|
||||
// Warn if any tags are incompatible with DNS
|
||||
for _, tag := range service.Tags {
|
||||
if dns.InvalidNameRe.MatchString(tag) {
|
||||
if libdns.InvalidNameRe.MatchString(tag) {
|
||||
a.logger.Debug("Service tag will not be discoverable "+
|
||||
"via DNS due to invalid characters. Valid characters include "+
|
||||
"all alpha-numerics and dashes.",
|
||||
"tag", tag,
|
||||
)
|
||||
} else if len(tag) > dns.MaxLabelLength {
|
||||
} else if len(tag) > libdns.MaxLabelLength {
|
||||
a.logger.Debug("Service tag will not be discoverable "+
|
||||
"via DNS due to it being too long. Valid lengths are between "+
|
||||
"1 and 63 bytes.",
|
||||
|
@ -4286,6 +4378,10 @@ func (a *Agent) reloadConfigInternal(newCfg *config.RuntimeConfig) error {
|
|||
return fmt.Errorf("Failed reloading dns config : %v", err)
|
||||
}
|
||||
}
|
||||
// This field is only populated for the V2 DNS server
|
||||
if a.catalogDataFetcher != nil {
|
||||
a.catalogDataFetcher.LoadConfig(newCfg)
|
||||
}
|
||||
|
||||
err := a.reloadEnterprise(newCfg)
|
||||
if err != nil {
|
||||
|
@ -4751,3 +4847,13 @@ func defaultIfEmpty(val, defaultVal string) string {
|
|||
}
|
||||
return defaultVal
|
||||
}
|
||||
|
||||
func (a *Agent) getTokenFunc() func() string {
|
||||
return func() string {
|
||||
if a.tokens.DNSToken() != "" {
|
||||
return a.tokens.DNSToken()
|
||||
} else {
|
||||
return a.tokens.UserToken()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,11 +34,11 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
|
||||
consulrate "github.com/hashicorp/consul/agent/consul/rate"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
hcpconfig "github.com/hashicorp/consul/agent/hcp/config"
|
||||
"github.com/hashicorp/consul/agent/rpc/middleware"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/internal/dnsutil"
|
||||
"github.com/hashicorp/consul/ipaddr"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/lib/stringslice"
|
||||
|
@ -1287,7 +1287,7 @@ func (b *builder) validate(rt RuntimeConfig) error {
|
|||
switch {
|
||||
case rt.NodeName == "":
|
||||
return fmt.Errorf("node_name cannot be empty")
|
||||
case dns.InvalidNameRe.MatchString(rt.NodeName):
|
||||
case dnsutil.InvalidNameRe.MatchString(rt.NodeName):
|
||||
b.warn("Node name %q will not be discoverable "+
|
||||
"via DNS due to invalid characters. Valid characters include "+
|
||||
"all alpha-numerics and dashes.", rt.NodeName)
|
||||
|
@ -1295,7 +1295,7 @@ func (b *builder) validate(rt RuntimeConfig) error {
|
|||
// todo(kyhavlov): Add stronger validation here for node names.
|
||||
b.warn("Found invalid characters in node name %q - whitespace and quotes "+
|
||||
"(', \", `) cannot be used with auto-config.", rt.NodeName)
|
||||
case len(rt.NodeName) > dns.MaxLabelLength:
|
||||
case len(rt.NodeName) > dnsutil.MaxLabelLength:
|
||||
b.warn("Node name %q will not be discoverable "+
|
||||
"via DNS due to it being too long. Valid lengths are between "+
|
||||
"1 and 63 bytes.", rt.NodeName)
|
||||
|
@ -1827,14 +1827,14 @@ func (b *builder) meshGatewayConfVal(mgConf *MeshGatewayConfig) structs.MeshGate
|
|||
return cfg
|
||||
}
|
||||
|
||||
func (b *builder) dnsRecursorStrategyVal(v string) dns.RecursorStrategy {
|
||||
var out dns.RecursorStrategy
|
||||
func (b *builder) dnsRecursorStrategyVal(v string) structs.RecursorStrategy {
|
||||
var out structs.RecursorStrategy
|
||||
|
||||
switch dns.RecursorStrategy(v) {
|
||||
case dns.RecursorStrategyRandom:
|
||||
out = dns.RecursorStrategyRandom
|
||||
case dns.RecursorStrategySequential, "":
|
||||
out = dns.RecursorStrategySequential
|
||||
switch structs.RecursorStrategy(v) {
|
||||
case structs.RecursorStrategyRandom:
|
||||
out = structs.RecursorStrategyRandom
|
||||
case structs.RecursorStrategySequential, "":
|
||||
out = structs.RecursorStrategySequential
|
||||
default:
|
||||
b.err = multierror.Append(b.err, fmt.Errorf("dns_config.recursor_strategy: invalid strategy: %q", v))
|
||||
}
|
||||
|
|
|
@ -16,7 +16,6 @@ import (
|
|||
"github.com/hashicorp/consul/agent/cache"
|
||||
"github.com/hashicorp/consul/agent/consul"
|
||||
consulrate "github.com/hashicorp/consul/agent/consul/rate"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
hcpconfig "github.com/hashicorp/consul/agent/hcp/config"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/token"
|
||||
|
@ -253,7 +252,7 @@ type RuntimeConfig struct {
|
|||
// client agents try the first server in the list every time.
|
||||
//
|
||||
// hcl: dns_config { recursor_strategy = "(random|sequential)" }
|
||||
DNSRecursorStrategy dns.RecursorStrategy
|
||||
DNSRecursorStrategy structs.RecursorStrategy
|
||||
|
||||
// DNSRecursorTimeout specifies the timeout in seconds
|
||||
// for Consul's internal dns client used for recursion.
|
||||
|
|
|
@ -11,12 +11,12 @@ import (
|
|||
"regexp"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/internal/dnsutil"
|
||||
|
||||
bexpr "github.com/hashicorp/go-bexpr"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/lib/template"
|
||||
"github.com/hashicorp/consul/proto/private/pbautoconf"
|
||||
|
@ -80,7 +80,7 @@ func (a *jwtAuthorizer) Authorize(req *pbautoconf.AutoConfigRequest) (AutoConfig
|
|||
if invalidSegmentName.MatchString(req.Segment) {
|
||||
return AutoConfigOptions{}, fmt.Errorf("Invalid request field. %v = `%v`", "segment", req.Segment)
|
||||
}
|
||||
if req.Partition != "" && !dns.IsValidLabel(req.Partition) {
|
||||
if req.Partition != "" && !dnsutil.IsValidLabel(req.Partition) {
|
||||
return AutoConfigOptions{}, fmt.Errorf("Invalid request field. %v = `%v`", "partition", req.Partition)
|
||||
}
|
||||
|
||||
|
|
|
@ -4,9 +4,10 @@
|
|||
package consul
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/consul/lib/stringslice"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/hashicorp/consul/lib/stringslice"
|
||||
|
||||
"github.com/hashicorp/consul-net-rpc/net/rpc"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
|
||||
|
@ -49,6 +50,15 @@ type Deps struct {
|
|||
EnterpriseDeps
|
||||
}
|
||||
|
||||
// UseV2DNS returns true if "v2-dns" is present in the Experiments
|
||||
// array of the agent config. It is assumed if the v2 resource APIs are enabled.
|
||||
func (d Deps) UseV2DNS() bool {
|
||||
if stringslice.Contains(d.Experiments, V2DNSExperimentName) || d.UseV2Resources() {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// UseV2Resources returns true if "resource-apis" is present in the Experiments
|
||||
// array of the agent config.
|
||||
func (d Deps) UseV2Resources() bool {
|
||||
|
|
|
@ -142,6 +142,7 @@ const (
|
|||
LeaderTransferMinVersion = "1.6.0"
|
||||
|
||||
CatalogResourceExperimentName = "resource-apis"
|
||||
V2DNSExperimentName = "v2dns"
|
||||
V2TenancyExperimentName = "v2tenancy"
|
||||
HCPAllowV2ResourceAPIs = "hcp-v2-resource-apis"
|
||||
)
|
||||
|
|
|
@ -0,0 +1,157 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package discovery
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
// Query is used to request a name-based Service Discovery lookup.
|
||||
type Query struct {
|
||||
QueryType QueryType
|
||||
QueryPayload QueryPayload
|
||||
}
|
||||
|
||||
// QueryType is used to filter service endpoints.
|
||||
// This is needed by the V1 catalog because of the
|
||||
// overlapping lookups through the service endpoint.
|
||||
type QueryType string
|
||||
|
||||
const (
|
||||
QueryTypeConnect QueryType = "CONNECT" // deprecated: use for V1 only
|
||||
QueryTypeIngress QueryType = "INGRESS" // deprecated: use for V1 only
|
||||
QueryTypeNode QueryType = "NODE"
|
||||
QueryTypePreparedQuery QueryType = "PREPARED_QUERY" // deprecated: use for V1 only
|
||||
QueryTypeService QueryType = "SERVICE"
|
||||
QueryTypeVirtual QueryType = "VIRTUAL"
|
||||
QueryTypeWorkload QueryType = "WORKLOAD" // V2-only
|
||||
)
|
||||
|
||||
type Context struct {
|
||||
Token string
|
||||
DefaultPartition string
|
||||
DefaultNamespace string
|
||||
DefaultLocality *structs.Locality
|
||||
}
|
||||
|
||||
type QueryTenancy struct {
|
||||
Partition string
|
||||
Namespace string
|
||||
SamenessGroup string
|
||||
Peer string
|
||||
Datacenter string
|
||||
}
|
||||
|
||||
// QueryPayload represents all information needed by the data backend
|
||||
// to decide which records to include.
|
||||
type QueryPayload struct {
|
||||
Name string
|
||||
PortName string // v1 - this could optionally be "connect" or "ingress"; v2 - this is the service port name
|
||||
Tag string // deprecated: use for V1 only
|
||||
RemoteAddr net.Addr // deprecated: used for prepared queries
|
||||
Tenancy QueryTenancy // tenancy includes any additional labels specified before the domain
|
||||
|
||||
// v2 fields only
|
||||
DisableFailover bool
|
||||
}
|
||||
|
||||
// Result is a generic format of targets that could be returned in a query.
|
||||
// It is the responsibility of the DNS encoder to know what to do with
|
||||
// each Result, based on the query type.
|
||||
type Result struct {
|
||||
Address string // A/AAAA/CNAME records - could be used in the Extra section. CNAME is required to handle hostname addresses in workloads & nodes.
|
||||
Weight uint32 // SRV queries
|
||||
Port uint32 // SRV queries
|
||||
TTL uint32
|
||||
Metadata []string // Used to collect metadata into TXT Records
|
||||
|
||||
// Used in SRV & PTR queries to point at an A/AAAA Record.
|
||||
// In V1, this could be a full-qualified Service or Node name.
|
||||
// In V2, this is generally a fully-qualified Workload name.
|
||||
Target string
|
||||
}
|
||||
|
||||
type LookupType string
|
||||
|
||||
const (
|
||||
LookupTypeService LookupType = "SERVICE"
|
||||
LookupTypeConnect LookupType = "CONNECT"
|
||||
LookupTypeIngress LookupType = "INGRESS"
|
||||
)
|
||||
|
||||
// CatalogDataFetcher is an interface that abstracts data collection
|
||||
// for Discovery queries. It is assumed that the instantiation also
|
||||
// includes any agent configuration that influences catalog queries.
|
||||
type CatalogDataFetcher interface {
|
||||
// LoadConfig is used to hot-reload the data fetcher with new agent config.
|
||||
LoadConfig(config *config.RuntimeConfig)
|
||||
|
||||
// FetchNodes fetches A/AAAA/CNAME
|
||||
FetchNodes(ctx Context, req *QueryPayload) ([]*Result, error)
|
||||
|
||||
// FetchEndpoints fetches records for A/AAAA/CNAME or SRV requests for services
|
||||
FetchEndpoints(ctx Context, req *QueryPayload, lookupType LookupType) ([]*Result, error)
|
||||
|
||||
// FetchVirtualIP fetches A/AAAA records for virtual IPs
|
||||
FetchVirtualIP(ctx Context, req *QueryPayload) (*Result, error)
|
||||
|
||||
// FetchRecordsByIp is used for PTR requests
|
||||
// to look up a service/node from an IP.
|
||||
FetchRecordsByIp(ctx Context, ip net.IP) ([]*Result, error)
|
||||
|
||||
// FetchWorkload fetches a single Result associated with
|
||||
// V2 Workload. V2-only.
|
||||
FetchWorkload(ctx Context, req *QueryPayload) (*Result, error)
|
||||
|
||||
// FetchPreparedQuery evaluates the results of a prepared query.
|
||||
// deprecated in V2
|
||||
FetchPreparedQuery(ctx Context, req *QueryPayload) ([]*Result, error)
|
||||
}
|
||||
|
||||
type QueryProcessor struct {
|
||||
dataFetcher CatalogDataFetcher
|
||||
}
|
||||
|
||||
func NewQueryProcessor(dataFetcher CatalogDataFetcher) *QueryProcessor {
|
||||
return &QueryProcessor{
|
||||
dataFetcher: dataFetcher,
|
||||
}
|
||||
}
|
||||
|
||||
func (p *QueryProcessor) QueryByName(query *Query, ctx Context) ([]*Result, error) {
|
||||
switch query.QueryType {
|
||||
case QueryTypeNode:
|
||||
return p.dataFetcher.FetchNodes(ctx, &query.QueryPayload)
|
||||
case QueryTypeService:
|
||||
return p.dataFetcher.FetchEndpoints(ctx, &query.QueryPayload, LookupTypeService)
|
||||
case QueryTypeConnect:
|
||||
return p.dataFetcher.FetchEndpoints(ctx, &query.QueryPayload, LookupTypeConnect)
|
||||
case QueryTypeIngress:
|
||||
return p.dataFetcher.FetchEndpoints(ctx, &query.QueryPayload, LookupTypeIngress)
|
||||
case QueryTypeVirtual:
|
||||
result, err := p.dataFetcher.FetchVirtualIP(ctx, &query.QueryPayload)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return []*Result{result}, nil
|
||||
case QueryTypeWorkload:
|
||||
result, err := p.dataFetcher.FetchWorkload(ctx, &query.QueryPayload)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return []*Result{result}, nil
|
||||
case QueryTypePreparedQuery:
|
||||
return p.dataFetcher.FetchPreparedQuery(ctx, &query.QueryPayload)
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown query type: %s", query.QueryType)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *QueryProcessor) QueryByIP(ip net.IP, ctx Context) ([]*Result, error) {
|
||||
return p.dataFetcher.FetchRecordsByIp(ctx, ip)
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package discovery
|
||||
|
||||
import (
|
||||
"net"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
)
|
||||
|
||||
const (
|
||||
// TODO (v2-dns): can we move the recursion into the data fetcher?
|
||||
maxRecursionLevelDefault = 3 // This field comes from the V1 DNS server and affects V1 catalog lookups
|
||||
maxRecurseRecords = 5
|
||||
)
|
||||
|
||||
type v1DataFetcherDynamicConfig struct {
|
||||
allowStale bool
|
||||
maxStale time.Duration
|
||||
useCache bool
|
||||
cacheMaxAge time.Duration
|
||||
onlyPassing bool
|
||||
}
|
||||
|
||||
type V1DataFetcher struct {
|
||||
dynamicConfig atomic.Value
|
||||
}
|
||||
|
||||
func NewV1DataFetcher(config *config.RuntimeConfig) *V1DataFetcher {
|
||||
f := &V1DataFetcher{}
|
||||
f.LoadConfig(config)
|
||||
return f
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) LoadConfig(config *config.RuntimeConfig) {
|
||||
dynamicConfig := &v1DataFetcherDynamicConfig{
|
||||
allowStale: config.DNSAllowStale,
|
||||
maxStale: config.DNSMaxStale,
|
||||
useCache: config.DNSUseCache,
|
||||
cacheMaxAge: config.DNSCacheMaxAge,
|
||||
onlyPassing: config.DNSOnlyPassing,
|
||||
}
|
||||
f.dynamicConfig.Store(dynamicConfig)
|
||||
}
|
||||
|
||||
// TODO (v2-dns): Implementation of the V1 data fetcher
|
||||
|
||||
func (f *V1DataFetcher) FetchNodes(ctx Context, req *QueryPayload) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) FetchEndpoints(ctx Context, req *QueryPayload, lookupType LookupType) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) FetchVirtualIP(ctx Context, req *QueryPayload) (*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) FetchRecordsByIp(ctx Context, ip net.IP) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) FetchWorkload(ctx Context, req *QueryPayload) (*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V1DataFetcher) FetchPreparedQuery(ctx Context, req *QueryPayload) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package discovery
|
||||
|
||||
import (
|
||||
"net"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
)
|
||||
|
||||
type v2DataFetcherDynamicConfig struct {
|
||||
onlyPassing bool
|
||||
}
|
||||
|
||||
type V2DataFetcher struct {
|
||||
dynamicConfig atomic.Value
|
||||
}
|
||||
|
||||
func NewV2DataFetcher(config *config.RuntimeConfig) *V2DataFetcher {
|
||||
f := &V2DataFetcher{}
|
||||
f.LoadConfig(config)
|
||||
return f
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) LoadConfig(config *config.RuntimeConfig) {
|
||||
dynamicConfig := &v2DataFetcherDynamicConfig{
|
||||
onlyPassing: config.DNSOnlyPassing,
|
||||
}
|
||||
f.dynamicConfig.Store(dynamicConfig)
|
||||
}
|
||||
|
||||
// TODO (v2-dns): Implementation of the V2 data fetcher
|
||||
|
||||
func (f *V2DataFetcher) FetchNodes(ctx Context, req *QueryPayload) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) FetchEndpoints(ctx Context, req *QueryPayload, lookupType LookupType) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) FetchVirtualIP(ctx Context, req *QueryPayload) (*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) FetchRecordsByIp(ctx Context, ip net.IP) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) FetchWorkload(ctx Context, req *QueryPayload) (*Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (f *V2DataFetcher) FetchPreparedQuery(ctx Context, req *QueryPayload) ([]*Result, error) {
|
||||
return nil, nil
|
||||
}
|
28
agent/dns.go
28
agent/dns.go
|
@ -25,9 +25,9 @@ import (
|
|||
"github.com/hashicorp/consul/acl"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
agentdns "github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
libdns "github.com/hashicorp/consul/internal/dnsutil"
|
||||
"github.com/hashicorp/consul/ipaddr"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/logging"
|
||||
|
@ -90,7 +90,7 @@ type dnsConfig struct {
|
|||
NodeName string
|
||||
NodeTTL time.Duration
|
||||
OnlyPassing bool
|
||||
RecursorStrategy agentdns.RecursorStrategy
|
||||
RecursorStrategy structs.RecursorStrategy
|
||||
RecursorTimeout time.Duration
|
||||
Recursors []string
|
||||
SegmentName string
|
||||
|
@ -264,6 +264,28 @@ func (d *DNSServer) ListenAndServe(network, addr string, notif func()) error {
|
|||
return d.Server.ListenAndServe()
|
||||
}
|
||||
|
||||
func (d *DNSServer) Shutdown() {
|
||||
if d.Server != nil {
|
||||
d.logger.Info("Stopping server",
|
||||
"protocol", "DNS",
|
||||
"address", d.Server.Addr,
|
||||
"network", d.Server.Net,
|
||||
)
|
||||
err := d.Server.Shutdown()
|
||||
if err != nil {
|
||||
d.logger.Error("Error stopping DNS server", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// GetAddr is a function to return the server address if is not nil.
|
||||
func (d *DNSServer) GetAddr() string {
|
||||
if d.Server != nil {
|
||||
return d.Server.Addr
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// toggleRecursorHandlerFromConfig enables or disables the recursor handler based on config idempotently
|
||||
func (d *DNSServer) toggleRecursorHandlerFromConfig(cfg *dnsConfig) {
|
||||
shouldEnable := len(cfg.Recursors) > 0
|
||||
|
@ -613,7 +635,7 @@ func (d *DNSServer) nameservers(questionName string, cfg *dnsConfig, maxRecursio
|
|||
for _, o := range out.Nodes {
|
||||
name, dc := o.Node.Node, o.Node.Datacenter
|
||||
|
||||
if agentdns.InvalidNameRe.MatchString(name) {
|
||||
if libdns.InvalidNameRe.MatchString(name) {
|
||||
d.logger.Warn("Skipping invalid node for NS records", "node", name)
|
||||
continue
|
||||
}
|
||||
|
|
|
@ -0,0 +1,148 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/armon/go-radix"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/miekg/dns"
|
||||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/discovery"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
// TODO (v2-dns): metrics
|
||||
|
||||
// RouterDynamicConfig is the dynamic configuration that can be hot-reloaded
|
||||
type RouterDynamicConfig struct {
|
||||
ARecordLimit int
|
||||
DisableCompression bool
|
||||
EnableDefaultFailover bool // TODO (v2-dns): plumbing required for this new V2 setting. This is the agent configured default
|
||||
EnableTruncate bool
|
||||
NodeMetaTXT bool
|
||||
NodeTTL time.Duration
|
||||
Recursors []string
|
||||
RecursorTimeout time.Duration
|
||||
RecursorStrategy structs.RecursorStrategy
|
||||
SOAConfig SOAConfig
|
||||
// TTLRadix sets service TTLs by prefix, eg: "database-*"
|
||||
TTLRadix *radix.Tree
|
||||
// TTLStrict sets TTLs to service by full name match. It Has higher priority than TTLRadix
|
||||
TTLStrict map[string]time.Duration
|
||||
UDPAnswerLimit int
|
||||
}
|
||||
|
||||
type SOAConfig struct {
|
||||
Refresh uint32 // 3600 by default
|
||||
Retry uint32 // 600
|
||||
Expire uint32 // 86400
|
||||
Minttl uint32 // 0
|
||||
}
|
||||
|
||||
// DiscoveryQueryProcessor is an interface that can be used by any consumer requesting Service Discovery results.
|
||||
// This could be attached to a gRPC endpoint in the future in addition to DNS.
|
||||
// Making this an interface means testing the router with a mock is trivial.
|
||||
type DiscoveryQueryProcessor interface {
|
||||
QueryByName(*discovery.Query, discovery.Context) ([]*discovery.Result, error)
|
||||
QueryByIP(net.IP, discovery.Context) ([]*discovery.Result, error)
|
||||
}
|
||||
|
||||
// Router replaces miekg/dns.ServeMux with a simpler router that only checks for the 2-3 valid domains
|
||||
// that Consul supports and forwards to a single DiscoveryQueryProcessor handler. If there is no match, it will recurse.
|
||||
type Router struct {
|
||||
processor DiscoveryQueryProcessor
|
||||
domain string
|
||||
altDomain string
|
||||
logger hclog.Logger
|
||||
|
||||
tokenFunc func() string
|
||||
|
||||
defaultNamespace string
|
||||
defaultPartition string
|
||||
|
||||
// TODO (v2-dns): default locality for request context?
|
||||
|
||||
// dynamicConfig stores the config as an atomic value (for hot-reloading).
|
||||
// It is always of type *RouterDynamicConfig
|
||||
dynamicConfig atomic.Value
|
||||
}
|
||||
|
||||
var _ = dns.Handler(&Router{})
|
||||
|
||||
func NewRouter(cfg Config) (*Router, error) {
|
||||
router := &Router{
|
||||
// TODO (v2-dns): implement stub
|
||||
}
|
||||
|
||||
if err := router.ReloadConfig(cfg.AgentConfig); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return router, nil
|
||||
}
|
||||
|
||||
// HandleRequest is used to process and individual DNS request. It returns a message in success or fail cases.
|
||||
func (r Router) HandleRequest(req *dns.Msg, reqCtx discovery.Context, remoteAddress net.Addr) *dns.Msg {
|
||||
cfg := r.dynamicConfig.Load().(*RouterDynamicConfig)
|
||||
|
||||
// TODO (v2-dns): implement HandleRequest. This is just temporary
|
||||
return createServerFailureResponse(req, cfg, false)
|
||||
|
||||
// Parse fields of the message
|
||||
|
||||
// Route the request to the appropriate destination
|
||||
// 1. r.processor.QueryByName
|
||||
// 2. r.processor.QueryByIP
|
||||
// 3. recurse
|
||||
|
||||
// Serialize the output
|
||||
|
||||
}
|
||||
|
||||
// ServeDNS implements the miekg/dns.Handler interface
|
||||
func (r Router) ServeDNS(w dns.ResponseWriter, req *dns.Msg) {
|
||||
reqCtx := r.defaultAgentDNSRequestContext()
|
||||
out := r.HandleRequest(req, reqCtx, w.RemoteAddr())
|
||||
w.WriteMsg(out)
|
||||
}
|
||||
|
||||
// GetDynamicRouterConfig takes global config and creates the config used by DNS server
|
||||
func GetDynamicRouterConfig(conf *config.RuntimeConfig) (*RouterDynamicConfig, error) {
|
||||
cfg := &RouterDynamicConfig{
|
||||
// TODO (v2-dns)
|
||||
}
|
||||
|
||||
return cfg, nil
|
||||
}
|
||||
|
||||
// ReloadConfig hot-reloads the router config with new parameters
|
||||
func (r Router) ReloadConfig(newCfg *config.RuntimeConfig) error {
|
||||
cfg, err := GetDynamicRouterConfig(newCfg)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error loading DNS config: %w", err)
|
||||
}
|
||||
r.dynamicConfig.Store(cfg)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r Router) defaultAgentDNSRequestContext() discovery.Context {
|
||||
return discovery.Context{
|
||||
// TODO (v2-dns): implement stub
|
||||
}
|
||||
}
|
||||
|
||||
func createServerFailureResponse(req *dns.Msg, cfg *RouterDynamicConfig, recursionAvailable bool) *dns.Msg {
|
||||
// Return a SERVFAIL message
|
||||
m := &dns.Msg{}
|
||||
m.SetReply(req)
|
||||
m.Compress = !cfg.DisableCompression
|
||||
m.SetRcode(req, dns.RcodeServerFailure)
|
||||
m.RecursionAvailable = recursionAvailable
|
||||
return m
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
|
||||
// TODO (v2-dns)
|
|
@ -0,0 +1,86 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/miekg/dns"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/logging"
|
||||
)
|
||||
|
||||
// Server is used to expose service discovery queries using a DNS interface.
|
||||
// It implements the agent.dnsServer interface.
|
||||
type Server struct {
|
||||
*dns.Server // Used for setting up listeners
|
||||
Router *Router // Used to routes and parse DNS requests
|
||||
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// Config represent all the DNS configuration required to construct a DNS server.
|
||||
type Config struct {
|
||||
AgentConfig *config.RuntimeConfig
|
||||
EntMeta *acl.EnterpriseMeta
|
||||
Logger hclog.Logger
|
||||
Processor DiscoveryQueryProcessor
|
||||
TokenFunc func() string
|
||||
}
|
||||
|
||||
func NewServer(config Config) (*Server, error) {
|
||||
router, err := NewRouter(config)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error creating DNS router: %w", err)
|
||||
}
|
||||
|
||||
srv := &Server{
|
||||
Router: router,
|
||||
logger: config.Logger.Named(logging.DNS),
|
||||
}
|
||||
return srv, nil
|
||||
}
|
||||
|
||||
func (d *Server) ListenAndServe(network, addr string, notif func()) error {
|
||||
d.Server = &dns.Server{
|
||||
Addr: addr,
|
||||
Net: network,
|
||||
Handler: d.Router,
|
||||
NotifyStartedFunc: notif,
|
||||
}
|
||||
if network == "udp" {
|
||||
d.UDPSize = 65535
|
||||
}
|
||||
return d.Server.ListenAndServe()
|
||||
}
|
||||
|
||||
// ReloadConfig hot-reloads the server config with new parameters under config.RuntimeConfig.DNS*
|
||||
func (d *Server) ReloadConfig(newCfg *config.RuntimeConfig) error {
|
||||
return d.Router.ReloadConfig(newCfg)
|
||||
}
|
||||
|
||||
func (d *Server) Shutdown() {
|
||||
if d.Server != nil {
|
||||
d.logger.Info("Stopping server",
|
||||
"protocol", "DNS",
|
||||
"address", d.Server.Addr,
|
||||
"network", d.Server.Net,
|
||||
)
|
||||
err := d.Server.Shutdown()
|
||||
if err != nil {
|
||||
d.logger.Error("Error stopping DNS server", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// GetAddr is a function to return the server address if is not nil.
|
||||
func (d *Server) GetAddr() string {
|
||||
if d.Server != nil {
|
||||
return d.Server.Addr
|
||||
}
|
||||
return ""
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
|
||||
// TODO (v2-dns): add at least one test to listen and serve with a dummy router.
|
|
@ -31,8 +31,8 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/consul"
|
||||
agentdns "github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
libdns "github.com/hashicorp/consul/internal/dnsutil"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
"github.com/hashicorp/consul/testrpc"
|
||||
|
@ -3550,7 +3550,7 @@ func TestDNSInvalidRegex(t *testing.T) {
|
|||
}
|
||||
for _, test := range tests {
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
if got, want := agentdns.InvalidNameRe.MatchString(test.in), test.invalid; got != want {
|
||||
if got, want := libdns.InvalidNameRe.MatchString(test.in), test.invalid; got != want {
|
||||
t.Fatalf("Expected %v to return %v", test.in, want)
|
||||
}
|
||||
})
|
||||
|
@ -3558,16 +3558,13 @@ func TestDNSInvalidRegex(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestDNS_ConfigReload(t *testing.T) {
|
||||
func TestDNS_V1ConfigReload(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("too slow for testing.Short")
|
||||
}
|
||||
|
||||
t.Parallel()
|
||||
|
||||
for name, experimentsHCL := range versionHCL {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
|
||||
a := NewTestAgent(t, `
|
||||
recursors = ["8.8.8.8:53"]
|
||||
dns_config = {
|
||||
|
@ -3592,14 +3589,17 @@ func TestDNS_ConfigReload(t *testing.T) {
|
|||
min_ttl = 4
|
||||
}
|
||||
}
|
||||
`+experimentsHCL)
|
||||
`)
|
||||
defer a.Shutdown()
|
||||
testrpc.WaitForLeader(t, a.RPC, "dc1")
|
||||
|
||||
for _, s := range a.dnsServers {
|
||||
cfg := s.config.Load().(*dnsConfig)
|
||||
server, ok := s.(*DNSServer)
|
||||
require.True(t, ok)
|
||||
|
||||
cfg := server.config.Load().(*dnsConfig)
|
||||
require.Equal(t, []string{"8.8.8.8:53"}, cfg.Recursors)
|
||||
require.Equal(t, agentdns.RecursorStrategy("sequential"), cfg.RecursorStrategy)
|
||||
require.Equal(t, structs.RecursorStrategy("sequential"), cfg.RecursorStrategy)
|
||||
require.False(t, cfg.AllowStale)
|
||||
require.Equal(t, 20*time.Second, cfg.MaxStale)
|
||||
require.Equal(t, 10*time.Second, cfg.NodeTTL)
|
||||
|
@ -3644,9 +3644,12 @@ func TestDNS_ConfigReload(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
|
||||
for _, s := range a.dnsServers {
|
||||
cfg := s.config.Load().(*dnsConfig)
|
||||
server, ok := s.(*DNSServer)
|
||||
require.True(t, ok)
|
||||
|
||||
cfg := server.config.Load().(*dnsConfig)
|
||||
require.Equal(t, []string{"1.1.1.1:53"}, cfg.Recursors)
|
||||
require.Equal(t, agentdns.RecursorStrategy("random"), cfg.RecursorStrategy)
|
||||
require.Equal(t, structs.RecursorStrategy("random"), cfg.RecursorStrategy)
|
||||
require.True(t, cfg.AllowStale)
|
||||
require.Equal(t, 21*time.Second, cfg.MaxStale)
|
||||
require.Equal(t, 11*time.Second, cfg.NodeTTL)
|
||||
|
@ -3669,10 +3672,11 @@ func TestDNS_ConfigReload(t *testing.T) {
|
|||
require.Equal(t, uint32(30), cfg.SOAConfig.Expire)
|
||||
require.Equal(t, uint32(40), cfg.SOAConfig.Minttl)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// TODO (v2-dns) add a test for checking the V2 DNS Server reloads the config
|
||||
|
||||
func TestDNS_ReloadConfig_DuringQuery(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("too slow for testing.Short")
|
||||
|
|
|
@ -1,19 +1,9 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
package structs
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"regexp"
|
||||
)
|
||||
|
||||
// MaxLabelLength is the maximum length for a name that can be used in DNS.
|
||||
const MaxLabelLength = 63
|
||||
|
||||
// InvalidNameRe is a regex that matches characters which can not be included in
|
||||
// a DNS name.
|
||||
var InvalidNameRe = regexp.MustCompile(`[^A-Za-z0-9\\-]+`)
|
||||
import "math/rand"
|
||||
|
||||
type RecursorStrategy string
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
package structs
|
||||
|
||||
import (
|
||||
"testing"
|
|
@ -74,7 +74,7 @@ type TestAgent struct {
|
|||
|
||||
// dns is a reference to the first started DNS endpoint.
|
||||
// It is valid after Start().
|
||||
dns *DNSServer
|
||||
dns dnsServer
|
||||
|
||||
// srv is an HTTPHandlers that may be used to test http endpoints.
|
||||
srv *HTTPHandlers
|
||||
|
@ -362,7 +362,7 @@ func (a *TestAgent) DNSAddr() string {
|
|||
if a.dns == nil {
|
||||
return ""
|
||||
}
|
||||
return a.dns.Addr
|
||||
return a.dns.GetAddr()
|
||||
}
|
||||
|
||||
func (a *TestAgent) HTTPAddr() string {
|
||||
|
|
|
@ -1,13 +1,20 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns
|
||||
package dnsutil
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"regexp"
|
||||
)
|
||||
|
||||
// MaxLabelLength is the maximum length for a name that can be used in DNS.
|
||||
const MaxLabelLength = 63
|
||||
|
||||
// InvalidNameRe is a regex that matches characters which can not be included in
|
||||
// a DNS name.
|
||||
var InvalidNameRe = regexp.MustCompile(`[^A-Za-z0-9\\-]+`)
|
||||
|
||||
// matches valid DNS labels according to RFC 1123, should be at most 63
|
||||
// characters according to the RFC
|
||||
var validLabel = regexp.MustCompile(`^[a-zA-Z0-9]([a-zA-Z0-9\-]{0,61}[a-zA-Z0-9])?$`)
|
|
@ -1,12 +1,11 @@
|
|||
// Copyright (c) HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: BUSL-1.1
|
||||
|
||||
package dns_test
|
||||
package dnsutil
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
|
@ -29,23 +28,23 @@ func TestValidLabel(t *testing.T) {
|
|||
|
||||
t.Run("*", func(t *testing.T) {
|
||||
t.Run("IsValidLabel", func(t *testing.T) {
|
||||
require.False(t, dns.IsValidLabel("*"))
|
||||
require.False(t, IsValidLabel("*"))
|
||||
})
|
||||
t.Run("ValidateLabel", func(t *testing.T) {
|
||||
require.Error(t, dns.ValidateLabel("*"))
|
||||
require.Error(t, ValidateLabel("*"))
|
||||
})
|
||||
})
|
||||
|
||||
for name, expect := range cases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
t.Run("IsValidDNSLabel", func(t *testing.T) {
|
||||
require.Equal(t, expect, dns.IsValidLabel(name))
|
||||
require.Equal(t, expect, IsValidLabel(name))
|
||||
})
|
||||
t.Run("ValidateLabel", func(t *testing.T) {
|
||||
if expect {
|
||||
require.NoError(t, dns.ValidateLabel(name))
|
||||
require.NoError(t, ValidateLabel(name))
|
||||
} else {
|
||||
require.Error(t, dns.ValidateLabel(name))
|
||||
require.Error(t, ValidateLabel(name))
|
||||
}
|
||||
})
|
||||
})
|
|
@ -9,7 +9,7 @@ import (
|
|||
|
||||
mapset "github.com/deckarep/golang-set/v2"
|
||||
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/internal/dnsutil"
|
||||
"github.com/hashicorp/consul/proto-public/pbresource"
|
||||
)
|
||||
|
||||
|
@ -27,7 +27,7 @@ const FinalizerKey = "finalizers"
|
|||
// ValidateName returns an error a name is not a valid resource name.
|
||||
// The error will contain reference to what constitutes a valid resource name.
|
||||
func ValidateName(name string) error {
|
||||
if !dns.IsValidLabel(name) || strings.ToLower(name) != name || len(name) > MaxNameLength {
|
||||
if !dnsutil.IsValidLabel(name) || strings.ToLower(name) != name || len(name) > MaxNameLength {
|
||||
return fmt.Errorf("a resource name must consist of lower case alphanumeric characters or '-', must start and end with an alphanumeric character and be less than %d characters, got: %q", MaxNameLength+1, name)
|
||||
}
|
||||
return nil
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
|
||||
"google.golang.org/protobuf/types/known/anypb"
|
||||
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/internal/dnsutil"
|
||||
"github.com/hashicorp/consul/internal/resource"
|
||||
"github.com/hashicorp/consul/proto-public/pbresource"
|
||||
pbtenancy "github.com/hashicorp/consul/proto-public/pbtenancy/v2beta1"
|
||||
|
@ -55,7 +55,7 @@ func ValidateNamespace(res *pbresource.Resource) error {
|
|||
return errInvalidName
|
||||
}
|
||||
|
||||
if !dns.IsValidLabel(res.Id.Name) {
|
||||
if !dnsutil.IsValidLabel(res.Id.Name) {
|
||||
return fmt.Errorf("namespace name %q is not a valid DNS hostname", res.Id.Name)
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue