Merge pull request #2945 from murgatroid99/grpc-js-xds_rbac_filter

grpc-js-xds: Implement RBAC HTTP filter
This commit is contained in:
Michael Lumish 2025-04-21 10:48:23 -07:00 committed by GitHub
commit 0157776059
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 528 additions and 18 deletions

View File

@ -71,6 +71,7 @@ const copyTestFixtures = checkTask(() =>
const runTests = checkTask(() => {
process.env.GRPC_EXPERIMENTAL_XDS_FEDERATION = 'true';
process.env.GRPC_EXPERIMENTAL_PICKFIRST_LB_CONFIG = 'true';
process.env.GRPC_XDS_EXPERIMENTAL_RBAC = 'true';
if (Number(process.versions.node.split('.')[0]) <= 14) {
process.env.GRPC_XDS_EXPERIMENTAL_ENABLE_RING_HASH = 'false';
}

View File

@ -46,7 +46,7 @@ COPY --from=build /node/src/grpc-node/packages/grpc-js ./packages/grpc-js/
COPY --from=build /node/src/grpc-node/packages/grpc-js-xds ./packages/grpc-js-xds/
ENV GRPC_VERBOSITY="DEBUG"
ENV GRPC_TRACE=xds_client,server,xds_server,http_filter,certificate_provider
ENV GRPC_TRACE=xds_client,server,xds_server,http_filter,certificate_provider,rbac_filter
# tini serves as PID 1 and enables the server to properly respond to signals.
COPY --from=build /tini /tini

View File

@ -12,7 +12,7 @@
"prepare": "npm run generate-types && npm run compile",
"pretest": "npm run compile",
"posttest": "npm run check",
"generate-types": "proto-loader-gen-types --keepCase --longs String --enums String --defaults --oneofs --includeComments --includeDirs deps/envoy-api/ deps/xds/ deps/googleapis/ deps/protoc-gen-validate/ -O src/generated/ --grpcLib @grpc/grpc-js envoy/service/discovery/v3/ads.proto envoy/service/load_stats/v3/lrs.proto envoy/config/listener/v3/listener.proto envoy/config/route/v3/route.proto envoy/config/cluster/v3/cluster.proto envoy/config/endpoint/v3/endpoint.proto envoy/extensions/filters/network/http_connection_manager/v3/http_connection_manager.proto udpa/type/v1/typed_struct.proto xds/type/v3/typed_struct.proto envoy/extensions/filters/http/fault/v3/fault.proto envoy/service/status/v3/csds.proto envoy/extensions/load_balancing_policies/wrr_locality/v3/wrr_locality.proto envoy/extensions/load_balancing_policies/ring_hash/v3/ring_hash.proto envoy/extensions/load_balancing_policies/pick_first/v3/pick_first.proto envoy/extensions/clusters/aggregate/v3/cluster.proto envoy/extensions/transport_sockets/tls/v3/tls.proto envoy/config/rbac/v3/rbac.proto",
"generate-types": "proto-loader-gen-types --keepCase --longs String --enums String --defaults --oneofs --includeComments --includeDirs deps/envoy-api/ deps/xds/ deps/googleapis/ deps/protoc-gen-validate/ -O src/generated/ --grpcLib @grpc/grpc-js envoy/service/discovery/v3/ads.proto envoy/service/load_stats/v3/lrs.proto envoy/config/listener/v3/listener.proto envoy/config/route/v3/route.proto envoy/config/cluster/v3/cluster.proto envoy/config/endpoint/v3/endpoint.proto envoy/extensions/filters/network/http_connection_manager/v3/http_connection_manager.proto udpa/type/v1/typed_struct.proto xds/type/v3/typed_struct.proto envoy/extensions/filters/http/fault/v3/fault.proto envoy/service/status/v3/csds.proto envoy/extensions/load_balancing_policies/wrr_locality/v3/wrr_locality.proto envoy/extensions/load_balancing_policies/ring_hash/v3/ring_hash.proto envoy/extensions/load_balancing_policies/pick_first/v3/pick_first.proto envoy/extensions/clusters/aggregate/v3/cluster.proto envoy/extensions/transport_sockets/tls/v3/tls.proto envoy/config/rbac/v3/rbac.proto envoy/extensions/filters/http/rbac/v3/rbac.proto",
"generate-interop-types": "proto-loader-gen-types --keep-case --longs String --enums String --defaults --oneofs --json --includeComments --includeDirs proto/ -O interop/generated --grpcLib @grpc/grpc-js grpc/testing/test.proto",
"generate-test-types": "proto-loader-gen-types --keep-case --longs String --enums String --defaults --oneofs --json --includeComments --includeDirs proto/ -O test/generated --grpcLib @grpc/grpc-js grpc/testing/echo.proto"
},

View File

@ -27,3 +27,4 @@ export const EXPERIMENTAL_RING_HASH = (process.env.GRPC_XDS_EXPERIMENTAL_ENABLE_
export const EXPERIMENTAL_PICK_FIRST = (process.env.GRPC_EXPERIMENTAL_PICKFIRST_LB_CONFIG ?? 'false') === 'true';
export const EXPERIMENTAL_DUALSTACK_ENDPOINTS = (process.env.GRPC_EXPERIMENTAL_XDS_DUALSTACK_ENDPOINTS ?? 'true') === 'true';
export const AGGREGATE_CLUSTER_BACKWARDS_COMPAT = (process.env.GRPC_XDS_AGGREGATE_CLUSTER_BACKWARD_COMPAT ?? 'false') === 'true';
export const EXPERIMENTAL_RBAC = (process.env.GRPC_XDS_EXPERIMENTAL_RBAC ?? 'false') === 'true';

View File

@ -0,0 +1,104 @@
// Original file: deps/envoy-api/envoy/extensions/filters/http/rbac/v3/rbac.proto
import type { RBAC as _envoy_config_rbac_v3_RBAC, RBAC__Output as _envoy_config_rbac_v3_RBAC__Output } from '../../../../../../envoy/config/rbac/v3/RBAC';
import type { Matcher as _xds_type_matcher_v3_Matcher, Matcher__Output as _xds_type_matcher_v3_Matcher__Output } from '../../../../../../xds/type/matcher/v3/Matcher';
/**
* RBAC filter config.
* [#next-free-field: 8]
*/
export interface RBAC {
/**
* Specify the RBAC rules to be applied globally.
* If absent, no enforcing RBAC policy will be applied.
* If present and empty, DENY.
* If both rules and matcher are configured, rules will be ignored.
*/
'rules'?: (_envoy_config_rbac_v3_RBAC | null);
/**
* Shadow rules are not enforced by the filter (i.e., returning a 403)
* but will emit stats and logs and can be used for rule testing.
* If absent, no shadow RBAC policy will be applied.
* If both shadow rules and shadow matcher are configured, shadow rules will be ignored.
*/
'shadow_rules'?: (_envoy_config_rbac_v3_RBAC | null);
/**
* If specified, shadow rules will emit stats with the given prefix.
* This is useful to distinguish the stat when there are more than 1 RBAC filter configured with
* shadow rules.
*/
'shadow_rules_stat_prefix'?: (string);
/**
* The match tree to use when resolving RBAC action for incoming requests. Requests do not
* match any matcher will be denied.
* If absent, no enforcing RBAC matcher will be applied.
* If present and empty, deny all requests.
*/
'matcher'?: (_xds_type_matcher_v3_Matcher | null);
/**
* The match tree to use for emitting stats and logs which can be used for rule testing for
* incoming requests.
* If absent, no shadow matcher will be applied.
*/
'shadow_matcher'?: (_xds_type_matcher_v3_Matcher | null);
/**
* If specified, rules will emit stats with the given prefix.
* This is useful to distinguish the stat when there are more than 1 RBAC filter configured with
* rules.
*/
'rules_stat_prefix'?: (string);
/**
* If track_per_rule_stats is true, counters will be published for each rule and shadow rule.
*/
'track_per_rule_stats'?: (boolean);
}
/**
* RBAC filter config.
* [#next-free-field: 8]
*/
export interface RBAC__Output {
/**
* Specify the RBAC rules to be applied globally.
* If absent, no enforcing RBAC policy will be applied.
* If present and empty, DENY.
* If both rules and matcher are configured, rules will be ignored.
*/
'rules': (_envoy_config_rbac_v3_RBAC__Output | null);
/**
* Shadow rules are not enforced by the filter (i.e., returning a 403)
* but will emit stats and logs and can be used for rule testing.
* If absent, no shadow RBAC policy will be applied.
* If both shadow rules and shadow matcher are configured, shadow rules will be ignored.
*/
'shadow_rules': (_envoy_config_rbac_v3_RBAC__Output | null);
/**
* If specified, shadow rules will emit stats with the given prefix.
* This is useful to distinguish the stat when there are more than 1 RBAC filter configured with
* shadow rules.
*/
'shadow_rules_stat_prefix': (string);
/**
* The match tree to use when resolving RBAC action for incoming requests. Requests do not
* match any matcher will be denied.
* If absent, no enforcing RBAC matcher will be applied.
* If present and empty, deny all requests.
*/
'matcher': (_xds_type_matcher_v3_Matcher__Output | null);
/**
* The match tree to use for emitting stats and logs which can be used for rule testing for
* incoming requests.
* If absent, no shadow matcher will be applied.
*/
'shadow_matcher': (_xds_type_matcher_v3_Matcher__Output | null);
/**
* If specified, rules will emit stats with the given prefix.
* This is useful to distinguish the stat when there are more than 1 RBAC filter configured with
* rules.
*/
'rules_stat_prefix': (string);
/**
* If track_per_rule_stats is true, counters will be published for each rule and shadow rule.
*/
'track_per_rule_stats': (boolean);
}

View File

@ -0,0 +1,19 @@
// Original file: deps/envoy-api/envoy/extensions/filters/http/rbac/v3/rbac.proto
import type { RBAC as _envoy_extensions_filters_http_rbac_v3_RBAC, RBAC__Output as _envoy_extensions_filters_http_rbac_v3_RBAC__Output } from '../../../../../../envoy/extensions/filters/http/rbac/v3/RBAC';
export interface RBACPerRoute {
/**
* Override the global configuration of the filter with this new config.
* If absent, the global RBAC policy will be disabled for this route.
*/
'rbac'?: (_envoy_extensions_filters_http_rbac_v3_RBAC | null);
}
export interface RBACPerRoute__Output {
/**
* Override the global configuration of the filter with this new config.
* If absent, the global RBAC policy will be disabled for this route.
*/
'rbac': (_envoy_extensions_filters_http_rbac_v3_RBAC__Output | null);
}

View File

@ -93,6 +93,18 @@ export interface ProtoGrpcType {
}
}
}
extensions: {
filters: {
http: {
rbac: {
v3: {
RBAC: MessageTypeDefinition
RBACPerRoute: MessageTypeDefinition
}
}
}
}
}
type: {
matcher: {
v3: {

View File

@ -0,0 +1,175 @@
/*
* Copyright 2021 gRPC authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
// This is a non-public, unstable API, but it's very convenient
import { loadProtosWithOptionsSync } from '@grpc/proto-loader/build/src/util';
import { experimental, logVerbosity, ServerInterceptingCall, ServerInterceptor, ServerListener, status } from '@grpc/grpc-js';
import { Any__Output } from '../generated/google/protobuf/Any';
import { HttpFilterConfig, registerHttpFilter } from '../http-filter';
import { RbacPolicyGroup, UnifiedInfo as UnifiedRbacInfo } from '../rbac';
import { RBAC__Output } from '../generated/envoy/extensions/filters/http/rbac/v3/RBAC';
import { RBACPerRoute__Output } from '../generated/envoy/extensions/filters/http/rbac/v3/RBACPerRoute';
import { parseConfig as parseRbacConfig } from '../rbac';
import { EXPERIMENTAL_RBAC } from '../environment';
const TRACER_NAME = 'rbac_filter';
function trace(text: string): void {
experimental.trace(logVerbosity.DEBUG, TRACER_NAME, text);
}
const resourceRoot = loadProtosWithOptionsSync([
'envoy/extensions/filters/http/rbac/v3/rbac.proto'], {
keepCase: true,
includeDirs: [
// Paths are relative to src/build/http-filter
__dirname + '/../../../deps/xds/',
__dirname + '/../../../deps/envoy-api/',
__dirname + '/../../../deps/protoc-gen-validate/',
__dirname + '/../../../deps/googleapis/'
],
}
);
const RBAC_FILTER_URL = 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC';
const RBAC_FILTER_OVERRIDE_URL ='type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBACPerRoute';
const toObjectOptions = {
longs: String,
enums: String,
defaults: true,
oneofs: true
}
function parseAnyMessage<MessageType>(message: Any__Output): MessageType | null {
const typeName = message.type_url.substring(message.type_url.lastIndexOf('/') + 1);
const messageType = resourceRoot.lookup(typeName);
if (messageType) {
const decodedMessage = (messageType as any).decode(message.value);
return decodedMessage.$type.toObject(decodedMessage, toObjectOptions) as MessageType;
} else {
return null;
}
}
interface RbacFilterConfig extends HttpFilterConfig {
typeUrl: 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC';
config: RbacPolicyGroup;
}
function parseTopLevelRbacConfig(encodedConfig: Any__Output): RbacFilterConfig | null {
if (encodedConfig.type_url !== RBAC_FILTER_URL) {
trace('Config parsing failed: unexpected type URL: ' + encodedConfig.type_url);
return null;
}
const parsedMessage = parseAnyMessage<RBAC__Output>(encodedConfig);
if (parsedMessage === null) {
trace('Config parsing failed: failed to parse RBAC message');
return null;
}
trace('Parsing RBAC message ' + JSON.stringify(parsedMessage, undefined, 2));
if (!parsedMessage.rules) {
trace('Config parsing failed: no rules found');
return null;
}
try {
return {
typeUrl: RBAC_FILTER_URL,
config: parseRbacConfig(parsedMessage.rules)
};
} catch (e) {
trace('Config parsing failed: ' + (e as Error).message);
return null;
}
}
function parseOverrideRbacConfig(encodedConfig: Any__Output): RbacFilterConfig | null {
if (encodedConfig.type_url !== RBAC_FILTER_OVERRIDE_URL) {
trace('Config parsing failed: unexpected type URL: ' + encodedConfig.type_url);
return null;
}
const parsedMessage = parseAnyMessage<RBACPerRoute__Output>(encodedConfig);
if (parsedMessage === null) {
trace('Config parsing failed: failed to parse RBACPerRoute message');
return null;
}
trace('Parsing RBAC message ' + JSON.stringify(parsedMessage, undefined, 2));
if (!parsedMessage.rbac?.rules) {
trace('Config parsing failed: no rules found');
return null;
}
try {
return {
typeUrl: RBAC_FILTER_URL,
config: parseRbacConfig(parsedMessage.rbac.rules)
};
} catch (e) {
trace('Config parsing failed: ' + (e as Error).message);
return null;
}
}
function createRbacServerFilter(config: HttpFilterConfig, overrideConfigMap: Map<string, HttpFilterConfig>): ServerInterceptor {
return function rbacServerFilter(methodDescriptor, call): ServerInterceptingCall {
const listener: ServerListener = {
onReceiveMetadata: (metadata, next) => {
let activeConfig = config;
const routeName = metadata.get('grpc-route')[0];
if (routeName) {
const overrideConfig = overrideConfigMap.get(routeName as string);
if (overrideConfig) {
activeConfig = overrideConfig;
}
}
const rbacMetadata = metadata.clone();
rbacMetadata.set(':method', 'POST');
rbacMetadata.set(':authority', call.getHost());
rbacMetadata.set(':path', methodDescriptor.path);
const connectionInfo = call.getConnectionInfo();
const authContext = call.getAuthContext();
const info: UnifiedRbacInfo = {
destinationIp: connectionInfo.localAddress!,
destinationPort: connectionInfo.localPort!,
sourceIp: connectionInfo.remoteAddress!,
headers: rbacMetadata,
tls: authContext.transportSecurityType !== undefined,
peerCertificate: authContext.sslPeerCertificate ?? null,
urlPath: methodDescriptor.path
};
if ((activeConfig as RbacFilterConfig).config.apply(info)) {
next(metadata);
} else {
call.sendStatus({code: status.PERMISSION_DENIED, details: 'Unauthorized RPC rejected'});
}
}
};
return new ServerInterceptingCall(call, {
start: next => {
next(listener);
}
});
}
}
export function setup() {
if (EXPERIMENTAL_RBAC) {
registerHttpFilter(RBAC_FILTER_URL, {
parseTopLevelFilterConfig: parseTopLevelRbacConfig,
parseOverrideFilterConfig: parseOverrideRbacConfig,
createServerFilter: createRbacServerFilter
});
}
}

View File

@ -25,6 +25,7 @@ import * as xds_wrr_locality from './load-balancer-xds-wrr-locality';
import * as ring_hash from './load-balancer-ring-hash';
import * as router_filter from './http-filter/router-filter';
import * as fault_injection_filter from './http-filter/fault-injection-filter';
import * as rbac_filter from './http-filter/rbac-filter';
import * as csds from './csds';
import * as round_robin_lb from './lb-policy-registry/round-robin';
import * as typed_struct_lb from './lb-policy-registry/typed-struct';
@ -53,6 +54,7 @@ export function register() {
ring_hash.setup();
router_filter.setup();
fault_injection_filter.setup();
rbac_filter.setup();
csds.setup();
round_robin_lb.setup();
typed_struct_lb.setup();

View File

@ -278,8 +278,6 @@ export class MetadataPrincipal implements PrincipalRule {
}
}
export type RbacAction = 'ALLOW' | 'DENY';
export interface UnifiedInfo extends PermissionInfo, PrincipalInfo {}
export class RbacPolicy {
@ -301,15 +299,20 @@ export class RbacPolicy {
}
export class RbacPolicyGroup {
constructor(private policies: Map<string, RbacPolicy>, private action: RbacAction) {}
constructor(private policies: Map<string, RbacPolicy>, private allow: boolean) {}
apply(info: UnifiedInfo): RbacAction | null {
/**
*
* @param info
* @returns True if the call should be accepted, false if it should be rejected
*/
apply(info: UnifiedInfo): boolean {
for (const policy of this.policies.values()) {
if (policy.matches(info)) {
return this.action;
return this.allow;
}
}
return null;
return !this.allow;
}
toString() {
@ -318,7 +321,7 @@ export class RbacPolicyGroup {
policyStrings.push(`${name}: ${policy.toString()}`);
}
return `RBAC
action=${this.action}
action=${this.allow ? 'ALLOW' : 'DENY'}
policies:
${policyStrings.join('\n')}`;
}
@ -392,5 +395,5 @@ export function parseConfig(rbac: RBAC__Output): RbacPolicyGroup {
for (const [name, policyConfig] of Object.entries(rbac.policies)) {
policyMap.set(name, parsePolicy(policyConfig));
}
return new RbacPolicyGroup(policyMap, rbac.action);
return new RbacPolicyGroup(policyMap, rbac.action === 'ALLOW');
}

View File

@ -31,6 +31,7 @@ import { ClusterConfig } from "../src/generated/envoy/extensions/clusters/aggreg
import { Any } from "../src/generated/google/protobuf/Any";
import { ControlPlaneServer } from "./xds-server";
import { UpstreamTlsContext } from "../src/generated/envoy/extensions/transport_sockets/tls/v3/UpstreamTlsContext";
import { HttpFilter } from "../src/generated/envoy/extensions/filters/network/http_connection_manager/v3/HttpFilter";
interface Endpoint {
locality: Locality;
@ -400,7 +401,7 @@ const DEFAULT_BASE_SERVER_ROUTE_CONFIG: RouteConfiguration = {
export class FakeServerRoute {
private listener: Listener;
private routeConfiguration: RouteConfiguration;
constructor(port: number, routeName: string, baseListener?: Listener | undefined, baseRouteConfiguration?: RouteConfiguration) {
constructor(port: number, routeName: string, baseListener?: Listener | undefined, baseRouteConfiguration?: RouteConfiguration | undefined, httpFilters?: HttpFilter[]) {
this.listener = baseListener ?? {...DEFAULT_BASE_SERVER_LISTENER};
this.listener.name = `[::1]:${port}`;
this.listener.address = {
@ -414,11 +415,9 @@ export class FakeServerRoute {
rds: {
route_config_name: routeName,
config_source: {ads: {}}
}
},
http_filters: httpFilters ?? []
};
this.listener.api_listener = {
api_listener: httpConnectionManager
}
const filterList = [{
typed_config: httpConnectionManager
}];

View File

@ -0,0 +1,188 @@
/*
* Copyright 2025 gRPC authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
import * as assert from 'assert';
import { createBackends } from "./backend";
import { XdsTestClient } from "./client";
import { FakeEdsCluster, FakeRouteGroup, FakeServerRoute } from "./framework";
import { ControlPlaneServer } from "./xds-server";
import { AnyExtension } from '@grpc/proto-loader';
import { RBAC } from '../src/generated/envoy/extensions/filters/http/rbac/v3/RBAC';
import { status } from '@grpc/grpc-js';
describe.only('RBAC HTTP filter', () => {
let xdsServer: ControlPlaneServer;
let client: XdsTestClient;
beforeEach(done => {
xdsServer = new ControlPlaneServer();
xdsServer.startServer(error => {
done(error);
});
});
afterEach(() => {
client?.close();
xdsServer?.shutdownServer();
});
it('Should accept matching requests with ALLOW action', async () => {
const [backend] = await createBackends(1);
const rbacFilter: AnyExtension & RBAC = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC',
rules: {
action: 'ALLOW',
policies: {
local: {
principals: [{any: true}],
permissions: [{any: true}]
}
}
}
};
const routerFilter: AnyExtension = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.router.v3.Router'
};
const serverRoute = new FakeServerRoute(backend.getPort(), 'serverRoute', undefined, undefined, [{typed_config: rbacFilter, name: 'rbac'}, {typed_config: routerFilter, name: 'router'}]);
xdsServer.setRdsResource(serverRoute.getRouteConfiguration());
xdsServer.setLdsResource(serverRoute.getListener());
xdsServer.addResponseListener((typeUrl, responseState) => {
if (responseState.state === 'NACKED') {
client?.stopCalls();
assert.fail(`Client NACKED ${typeUrl} resource with message ${responseState.errorMessage}`);
}
});
const cluster = new FakeEdsCluster('cluster1', 'endpoint1', [{backends: [backend], locality:{region: 'region1'}}]);
const routeGroup = new FakeRouteGroup('listener1', 'route1', [{cluster: cluster}]);
await routeGroup.startAllBackends(xdsServer);
xdsServer.setEdsResource(cluster.getEndpointConfig());
xdsServer.setCdsResource(cluster.getClusterConfig());
xdsServer.setRdsResource(routeGroup.getRouteConfiguration());
xdsServer.setLdsResource(routeGroup.getListener());
client = XdsTestClient.createFromServer('listener1', xdsServer);
const error = await client.sendOneCallAsync();
assert.strictEqual(error, null);
});
it('Should reject matching requests with DENY action', async () => {
const [backend] = await createBackends(1);
const rbacFilter: AnyExtension & RBAC = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC',
rules: {
action: 'DENY',
policies: {
local: {
principals: [{any: true}],
permissions: [{any: true}]
}
}
}
};
const routerFilter: AnyExtension = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.router.v3.Router'
};
const serverRoute = new FakeServerRoute(backend.getPort(), 'serverRoute', undefined, undefined, [{typed_config: rbacFilter, name: 'rbac'}, {typed_config: routerFilter, name: 'router'}]);
xdsServer.setRdsResource(serverRoute.getRouteConfiguration());
xdsServer.setLdsResource(serverRoute.getListener());
xdsServer.addResponseListener((typeUrl, responseState) => {
if (responseState.state === 'NACKED') {
client?.stopCalls();
assert.fail(`Client NACKED ${typeUrl} resource with message ${responseState.errorMessage}`);
}
});
const cluster = new FakeEdsCluster('cluster1', 'endpoint1', [{backends: [backend], locality:{region: 'region1'}}]);
const routeGroup = new FakeRouteGroup('listener1', 'route1', [{cluster: cluster}]);
await routeGroup.startAllBackends(xdsServer);
xdsServer.setEdsResource(cluster.getEndpointConfig());
xdsServer.setCdsResource(cluster.getClusterConfig());
xdsServer.setRdsResource(routeGroup.getRouteConfiguration());
xdsServer.setLdsResource(routeGroup.getListener());
client = XdsTestClient.createFromServer('listener1', xdsServer);
const error = await client.sendOneCallAsync();
assert.strictEqual(error?.code, status.PERMISSION_DENIED);
});
it('Should reject non-matching requests with ALLOW action', async () => {
const [backend] = await createBackends(1);
const rbacFilter: AnyExtension & RBAC = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC',
rules: {
action: 'ALLOW',
policies: {
local: {
principals: [{any: true}],
permissions: [{not_rule: {any: true}}]
}
}
}
};
const routerFilter: AnyExtension = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.router.v3.Router'
};
const serverRoute = new FakeServerRoute(backend.getPort(), 'serverRoute', undefined, undefined, [{typed_config: rbacFilter, name: 'rbac'}, {typed_config: routerFilter, name: 'router'}]);
xdsServer.setRdsResource(serverRoute.getRouteConfiguration());
xdsServer.setLdsResource(serverRoute.getListener());
xdsServer.addResponseListener((typeUrl, responseState) => {
if (responseState.state === 'NACKED') {
client?.stopCalls();
assert.fail(`Client NACKED ${typeUrl} resource with message ${responseState.errorMessage}`);
}
});
const cluster = new FakeEdsCluster('cluster1', 'endpoint1', [{backends: [backend], locality:{region: 'region1'}}]);
const routeGroup = new FakeRouteGroup('listener1', 'route1', [{cluster: cluster}]);
await routeGroup.startAllBackends(xdsServer);
xdsServer.setEdsResource(cluster.getEndpointConfig());
xdsServer.setCdsResource(cluster.getClusterConfig());
xdsServer.setRdsResource(routeGroup.getRouteConfiguration());
xdsServer.setLdsResource(routeGroup.getListener());
client = XdsTestClient.createFromServer('listener1', xdsServer);
const error = await client.sendOneCallAsync();
assert.strictEqual(error?.code, status.PERMISSION_DENIED);
});
it('Should accept non-matching requests with DENY action', async () => {
const [backend] = await createBackends(1);
const rbacFilter: AnyExtension & RBAC = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.rbac.v3.RBAC',
rules: {
action: 'DENY',
policies: {
local: {
principals: [{any: true}],
permissions: [{not_rule: {any: true}}]
}
}
}
};
const routerFilter: AnyExtension = {
'@type': 'type.googleapis.com/envoy.extensions.filters.http.router.v3.Router'
};
const serverRoute = new FakeServerRoute(backend.getPort(), 'serverRoute', undefined, undefined, [{typed_config: rbacFilter, name: 'rbac'}, {typed_config: routerFilter, name: 'router'}]);
xdsServer.setRdsResource(serverRoute.getRouteConfiguration());
xdsServer.setLdsResource(serverRoute.getListener());
xdsServer.addResponseListener((typeUrl, responseState) => {
if (responseState.state === 'NACKED') {
client?.stopCalls();
assert.fail(`Client NACKED ${typeUrl} resource with message ${responseState.errorMessage}`);
}
});
const cluster = new FakeEdsCluster('cluster1', 'endpoint1', [{backends: [backend], locality:{region: 'region1'}}]);
const routeGroup = new FakeRouteGroup('listener1', 'route1', [{cluster: cluster}]);
await routeGroup.startAllBackends(xdsServer);
xdsServer.setEdsResource(cluster.getEndpointConfig());
xdsServer.setCdsResource(cluster.getClusterConfig());
xdsServer.setRdsResource(routeGroup.getRouteConfiguration());
xdsServer.setLdsResource(routeGroup.getListener());
client = XdsTestClient.createFromServer('listener1', xdsServer);
const error = await client.sendOneCallAsync();
assert.strictEqual(error, null);
});
});

View File

@ -54,7 +54,9 @@ const loadedProtos = loadPackageDefinition(loadSync(
'envoy/extensions/load_balancing_policies/ring_hash/v3/ring_hash.proto',
'envoy/extensions/load_balancing_policies/pick_first/v3/pick_first.proto',
'envoy/extensions/transport_sockets/tls/v3/tls.proto',
'xds/type/v3/typed_struct.proto'
'xds/type/v3/typed_struct.proto',
'envoy/extensions/filters/http/router/v3/router.proto',
'envoy/extensions/filters/http/rbac/v3/rbac.proto'
],
{
keepCase: true,

View File

@ -19,7 +19,7 @@ import * as http2 from 'http2';
import { log } from './logging';
import { LogVerbosity } from './constants';
import { getErrorMessage } from './error';
const LEGAL_KEY_REGEX = /^[0-9a-z_.-]+$/;
const LEGAL_KEY_REGEX = /^[:0-9a-z_.-]+$/;
const LEGAL_NON_BINARY_VALUE_REGEX = /^[ -~]*$/;
export type MetadataValue = string | Buffer;
@ -222,6 +222,9 @@ export class Metadata {
const result: http2.OutgoingHttpHeaders = {};
for (const [key, values] of this.internalRepr) {
if (key.startsWith(':')) {
continue;
}
// We assume that the user's interaction with this object is limited to
// through its public API (i.e. keys and values are already validated).
result[key] = values.map(bufToString);

View File

@ -1005,9 +1005,10 @@ export class BaseServerInterceptingCall
}
getAuthContext(): AuthContext {
if (this.stream.session?.socket instanceof TLSSocket) {
const peerCertificate = this.stream.session.socket.getPeerCertificate();
return {
transportSecurityType: 'ssl',
sslPeerCertificate: this.stream.session.socket.getPeerCertificate()
sslPeerCertificate: peerCertificate.raw ? peerCertificate : undefined
}
} else {
return {};