Groundwork for exposing when queries are filtered by ACLs (#11569)

This commit is contained in:
Dan Upton 2021-12-03 17:11:26 +00:00 committed by GitHub
parent da929237b5
commit 0efe478044
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 275 additions and 74 deletions

3
.changelog/11569.txt Normal file
View File

@ -0,0 +1,3 @@
```release-note:enhancement
api: responses that contain only a partial subset of results, due to filtering by ACL policies, may now include an `X-Consul-Results-Filtered-By-ACLs` header
```

View File

@ -1807,6 +1807,7 @@ func filterACLWithAuthorizer(logger hclog.Logger, authorizer acl.Authorizer, sub
filtered := filt.filterServiceTopology(v.ServiceTopology) filtered := filt.filterServiceTopology(v.ServiceTopology)
if filtered { if filtered {
v.FilteredByACLs = true v.FilteredByACLs = true
v.QueryMeta.ResultsFilteredByACLs = true
} }
case *structs.DatacenterIndexedCheckServiceNodes: case *structs.DatacenterIndexedCheckServiceNodes:

View File

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

View File

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

View File

@ -1728,6 +1728,7 @@ func TestInternal_ServiceTopology(t *testing.T) {
var out structs.IndexedServiceTopology var out structs.IndexedServiceTopology
require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.False(r, out.FilteredByACLs) require.False(r, out.FilteredByACLs)
require.False(r, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(r, "http", out.ServiceTopology.MetricsProtocol) require.Equal(r, "http", out.ServiceTopology.MetricsProtocol)
// foo/api, foo/api-proxy // foo/api, foo/api-proxy
@ -1767,6 +1768,7 @@ func TestInternal_ServiceTopology(t *testing.T) {
var out structs.IndexedServiceTopology var out structs.IndexedServiceTopology
require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.False(r, out.FilteredByACLs) require.False(r, out.FilteredByACLs)
require.False(r, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(r, "http", out.ServiceTopology.MetricsProtocol) require.Equal(r, "http", out.ServiceTopology.MetricsProtocol)
// edge/ingress // edge/ingress
@ -1822,6 +1824,7 @@ func TestInternal_ServiceTopology(t *testing.T) {
var out structs.IndexedServiceTopology var out structs.IndexedServiceTopology
require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.False(r, out.FilteredByACLs) require.False(r, out.FilteredByACLs)
require.False(r, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(r, "http", out.ServiceTopology.MetricsProtocol) require.Equal(r, "http", out.ServiceTopology.MetricsProtocol)
// foo/api, foo/api-proxy // foo/api, foo/api-proxy
@ -1875,6 +1878,7 @@ func TestInternal_ServiceTopology(t *testing.T) {
var out structs.IndexedServiceTopology var out structs.IndexedServiceTopology
require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.False(r, out.FilteredByACLs) require.False(r, out.FilteredByACLs)
require.False(r, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(r, "http", out.ServiceTopology.MetricsProtocol) require.Equal(r, "http", out.ServiceTopology.MetricsProtocol)
require.Len(r, out.ServiceTopology.Upstreams, 0) require.Len(r, out.ServiceTopology.Upstreams, 0)
@ -1931,6 +1935,7 @@ func TestInternal_ServiceTopology_RoutingConfig(t *testing.T) {
var out structs.IndexedServiceTopology var out structs.IndexedServiceTopology
require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(r, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.False(r, out.FilteredByACLs) require.False(r, out.FilteredByACLs)
require.False(r, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(r, "http", out.ServiceTopology.MetricsProtocol) require.Equal(r, "http", out.ServiceTopology.MetricsProtocol)
require.Empty(r, out.ServiceTopology.Downstreams) require.Empty(r, out.ServiceTopology.Downstreams)
@ -2010,6 +2015,7 @@ service "web" { policy = "read" }
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(t, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.True(t, out.FilteredByACLs) require.True(t, out.FilteredByACLs)
require.True(t, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(t, "http", out.ServiceTopology.MetricsProtocol) require.Equal(t, "http", out.ServiceTopology.MetricsProtocol)
// The web-proxy upstream gets filtered out from both bar and baz // The web-proxy upstream gets filtered out from both bar and baz
@ -2030,6 +2036,7 @@ service "web" { policy = "read" }
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out)) require.NoError(t, msgpackrpc.CallWithCodec(codec, "Internal.ServiceTopology", &args, &out))
require.True(t, out.FilteredByACLs) require.True(t, out.FilteredByACLs)
require.True(t, out.QueryMeta.ResultsFilteredByACLs)
require.Equal(t, "http", out.ServiceTopology.MetricsProtocol) require.Equal(t, "http", out.ServiceTopology.MetricsProtocol)
// The redis upstream gets filtered out but the api and proxy downstream are returned // The redis upstream gets filtered out but the api and proxy downstream are returned

View File

@ -299,7 +299,7 @@ func (p *PreparedQuery) Explain(args *structs.PreparedQueryExecuteRequest,
defer metrics.MeasureSince([]string{"prepared-query", "explain"}, time.Now()) defer metrics.MeasureSince([]string{"prepared-query", "explain"}, time.Now())
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta) p.srv.setQueryMeta(&reply.QueryMeta, args.Token)
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.consistentRead(); err != nil {
return err return err
@ -346,7 +346,6 @@ func (p *PreparedQuery) Execute(args *structs.PreparedQueryExecuteRequest,
defer metrics.MeasureSince([]string{"prepared-query", "execute"}, time.Now()) defer metrics.MeasureSince([]string{"prepared-query", "execute"}, time.Now())
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta)
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.consistentRead(); err != nil {
return err return err
@ -383,6 +382,9 @@ func (p *PreparedQuery) Execute(args *structs.PreparedQueryExecuteRequest,
// might not be worth the code complexity and behavior differences, // might not be worth the code complexity and behavior differences,
// though, since this is essentially a misconfiguration. // though, since this is essentially a misconfiguration.
// We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta, token)
// Shuffle the results in case coordinates are not available if they // Shuffle the results in case coordinates are not available if they
// requested an RTT sort. // requested an RTT sort.
reply.Nodes.Shuffle() reply.Nodes.Shuffle()
@ -481,7 +483,6 @@ func (p *PreparedQuery) ExecuteRemote(args *structs.PreparedQueryExecuteRemoteRe
defer metrics.MeasureSince([]string{"prepared-query", "execute_remote"}, time.Now()) defer metrics.MeasureSince([]string{"prepared-query", "execute_remote"}, time.Now())
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta)
if args.RequireConsistent { if args.RequireConsistent {
if err := p.srv.consistentRead(); err != nil { if err := p.srv.consistentRead(); err != nil {
return err return err
@ -503,6 +504,9 @@ func (p *PreparedQuery) ExecuteRemote(args *structs.PreparedQueryExecuteRemoteRe
return err return err
} }
// We have to do this ourselves since we are not doing a blocking RPC.
p.srv.setQueryMeta(&reply.QueryMeta, token)
// We don't bother trying to do an RTT sort here since we are by // We don't bother trying to do an RTT sort here since we are by
// definition in another DC. We just shuffle to make sure that we // definition in another DC. We just shuffle to make sure that we
// balance the load across the results. // balance the load across the results.

View File

@ -938,8 +938,6 @@ func (s *Server) blockingQuery(queryOpts structs.QueryOptionsCompat, queryMeta s
RUN_QUERY: RUN_QUERY:
// Setup blocking loop // Setup blocking loop
// Update the query metadata.
s.setQueryMeta(queryMeta)
// Validate // Validate
// If the read must be consistent we verify that we are still the leader. // If the read must be consistent we verify that we are still the leader.
@ -968,6 +966,10 @@ RUN_QUERY:
// Execute the queryFn // Execute the queryFn
err := fn(ws, state) err := fn(ws, state)
// Update the query metadata.
s.setQueryMeta(queryMeta, queryOpts.GetToken())
// Note we check queryOpts.MinQueryIndex is greater than zero to determine if // Note we check queryOpts.MinQueryIndex is greater than zero to determine if
// blocking was requested by client, NOT meta.Index since the state function // blocking was requested by client, NOT meta.Index since the state function
// might return zero if something is not initialized and care wasn't taken to // might return zero if something is not initialized and care wasn't taken to
@ -1001,7 +1003,9 @@ RUN_QUERY:
} }
// setQueryMeta is used to populate the QueryMeta data for an RPC call // setQueryMeta is used to populate the QueryMeta data for an RPC call
func (s *Server) setQueryMeta(m structs.QueryMetaCompat) { //
// Note: This method must be called *after* filtering query results with ACLs.
func (s *Server) setQueryMeta(m structs.QueryMetaCompat, token string) {
if s.IsLeader() { if s.IsLeader() {
m.SetLastContact(0) m.SetLastContact(0)
m.SetKnownLeader(true) m.SetKnownLeader(true)
@ -1009,6 +1013,7 @@ func (s *Server) setQueryMeta(m structs.QueryMetaCompat) {
m.SetLastContact(time.Since(s.raft.LastContact())) m.SetLastContact(time.Since(s.raft.LastContact()))
m.SetKnownLeader(s.raft.Leader() != "") m.SetKnownLeader(s.raft.Leader() != "")
} }
maskResultsFilteredByACLs(token, m)
} }
// consistentRead is used to ensure we do not perform a stale // consistentRead is used to ensure we do not perform a stale
@ -1043,3 +1048,31 @@ func (s *Server) consistentRead() error {
return structs.ErrNotReadyForConsistentReads return structs.ErrNotReadyForConsistentReads
} }
// maskResultsFilteredByACLs blanks out the ResultsFilteredByACLs flag if the
// request is unauthenticated, to limit information leaking.
//
// Endpoints that support bexpr filtering could be used in combination with
// this flag/header to discover the existence of resources to which the user
// does not have access, therefore we only expose it when the user presents
// a valid ACL token. This doesn't completely remove the risk (by nature the
// purpose of this flag is to let the user know there are resources they can
// not access) but it prevents completely unauthenticated users from doing so.
//
// Notes:
//
// * The definition of "unauthenticated" here is incomplete, as it doesn't
// account for the fact that operators can modify the anonymous token with
// custom policies, or set namespace default policies. As these scenarios
// are less common and this flag is a best-effort UX improvement, we think
// the trade-off for reduced complexity is acceptable.
//
// * This method assumes that the given token has already been validated (and
// will only check whether it is blank or not). It's a safe assumption because
// ResultsFilteredByACLs is only set to try when applying the already-resolved
// token's policies.
func maskResultsFilteredByACLs(token string, meta structs.QueryMetaCompat) {
if token == "" {
meta.SetResultsFilteredByACLs(false)
}
}

View File

@ -36,6 +36,7 @@ import (
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
tokenStore "github.com/hashicorp/consul/agent/token" tokenStore "github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/proto/pbsubscribe" "github.com/hashicorp/consul/proto/pbsubscribe"
"github.com/hashicorp/consul/sdk/testutil" "github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/sdk/testutil/retry" "github.com/hashicorp/consul/sdk/testutil/retry"
@ -369,6 +370,39 @@ func TestRPC_blockingQuery(t *testing.T) {
t.Fatalf("bad: %d", calls) t.Fatalf("bad: %d", calls)
} }
} }
t.Run("ResultsFilteredByACLs is reset for unauthenticated calls", func(t *testing.T) {
opts := structs.QueryOptions{
Token: "",
}
var meta structs.QueryMeta
fn := func(_ memdb.WatchSet, _ *state.Store) error {
meta.ResultsFilteredByACLs = true
return nil
}
err := s.blockingQuery(&opts, &meta, fn)
require.NoError(err)
require.False(meta.ResultsFilteredByACLs, "ResultsFilteredByACLs should be reset for unauthenticated calls")
})
t.Run("ResultsFilteredByACLs is honored for authenticated calls", func(t *testing.T) {
token, err := lib.GenerateUUID(nil)
require.NoError(err)
opts := structs.QueryOptions{
Token: token,
}
var meta structs.QueryMeta
fn := func(_ memdb.WatchSet, _ *state.Store) error {
meta.ResultsFilteredByACLs = true
return nil
}
err = s.blockingQuery(&opts, &meta, fn)
require.NoError(err)
require.True(meta.ResultsFilteredByACLs, "ResultsFilteredByACLs should be honored for authenticated calls")
})
} }
func TestRPC_ReadyForConsistentReads(t *testing.T) { func TestRPC_ReadyForConsistentReads(t *testing.T) {

View File

@ -77,7 +77,7 @@ func (s *Server) dispatchSnapshotRequest(args *structs.SnapshotRequest, in io.Re
// Set the metadata here before we do anything; this should always be // Set the metadata here before we do anything; this should always be
// pessimistic if we get more data while the snapshot is being taken. // pessimistic if we get more data while the snapshot is being taken.
s.setQueryMeta(&reply.QueryMeta) s.setQueryMeta(&reply.QueryMeta, args.Token)
// Take the snapshot and capture the index. // Take the snapshot and capture the index.
snap, err := snapshot.New(s.logger, s.raft) snap, err := snapshot.New(s.logger, s.raft)

View File

@ -183,7 +183,6 @@ func (t *Txn) Read(args *structs.TxnReadRequest, reply *structs.TxnReadResponse)
defer metrics.MeasureSince([]string{"txn", "read"}, time.Now()) defer metrics.MeasureSince([]string{"txn", "read"}, time.Now())
// We have to do this ourselves since we are not doing a blocking RPC. // We have to do this ourselves since we are not doing a blocking RPC.
t.srv.setQueryMeta(&reply.QueryMeta)
if args.RequireConsistent { if args.RequireConsistent {
if err := t.srv.consistentRead(); err != nil { if err := t.srv.consistentRead(); err != nil {
return err return err
@ -204,5 +203,9 @@ func (t *Txn) Read(args *structs.TxnReadRequest, reply *structs.TxnReadResponse)
state := t.srv.fsm.State() state := t.srv.fsm.State()
reply.Results, reply.Errors = state.TxnRO(args.Ops) reply.Results, reply.Errors = state.TxnRO(args.Ops)
reply.Results = FilterTxnResults(authz, reply.Results) reply.Results = FilterTxnResults(authz, reply.Results)
// We have to do this ourselves since we are not doing a blocking RPC.
t.srv.setQueryMeta(&reply.QueryMeta, args.Token)
return nil return nil
} }

View File

@ -941,11 +941,7 @@ func TestTxn_Read_ACLDeny(t *testing.T) {
} }
// Verify the transaction's return value. // Verify the transaction's return value.
expected := structs.TxnReadResponse{ var expected structs.TxnReadResponse
QueryMeta: structs.QueryMeta{
KnownLeader: true,
},
}
for i, op := range arg.Ops { for i, op := range arg.Ops {
switch { switch {
case op.KV != nil: case op.KV != nil:

View File

@ -734,6 +734,7 @@ func setMeta(resp http.ResponseWriter, m structs.QueryMetaCompat) {
setKnownLeader(resp, m.GetKnownLeader()) setKnownLeader(resp, m.GetKnownLeader())
setConsistency(resp, m.GetConsistencyLevel()) setConsistency(resp, m.GetConsistencyLevel())
setQueryBackend(resp, m.GetBackend()) setQueryBackend(resp, m.GetBackend())
setResultsFilteredByACLs(resp, m.GetResultsFilteredByACLs())
} }
func setQueryBackend(resp http.ResponseWriter, backend structs.QueryBackend) { func setQueryBackend(resp http.ResponseWriter, backend structs.QueryBackend) {
@ -757,6 +758,16 @@ func setCacheMeta(resp http.ResponseWriter, m *cache.ResultMeta) {
} }
} }
// setResultsFilteredByACLs sets an HTTP response header to indicate that the
// query results were filtered by enforcing ACLs. If the given filtered value
// is false the header will be omitted, as its ambiguous whether the results
// were not filtered or whether the endpoint doesn't yet support this header.
func setResultsFilteredByACLs(resp http.ResponseWriter, filtered bool) {
if filtered {
resp.Header().Set("X-Consul-Results-Filtered-By-ACLs", "true")
}
}
// setHeaders is used to set canonical response header fields // setHeaders is used to set canonical response header fields
func setHeaders(resp http.ResponseWriter, headers map[string]string) { func setHeaders(resp http.ResponseWriter, headers map[string]string) {
for field, value := range headers { for field, value := range headers {

View File

@ -264,6 +264,22 @@ func TestSetKnownLeader(t *testing.T) {
} }
} }
func TestSetFilteredByACLs(t *testing.T) {
t.Parallel()
resp := httptest.NewRecorder()
setResultsFilteredByACLs(resp, true)
header := resp.Header().Get("X-Consul-Results-Filtered-By-ACLs")
if header != "true" {
t.Fatalf("Bad: %v", header)
}
resp = httptest.NewRecorder()
setResultsFilteredByACLs(resp, false)
header = resp.Header().Get("X-Consul-Results-Filtered-By-ACLs")
if header != "" {
t.Fatalf("Bad: %v", header)
}
}
func TestSetLastContact(t *testing.T) { func TestSetLastContact(t *testing.T) {
t.Parallel() t.Parallel()
tests := []struct { tests := []struct {
@ -291,23 +307,24 @@ func TestSetLastContact(t *testing.T) {
func TestSetMeta(t *testing.T) { func TestSetMeta(t *testing.T) {
t.Parallel() t.Parallel()
meta := structs.QueryMeta{ meta := structs.QueryMeta{
Index: 1000, Index: 1000,
KnownLeader: true, KnownLeader: true,
LastContact: 123456 * time.Microsecond, LastContact: 123456 * time.Microsecond,
ResultsFilteredByACLs: true,
} }
resp := httptest.NewRecorder() resp := httptest.NewRecorder()
setMeta(resp, &meta) setMeta(resp, &meta)
header := resp.Header().Get("X-Consul-Index")
if header != "1000" { testCases := map[string]string{
t.Fatalf("Bad: %v", header) "X-Consul-Index": "1000",
"X-Consul-KnownLeader": "true",
"X-Consul-LastContact": "123",
"X-Consul-Results-Filtered-By-ACLs": "true",
} }
header = resp.Header().Get("X-Consul-KnownLeader") for header, expectedValue := range testCases {
if header != "true" { if v := resp.Header().Get(header); v != expectedValue {
t.Fatalf("Bad: %v", header) t.Fatalf("expected %q for header %s got %q", expectedValue, header, v)
} }
header = resp.Header().Get("X-Consul-LastContact")
if header != "123" {
t.Fatalf("Bad: %v", header)
} }
} }

View File

@ -45,6 +45,8 @@ type QueryMetaCompat interface {
GetConsistencyLevel() string GetConsistencyLevel() string
SetConsistencyLevel(string) SetConsistencyLevel(string)
GetBackend() QueryBackend GetBackend() QueryBackend
GetResultsFilteredByACLs() bool
SetResultsFilteredByACLs(bool)
} }
// GetToken helps implement the QueryOptionsCompat interface // GetToken helps implement the QueryOptionsCompat interface
@ -274,3 +276,15 @@ func (q *QueryMeta) SetConsistencyLevel(consistencyLevel string) {
func (q *QueryMeta) GetBackend() QueryBackend { func (q *QueryMeta) GetBackend() QueryBackend {
return q.Backend return q.Backend
} }
// GetResultsFilteredByACLs is needed to implement the structs.QueryMetaCompat
// interface.
func (q *QueryMeta) GetResultsFilteredByACLs() bool {
return q.ResultsFilteredByACLs
}
// SetResultsFilteredByACLs is needed to implement the structs.QueryMetaCompat
// interface.
func (q *QueryMeta) SetResultsFilteredByACLs(v bool) {
q.ResultsFilteredByACLs = v
}

View File

@ -392,6 +392,11 @@ type QueryMeta struct {
// Backend used to handle this query, either blocking-query or streaming. // Backend used to handle this query, either blocking-query or streaming.
Backend QueryBackend Backend QueryBackend
// ResultsFilteredByACLs is true when some of the query's results were
// filtered out by enforcing ACLs. It may be false because nothing was
// removed, or because the endpoint does not yet support this flag.
ResultsFilteredByACLs bool
} }
// RegisterRequest is used for the Catalog.Register endpoint // RegisterRequest is used for the Catalog.Register endpoint

View File

@ -2449,10 +2449,11 @@ func TestSnapshotRequestResponse_MsgpackEncodeDecode(t *testing.T) {
in := &SnapshotResponse{ in := &SnapshotResponse{
Error: "blah", Error: "blah",
QueryMeta: QueryMeta{ QueryMeta: QueryMeta{
Index: 3, Index: 3,
LastContact: 5 * time.Second, LastContact: 5 * time.Second,
KnownLeader: true, KnownLeader: true,
ConsistencyLevel: "default", ConsistencyLevel: "default",
ResultsFilteredByACLs: true,
}, },
} }
TestMsgpackEncodeDecode(t, in, true) TestMsgpackEncodeDecode(t, in, true)

View File

@ -281,6 +281,11 @@ type QueryMeta struct {
// defined policy. This can be "allow" which means ACLs are used to // defined policy. This can be "allow" which means ACLs are used to
// deny-list, or "deny" which means ACLs are allow-lists. // deny-list, or "deny" which means ACLs are allow-lists.
DefaultACLPolicy string DefaultACLPolicy string
// ResultsFilteredByACLs is true when some of the query's results were
// filtered out by enforcing ACLs. It may be false because nothing was
// removed, or because the endpoint does not yet support this flag.
ResultsFilteredByACLs bool
} }
// WriteMeta is used to return meta data about a write // WriteMeta is used to return meta data about a write
@ -1071,6 +1076,14 @@ func parseQueryMeta(resp *http.Response, q *QueryMeta) error {
q.DefaultACLPolicy = v q.DefaultACLPolicy = v
} }
// Parse the X-Consul-Results-Filtered-By-ACLs
switch header.Get("X-Consul-Results-Filtered-By-ACLs") {
case "true":
q.ResultsFilteredByACLs = true
default:
q.ResultsFilteredByACLs = false
}
// Parse Cache info // Parse Cache info
if cacheStr := header.Get("X-Cache"); cacheStr != "" { if cacheStr := header.Get("X-Cache"); cacheStr != "" {
q.CacheHit = strings.EqualFold(cacheStr, "HIT") q.CacheHit = strings.EqualFold(cacheStr, "HIT")

View File

@ -932,6 +932,7 @@ func TestAPI_ParseQueryMeta(t *testing.T) {
resp.Header.Set("X-Consul-KnownLeader", "true") resp.Header.Set("X-Consul-KnownLeader", "true")
resp.Header.Set("X-Consul-Translate-Addresses", "true") resp.Header.Set("X-Consul-Translate-Addresses", "true")
resp.Header.Set("X-Consul-Default-ACL-Policy", "deny") resp.Header.Set("X-Consul-Default-ACL-Policy", "deny")
resp.Header.Set("X-Consul-Results-Filtered-By-ACLs", "true")
qm := &QueryMeta{} qm := &QueryMeta{}
if err := parseQueryMeta(resp, qm); err != nil { if err := parseQueryMeta(resp, qm); err != nil {
@ -953,6 +954,9 @@ func TestAPI_ParseQueryMeta(t *testing.T) {
if qm.DefaultACLPolicy != "deny" { if qm.DefaultACLPolicy != "deny" {
t.Fatalf("Bad: %v", qm) t.Fatalf("Bad: %v", qm)
} }
if !qm.ResultsFilteredByACLs {
t.Fatalf("Bad: %v", qm)
}
} }
func TestAPI_UnixSocket(t *testing.T) { func TestAPI_UnixSocket(t *testing.T) {

View File

@ -391,6 +391,10 @@ type QueryMeta struct {
// Having `discovery_max_stale` on the agent can affect whether // Having `discovery_max_stale` on the agent can affect whether
// the request was served by a leader. // the request was served by a leader.
ConsistencyLevel string `protobuf:"bytes,4,opt,name=ConsistencyLevel,proto3" json:"ConsistencyLevel,omitempty"` ConsistencyLevel string `protobuf:"bytes,4,opt,name=ConsistencyLevel,proto3" json:"ConsistencyLevel,omitempty"`
// ResultsFilteredByACLs is true when some of the query's results were
// filtered out by enforcing ACLs. It may be false because nothing was
// removed, or because the endpoint does not yet support this flag.
ResultsFilteredByACLs bool `protobuf:"varint,7,opt,name=ResultsFilteredByACLs,proto3" json:"ResultsFilteredByACLs,omitempty"`
} }
func (m *QueryMeta) Reset() { *m = QueryMeta{} } func (m *QueryMeta) Reset() { *m = QueryMeta{} }
@ -454,6 +458,13 @@ func (m *QueryMeta) GetConsistencyLevel() string {
return "" return ""
} }
func (m *QueryMeta) GetResultsFilteredByACLs() bool {
if m != nil {
return m.ResultsFilteredByACLs
}
return false
}
// EnterpriseMeta contains metadata that is only used by the Enterprise version // EnterpriseMeta contains metadata that is only used by the Enterprise version
// of Consul. // of Consul.
type EnterpriseMeta struct { type EnterpriseMeta struct {
@ -509,46 +520,49 @@ func init() {
func init() { proto.RegisterFile("proto/pbcommon/common.proto", fileDescriptor_a6f5ac44994d718c) } func init() { proto.RegisterFile("proto/pbcommon/common.proto", fileDescriptor_a6f5ac44994d718c) }
var fileDescriptor_a6f5ac44994d718c = []byte{ var fileDescriptor_a6f5ac44994d718c = []byte{
// 620 bytes of a gzipped FileDescriptorProto // 657 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0x41, 0x4f, 0xd4, 0x40, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x4e, 0xdb, 0x4a,
0x14, 0xde, 0xe2, 0xb2, 0x6e, 0xdf, 0x02, 0xc1, 0x09, 0x31, 0x15, 0x4d, 0x97, 0x6c, 0x8c, 0x21, 0x18, 0x8d, 0xb9, 0x21, 0xd8, 0x13, 0x40, 0xb9, 0x23, 0xee, 0x95, 0x2f, 0xb7, 0x72, 0x90, 0x55,
0x44, 0xb7, 0x09, 0xde, 0xf0, 0x04, 0x0b, 0x1a, 0xe2, 0x56, 0x74, 0xc4, 0x90, 0x78, 0x9b, 0xed, 0x55, 0x08, 0xb5, 0xb1, 0x44, 0xbb, 0xa2, 0x2b, 0x12, 0x68, 0x05, 0x8d, 0x4b, 0x3b, 0xa5, 0x42,
0xbe, 0xed, 0x4e, 0x6c, 0x3b, 0x75, 0x3a, 0x85, 0xe5, 0x1f, 0x78, 0xf4, 0x48, 0x3c, 0xf9, 0x43, 0xea, 0x6e, 0x62, 0x7f, 0x71, 0xac, 0x3a, 0x1e, 0x77, 0x66, 0x0c, 0xc9, 0x1b, 0x74, 0xd9, 0x25,
0xfc, 0x01, 0x1c, 0x39, 0x7a, 0x42, 0x65, 0xff, 0x81, 0xbf, 0xc0, 0x74, 0x5a, 0xa0, 0x08, 0x18, 0xea, 0xaa, 0x8f, 0xc3, 0x92, 0x65, 0x57, 0xb4, 0x25, 0x6f, 0xd0, 0x07, 0xa8, 0x2a, 0x8f, 0x0d,
0x3c, 0xed, 0x7e, 0xdf, 0x7c, 0xdf, 0xeb, 0x9b, 0xf7, 0xbe, 0x16, 0xee, 0xc7, 0x52, 0x28, 0xe1, 0x98, 0x02, 0x15, 0x5d, 0x25, 0xe7, 0xcc, 0x39, 0xdf, 0x7c, 0x7f, 0x63, 0xf4, 0x7f, 0xc2, 0x99,
0xc4, 0x3d, 0x4f, 0x84, 0xa1, 0x88, 0x9c, 0xfc, 0xa7, 0xad, 0x59, 0x52, 0xcb, 0xd1, 0xbc, 0xed, 0x64, 0x4e, 0xd2, 0xf3, 0xd8, 0x70, 0xc8, 0x62, 0x27, 0xff, 0x69, 0x29, 0x16, 0xd7, 0x72, 0xb4,
0x0b, 0xe1, 0x07, 0xe8, 0x68, 0xb6, 0x97, 0x0e, 0x9c, 0x7e, 0x2a, 0x99, 0xe2, 0xa7, 0xba, 0xf9, 0x68, 0x05, 0x8c, 0x05, 0x11, 0x38, 0x8a, 0xed, 0xa5, 0x7d, 0xc7, 0x4f, 0x39, 0x95, 0xe1, 0x99,
0x39, 0x5f, 0xf8, 0x22, 0x2f, 0x94, 0xfd, 0xcb, 0xd9, 0x56, 0x08, 0x26, 0x65, 0x03, 0xb5, 0x19, 0x6e, 0x71, 0x21, 0x60, 0x01, 0xcb, 0x03, 0x65, 0xff, 0x72, 0xd6, 0x1e, 0x22, 0x83, 0xd0, 0xbe,
0xf5, 0x71, 0x44, 0x1c, 0x68, 0x74, 0x24, 0x32, 0x85, 0x1a, 0x5a, 0xc6, 0x82, 0xb1, 0x58, 0x5d, 0xdc, 0x8a, 0x7d, 0x18, 0x61, 0x07, 0xd5, 0x3b, 0x1c, 0xa8, 0x04, 0x05, 0x4d, 0x6d, 0x49, 0x5b,
0x9b, 0xfe, 0x7d, 0xdc, 0x34, 0x7b, 0x38, 0x8a, 0xe5, 0x4a, 0xeb, 0x49, 0x8b, 0x96, 0x15, 0x99, 0xae, 0xb6, 0xe7, 0xbe, 0x9f, 0x34, 0x8d, 0x1e, 0x8c, 0x12, 0xbe, 0x66, 0x3f, 0xb0, 0x49, 0x59,
0xc1, 0x15, 0x7d, 0x3e, 0xd8, 0xcf, 0x0d, 0x13, 0x57, 0x1a, 0x4a, 0x8a, 0xd6, 0x32, 0xcc, 0x6e, 0x91, 0x19, 0x5c, 0xe6, 0x87, 0xfd, 0x71, 0x6e, 0x98, 0xba, 0xd6, 0x50, 0x52, 0xd8, 0xab, 0xa8,
0x33, 0xe9, 0xa3, 0x5a, 0x67, 0x8a, 0x79, 0x18, 0x29, 0x94, 0xc4, 0x06, 0x38, 0x47, 0xfa, 0xa1, 0xb1, 0x4b, 0x79, 0x00, 0x72, 0x83, 0x4a, 0xea, 0x41, 0x2c, 0x81, 0x63, 0x0b, 0xa1, 0x0b, 0xa4,
0x26, 0x2d, 0x31, 0xad, 0x25, 0x98, 0xda, 0x91, 0x5c, 0x21, 0xc5, 0x8f, 0x29, 0x26, 0x8a, 0xcc, 0x2e, 0x35, 0x48, 0x89, 0xb1, 0x57, 0xd0, 0xec, 0x1e, 0x0f, 0x25, 0x10, 0x78, 0x97, 0x82, 0x90,
0xc1, 0xe4, 0xb6, 0xf8, 0x80, 0x51, 0x21, 0xcd, 0xc1, 0x4a, 0xf5, 0xd3, 0xd7, 0xa6, 0xd1, 0xda, 0x78, 0x01, 0x4d, 0xef, 0xb2, 0xb7, 0x10, 0x17, 0xd2, 0x1c, 0xac, 0x55, 0xdf, 0x7f, 0x6a, 0x6a,
0x81, 0x06, 0x45, 0xd6, 0xff, 0xa7, 0x94, 0x3c, 0x86, 0x3b, 0x99, 0x80, 0x4b, 0xec, 0x88, 0x28, 0xf6, 0x1e, 0xaa, 0x13, 0xa0, 0xfe, 0x6f, 0xa5, 0xf8, 0x3e, 0xfa, 0x3b, 0x13, 0x84, 0x1c, 0x3a,
0xe1, 0x89, 0xc2, 0x48, 0xe9, 0xde, 0xeb, 0xf4, 0xf2, 0x41, 0x51, 0xf8, 0x4b, 0x15, 0xa6, 0xde, 0x2c, 0x16, 0xa1, 0x90, 0x10, 0x4b, 0x95, 0xbb, 0x4e, 0xae, 0x1e, 0x14, 0x81, 0x3f, 0x56, 0xd1,
0xa4, 0x28, 0xf7, 0xb7, 0xe2, 0x6c, 0xa6, 0xc9, 0x35, 0xa5, 0x1f, 0xc2, 0xb4, 0xcb, 0x23, 0x2d, 0xec, 0xcb, 0x14, 0xf8, 0x78, 0x27, 0xc9, 0x7a, 0x2a, 0x6e, 0x08, 0x7d, 0x17, 0xcd, 0xb9, 0x61,
0x2c, 0x8d, 0x84, 0x5e, 0x24, 0xc9, 0x0b, 0x98, 0x72, 0xd9, 0x48, 0x13, 0xdb, 0x3c, 0x44, 0xeb, 0xac, 0x84, 0xa5, 0x96, 0x90, 0xcb, 0x24, 0x7e, 0x8a, 0x66, 0x5d, 0x3a, 0x52, 0xc4, 0x6e, 0x38,
0xd6, 0x82, 0xb1, 0xd8, 0x58, 0xbe, 0xd7, 0xce, 0x37, 0xd8, 0x3e, 0xdd, 0x60, 0x7b, 0xbd, 0xd8, 0x04, 0xf3, 0xaf, 0x25, 0x6d, 0xb9, 0xbe, 0xfa, 0x5f, 0x2b, 0x9f, 0x60, 0xeb, 0x6c, 0x82, 0xad,
0xe0, 0x5a, 0xfd, 0xf0, 0xb8, 0x59, 0x39, 0xf8, 0xd1, 0x34, 0xe8, 0x05, 0x63, 0x36, 0xba, 0xd5, 0x8d, 0x62, 0x82, 0x6d, 0xfd, 0xe8, 0xa4, 0x59, 0x39, 0xfc, 0xd2, 0xd4, 0xc8, 0x25, 0x63, 0xd6,
0x20, 0x10, 0x7b, 0x6f, 0x15, 0x0b, 0xd0, 0xaa, 0xea, 0x2b, 0x94, 0x98, 0xab, 0x6f, 0x3a, 0x79, 0xba, 0xf5, 0x28, 0x62, 0x07, 0xaf, 0x24, 0x8d, 0xc0, 0xac, 0xaa, 0x12, 0x4a, 0xcc, 0xf5, 0x95,
0xcd, 0x4d, 0xc9, 0x3c, 0xd4, 0xdf, 0x25, 0xd8, 0x61, 0xde, 0x10, 0xad, 0x9a, 0x16, 0x9d, 0x61, 0x4e, 0xdf, 0x50, 0x29, 0x5e, 0x44, 0xfa, 0x6b, 0x01, 0x1d, 0xea, 0x0d, 0xc0, 0xac, 0x29, 0xd1,
0xb2, 0x05, 0xb3, 0x2e, 0x1b, 0xe9, 0xaa, 0xa7, 0x5d, 0x59, 0xb7, 0x6f, 0xde, 0xf6, 0x25, 0x33, 0x39, 0xc6, 0x3b, 0xa8, 0xe1, 0xd2, 0x91, 0x8a, 0x7a, 0x96, 0x95, 0x39, 0x73, 0xfb, 0xb4, 0xaf,
0x79, 0x06, 0x35, 0x97, 0x8d, 0x56, 0x7d, 0xb4, 0xea, 0x37, 0x2f, 0x53, 0x58, 0xc8, 0x23, 0x98, 0x98, 0xf1, 0x63, 0x54, 0x73, 0xe9, 0x68, 0x3d, 0x00, 0x53, 0xbf, 0x7d, 0x98, 0xc2, 0x82, 0xef,
0x71, 0xd3, 0x44, 0x51, 0xdc, 0x65, 0x01, 0xef, 0x33, 0x85, 0x96, 0xa9, 0xfb, 0xfd, 0x8b, 0xcd, 0xa1, 0x79, 0x37, 0x15, 0x92, 0xc0, 0x3e, 0x8d, 0x42, 0x9f, 0x4a, 0x30, 0x0d, 0x95, 0xef, 0x2f,
0x06, 0xad, 0x9f, 0xba, 0x39, 0xd8, 0x90, 0x52, 0x48, 0x0b, 0xfe, 0x63, 0xd0, 0x65, 0x23, 0xb9, 0x6c, 0xd6, 0x68, 0x75, 0xeb, 0x56, 0x7f, 0x93, 0x73, 0xc6, 0x4d, 0xf4, 0x07, 0x8d, 0x2e, 0x1b,
0x0b, 0xb5, 0xe7, 0x3c, 0xc8, 0xf2, 0xd9, 0xd0, 0xeb, 0x2e, 0x50, 0x11, 0x8e, 0x6f, 0x06, 0x98, 0xf1, 0xbf, 0xa8, 0xf6, 0x24, 0x8c, 0xb2, 0xfd, 0xac, 0xab, 0x71, 0x17, 0xa8, 0x58, 0x8e, 0x1f,
0x7a, 0x29, 0x2e, 0x2a, 0x96, 0x25, 0xa3, 0xf4, 0xfe, 0xd0, 0x1c, 0x90, 0x0d, 0x68, 0x74, 0x59, 0x1a, 0x32, 0xd4, 0x50, 0x5c, 0x90, 0x34, 0xdb, 0x8c, 0xd2, 0xfb, 0x21, 0x39, 0xc0, 0x9b, 0xa8,
0xa2, 0x3a, 0x22, 0x52, 0xcc, 0xcb, 0xe3, 0x76, 0xc3, 0x4e, 0xca, 0x3e, 0xb2, 0x00, 0x8d, 0x97, 0xde, 0xa5, 0x42, 0x76, 0x58, 0x2c, 0xa9, 0x97, 0xaf, 0xdb, 0x2d, 0x33, 0x29, 0xfb, 0xf0, 0x12,
0x91, 0xd8, 0x8b, 0xba, 0xc8, 0xfa, 0x28, 0x75, 0x72, 0xea, 0xb4, 0x4c, 0x91, 0x25, 0x98, 0x3d, 0xaa, 0x3f, 0x8b, 0xd9, 0x41, 0xdc, 0x05, 0xea, 0x03, 0x57, 0x9b, 0xa3, 0x93, 0x32, 0x85, 0x57,
0xdb, 0xa9, 0xb7, 0xdf, 0xc5, 0x5d, 0x0c, 0x74, 0x32, 0x4c, 0x7a, 0x89, 0x2f, 0xda, 0xef, 0xc2, 0x50, 0xe3, 0x7c, 0xa6, 0xde, 0xb8, 0x0b, 0xfb, 0x10, 0xa9, 0xcd, 0x30, 0xc8, 0x15, 0x1e, 0x3f,
0xcc, 0x46, 0xf6, 0xa6, 0xc5, 0x92, 0x27, 0xa8, 0xaf, 0xf0, 0x00, 0xcc, 0x57, 0x2c, 0xc4, 0x24, 0x42, 0xff, 0x10, 0x10, 0x69, 0x24, 0x45, 0x5e, 0x0f, 0xf8, 0xed, 0xf1, 0x7a, 0xa7, 0x2b, 0xd4,
0x66, 0x1e, 0x16, 0x01, 0x3f, 0x27, 0xb2, 0xd3, 0xd7, 0x4c, 0x2a, 0xae, 0x43, 0x30, 0x91, 0x9f, 0x68, 0x75, 0x72, 0xfd, 0x61, 0x5e, 0xf4, 0x76, 0x55, 0x9f, 0x6e, 0xd4, 0xb6, 0xab, 0x7a, 0xad,
0x9e, 0x11, 0x6b, 0xdd, 0xc3, 0x5f, 0x76, 0xe5, 0xf0, 0xc4, 0x36, 0x8e, 0x4e, 0x6c, 0xe3, 0xe7, 0x31, 0x63, 0x77, 0xd1, 0xfc, 0x66, 0xf6, 0x56, 0x13, 0x1e, 0x0a, 0x50, 0x4d, 0xb8, 0x83, 0x8c,
0x89, 0x6d, 0x7c, 0x1e, 0xdb, 0x95, 0x83, 0xb1, 0x5d, 0x39, 0x1a, 0xdb, 0x95, 0xef, 0x63, 0xbb, 0xe7, 0x74, 0x08, 0x22, 0xa1, 0x1e, 0x14, 0x4f, 0xe4, 0x82, 0xc8, 0x4e, 0x5f, 0x50, 0x2e, 0x43,
0xf2, 0x7e, 0xc9, 0xe7, 0x6a, 0x98, 0xf6, 0xda, 0x9e, 0x08, 0x9d, 0x21, 0x4b, 0x86, 0xdc, 0x13, 0xb5, 0x46, 0x53, 0xf9, 0xe9, 0x39, 0xd1, 0xee, 0x1e, 0x7d, 0xb3, 0x2a, 0x47, 0xa7, 0x96, 0x76,
0x32, 0x76, 0x3c, 0x11, 0x25, 0x69, 0xe0, 0x5c, 0xfc, 0xd4, 0xf5, 0x6a, 0x1a, 0x3f, 0xfd, 0x13, 0x7c, 0x6a, 0x69, 0x5f, 0x4f, 0x2d, 0xed, 0xc3, 0xc4, 0xaa, 0x1c, 0x4e, 0xac, 0xca, 0xf1, 0xc4,
0x00, 0x00, 0xff, 0xff, 0x9c, 0xf6, 0xbd, 0xcc, 0x03, 0x05, 0x00, 0x00, 0xaa, 0x7c, 0x9e, 0x58, 0x95, 0x37, 0x2b, 0x41, 0x28, 0x07, 0x69, 0xaf, 0xe5, 0xb1, 0xa1, 0x33,
0xa0, 0x62, 0x10, 0x7a, 0x8c, 0x27, 0x8e, 0xc7, 0x62, 0x91, 0x46, 0xce, 0xe5, 0x8f, 0x65, 0xaf,
0xa6, 0xf0, 0xc3, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xab, 0xfa, 0x4f, 0xec, 0x45, 0x05, 0x00,
0x00,
} }
func (m *RaftIndex) Marshal() (dAtA []byte, err error) { func (m *RaftIndex) Marshal() (dAtA []byte, err error) {
@ -818,6 +832,16 @@ func (m *QueryMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i _ = i
var l int var l int
_ = l _ = l
if m.ResultsFilteredByACLs {
i--
if m.ResultsFilteredByACLs {
dAtA[i] = 1
} else {
dAtA[i] = 0
}
i--
dAtA[i] = 0x38
}
if len(m.ConsistencyLevel) > 0 { if len(m.ConsistencyLevel) > 0 {
i -= len(m.ConsistencyLevel) i -= len(m.ConsistencyLevel)
copy(dAtA[i:], m.ConsistencyLevel) copy(dAtA[i:], m.ConsistencyLevel)
@ -1014,6 +1038,9 @@ func (m *QueryMeta) Size() (n int) {
if l > 0 { if l > 0 {
n += 1 + l + sovCommon(uint64(l)) n += 1 + l + sovCommon(uint64(l))
} }
if m.ResultsFilteredByACLs {
n += 2
}
return n return n
} }
@ -1872,6 +1899,26 @@ func (m *QueryMeta) Unmarshal(dAtA []byte) error {
} }
m.ConsistencyLevel = string(dAtA[iNdEx:postIndex]) m.ConsistencyLevel = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex iNdEx = postIndex
case 7:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field ResultsFilteredByACLs", wireType)
}
var v int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowCommon
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
v |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
m.ResultsFilteredByACLs = bool(v != 0)
default: default:
iNdEx = preIndex iNdEx = preIndex
skippy, err := skipCommon(dAtA[iNdEx:]) skippy, err := skipCommon(dAtA[iNdEx:])

View File

@ -150,6 +150,14 @@ message QueryMeta {
// Having `discovery_max_stale` on the agent can affect whether // Having `discovery_max_stale` on the agent can affect whether
// the request was served by a leader. // the request was served by a leader.
string ConsistencyLevel = 4; string ConsistencyLevel = 4;
// Reserved for NotModified and Backend.
reserved 5, 6;
// ResultsFilteredByACLs is true when some of the query's results were
// filtered out by enforcing ACLs. It may be false because nothing was
// removed, or because the endpoint does not yet support this flag.
bool ResultsFilteredByACLs = 7;
} }
// EnterpriseMeta contains metadata that is only used by the Enterprise version // EnterpriseMeta contains metadata that is only used by the Enterprise version