diff --git a/build.gradle b/build.gradle index 7ce8bb4de..a0d6fe243 100644 --- a/build.gradle +++ b/build.gradle @@ -46,8 +46,9 @@ allprojects { apply plugin: 'kotlin-spring' project.ext.versions = [ + kotlin : '1.3.72', - java_controlplane : '0.1.24-optimization-SNAPSHOT', + java_controlplane : '0.1.24', spring_boot : '2.3.4.RELEASE', grpc : '1.21.0', jaxb : '2.3.1', diff --git a/docs/features/load_balancing.md b/docs/features/load_balancing.md index 85cded581..e733ccd7e 100644 --- a/docs/features/load_balancing.md +++ b/docs/features/load_balancing.md @@ -70,7 +70,7 @@ egress http connection manager in Envoy static config: key: canary type: STRING remove: false - - name: envoy.router + - name: envoy.filters.http.router ``` The `envoy.filters.http.header_to_metadata` should be added before diff --git a/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/GroupCacheStatusInfo.java b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/GroupCacheStatusInfo.java new file mode 100644 index 000000000..de714b084 --- /dev/null +++ b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/GroupCacheStatusInfo.java @@ -0,0 +1,44 @@ +package pl.allegro.tech.servicemesh.envoycontrol; + +import io.envoyproxy.controlplane.cache.CacheStatusInfo; +import io.envoyproxy.controlplane.cache.StatusInfo; +import java.util.ArrayList; +import java.util.Collection; +import javax.annotation.concurrent.ThreadSafe; + +/** + * {@code GroupCacheStatusInfo} provides an implementation of {@link StatusInfo} for a group of {@link CacheStatusInfo}. + * This class is copy of {@link io.envoyproxy.controlplane.cache.GroupCacheStatusInfo} + */ +@ThreadSafe +class GroupCacheStatusInfo implements StatusInfo { + private final Collection> statuses; + + public GroupCacheStatusInfo(Collection> statuses) { + this.statuses = new ArrayList<>(statuses); + } + + /** + * {@inheritDoc} + */ + @Override + public long lastWatchRequestTime() { + return statuses.stream().mapToLong(CacheStatusInfo::lastWatchRequestTime).max().orElse(0); + } + + /** + * {@inheritDoc} + */ + @Override + public T nodeGroup() { + return statuses.stream().map(CacheStatusInfo::nodeGroup).findFirst().orElse(null); + } + + /** + * {@inheritDoc} + */ + @Override + public int numWatches() { + return statuses.stream().mapToInt(CacheStatusInfo::numWatches).sum(); + } +} \ No newline at end of file diff --git a/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCache.java b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCache.java index 644804c74..09a1c59b1 100644 --- a/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCache.java +++ b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCache.java @@ -1,6 +1,7 @@ package pl.allegro.tech.servicemesh.envoycontrol; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import com.google.protobuf.Message; @@ -13,8 +14,8 @@ import io.envoyproxy.controlplane.cache.StatusInfo; import io.envoyproxy.controlplane.cache.Watch; import io.envoyproxy.controlplane.cache.WatchCancelledException; -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment; -import io.envoyproxy.envoy.api.v2.DiscoveryRequest; +import io.envoyproxy.controlplane.cache.XdsRequest; +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,10 +35,12 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static io.envoyproxy.controlplane.cache.Resources.RESOURCE_TYPES_IN_ORDER; + /** * This class is copy of {@link io.envoyproxy.controlplane.cache.SimpleCache} */ -public class SimpleCache implements SnapshotCache { +public class SimpleCache implements SnapshotCache { private static final Logger LOGGER = LoggerFactory.getLogger(SimpleCache.class); @@ -49,8 +52,8 @@ public class SimpleCache implements SnapshotCache { private final Lock writeLock = lock.writeLock(); @GuardedBy("lock") - private final Map snapshots = new HashMap<>(); - private final ConcurrentMap> statuses = new ConcurrentHashMap<>(); + private final Map snapshots = new HashMap<>(); + private final ConcurrentMap>> statuses = new ConcurrentHashMap<>(); private AtomicLong watchCount = new AtomicLong(); @@ -73,10 +76,10 @@ public boolean clearSnapshot(T group) { // we take a writeLock to prevent watches from being created writeLock.lock(); try { - CacheStatusInfo status = statuses.get(group); + Map> status = statuses.get(group); // If we don't know about this group, do nothing. - if (status != null && status.numWatches() > 0) { + if (status != null && status.values().stream().mapToLong(CacheStatusInfo::numWatches).sum() > 0) { LOGGER.warn("tried to clear snapshot for group with existing watches, group={}", group); return false; @@ -97,21 +100,30 @@ public boolean clearSnapshot(T group) { @Override public Watch createWatch( boolean ads, - DiscoveryRequest request, + XdsRequest request, Set knownResourceNames, Consumer responseConsumer, boolean hasClusterChanged) { + Resources.ResourceType requestResourceType = request.getResourceType(); + Preconditions.checkNotNull(requestResourceType, "unsupported type URL %s", + request.getTypeUrl()); + T group; + if (request.v3Request() != null) { + group = groups.hash(request.v3Request().getNode()); + } else { + group = groups.hash(request.v2Request().getNode()); + } - T group = groups.hash(request.getNode()); // even though we're modifying, we take a readLock to allow multiple watches to be created in parallel since it // doesn't conflict readLock.lock(); try { - CacheStatusInfo status = statuses.computeIfAbsent(group, g -> new CacheStatusInfo<>(group)); + CacheStatusInfo status = statuses.computeIfAbsent(group, g -> new ConcurrentHashMap<>()) + .computeIfAbsent(requestResourceType, s -> new CacheStatusInfo<>(group)); status.setLastWatchRequestTime(System.currentTimeMillis()); - Snapshot snapshot = snapshots.get(group); - String version = snapshot == null ? "" : snapshot.version(request.getTypeUrl(), request.getResourceNamesList()); + U snapshot = snapshots.get(group); + String version = snapshot == null ? "" : snapshot.version(requestResourceType, request.getResourceNamesList()); Watch watch = new Watch(ads, request, responseConsumer); @@ -124,7 +136,7 @@ public Watch createWatch( // If any of the newly requested resources are in the snapshot respond immediately. If not we'll fall back to // version comparisons. - if (snapshot.resources(request.getTypeUrl()) + if (snapshot.resources(requestResourceType) .keySet() .stream() .anyMatch(newResourceHints::contains)) { @@ -132,7 +144,7 @@ public Watch createWatch( return watch; } - } else if (hasClusterChanged && request.getTypeUrl().equals(Resources.ENDPOINT_TYPE_URL)) { + } else if (hasClusterChanged && requestResourceType.equals(Resources.ResourceType.ENDPOINT)) { respond(watch, snapshot, group); return watch; @@ -187,9 +199,10 @@ public Watch createWatch( /** * {@inheritDoc} + * @return */ @Override - public Snapshot getSnapshot(T group) { + public U getSnapshot(T group) { readLock.lock(); try { @@ -214,9 +227,9 @@ public Collection groups() { * It can be called concurrently for different groups. */ @Override - public void setSnapshot(T group, Snapshot snapshot) { + public void setSnapshot(T group, U snapshot) { // we take a writeLock to prevent watches from being created while we update the snapshot - CacheStatusInfo status; + ConcurrentMap> status; writeLock.lock(); try { // Update the existing snapshot entry. @@ -242,20 +255,27 @@ public StatusInfo statusInfo(T group) { readLock.lock(); try { - return statuses.get(group); + ConcurrentMap> statusMap = statuses.get(group); + if (statusMap == null || statusMap.isEmpty()) { + return null; + } + + return new GroupCacheStatusInfo<>(statusMap.values()); } finally { readLock.unlock(); } } @VisibleForTesting - protected void respondWithSpecificOrder(T group, Snapshot snapshot, CacheStatusInfo status) { - for (String typeUrl : Resources.TYPE_URLS) { + protected void respondWithSpecificOrder(T group, U snapshot, ConcurrentMap> statusMap) { + for (Resources.ResourceType resourceType : RESOURCE_TYPES_IN_ORDER) { + CacheStatusInfo status = statusMap.get(resourceType); + if (status == null) continue; // todo: why this happens? status.watchesRemoveIf((id, watch) -> { - if (!watch.request().getTypeUrl().equals(typeUrl)) { + if (!watch.request().getResourceType().equals(resourceType)) { return false; } - String version = snapshot.version(watch.request().getTypeUrl(), watch.request().getResourceNamesList()); + String version = snapshot.version(watch.request().getResourceType(), watch.request().getResourceNamesList()); if (!watch.request().getVersionInfo().equals(version)) { if (LOGGER.isDebugEnabled()) { @@ -277,7 +297,7 @@ protected void respondWithSpecificOrder(T group, Snapshot snapshot, CacheStatusI } } - private Response createResponse(DiscoveryRequest request, Map resources, String version) { + private Response createResponse(XdsRequest request, Map resources, String version) { Collection filtered = request.getResourceNamesList().isEmpty() ? resources.values() : request.getResourceNamesList().stream() @@ -288,8 +308,8 @@ private Response createResponse(DiscoveryRequest request, Map snapshotResources = snapshot.resources(watch.request().getTypeUrl()); + private boolean respond(Watch watch, U snapshot, T group) { + Map snapshotResources = snapshot.resources(watch.request().getResourceType()); Map snapshotForMissingResources = Collections.emptyMap(); if (!watch.request().getResourceNamesList().isEmpty() && watch.ads()) { @@ -309,12 +329,12 @@ private boolean respond(Watch watch, Snapshot snapshot, T group) { // If shouldSendMissingEndpoints is set to true, we will respond to such request anyway, to prevent // such problems with Envoy. if (shouldSendMissingEndpoints - && watch.request().getTypeUrl().equals(Resources.ENDPOINT_TYPE_URL)) { + && watch.request().getResourceType().equals(Resources.ResourceType.ENDPOINT)) { LOGGER.info("adding missing resources [{}] to response for {} in ADS mode from node {} at version {}", String.join(", ", missingNames), watch.request().getTypeUrl(), group, - snapshot.version(watch.request().getTypeUrl(), watch.request().getResourceNamesList()) + snapshot.version(watch.request().getResourceType(), watch.request().getResourceNamesList()) ); snapshotForMissingResources = new HashMap<>(missingNames.size()); for (String missingName : missingNames) { @@ -328,7 +348,7 @@ private boolean respond(Watch watch, Snapshot snapshot, T group) { "not responding in ADS mode for {} from node {} at version {} for request [{}] since [{}] not in snapshot", watch.request().getTypeUrl(), group, - snapshot.version(watch.request().getTypeUrl(), watch.request().getResourceNamesList()), + snapshot.version(watch.request().getResourceType(), watch.request().getResourceNamesList()), String.join(", ", watch.request().getResourceNamesList()), String.join(", ", missingNames)); @@ -337,7 +357,7 @@ private boolean respond(Watch watch, Snapshot snapshot, T group) { } } - String version = snapshot.version(watch.request().getTypeUrl(), watch.request().getResourceNamesList()); + String version = snapshot.version(watch.request().getResourceType(), watch.request().getResourceNamesList()); LOGGER.debug("responding for {} from node {} at version {} with version {}", watch.request().getTypeUrl(), diff --git a/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCache.java b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCache.java new file mode 100644 index 000000000..84837aceb --- /dev/null +++ b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCache.java @@ -0,0 +1,10 @@ +package pl.allegro.tech.servicemesh.envoycontrol.v2; + +import io.envoyproxy.controlplane.cache.NodeGroup; +import io.envoyproxy.controlplane.cache.v2.Snapshot; + +public class SimpleCache extends pl.allegro.tech.servicemesh.envoycontrol.SimpleCache { + public SimpleCache(NodeGroup nodeGroup, Boolean shouldSendMissingEndpoints) { + super(nodeGroup, shouldSendMissingEndpoints); + } +} \ No newline at end of file diff --git a/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCache.java b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCache.java new file mode 100644 index 000000000..46e3a62e3 --- /dev/null +++ b/envoy-control-core/src/main/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCache.java @@ -0,0 +1,10 @@ +package pl.allegro.tech.servicemesh.envoycontrol.v3; + +import io.envoyproxy.controlplane.cache.NodeGroup; +import io.envoyproxy.controlplane.cache.v3.Snapshot; + +public class SimpleCache extends pl.allegro.tech.servicemesh.envoycontrol.SimpleCache { + public SimpleCache(NodeGroup nodeGroup, Boolean shouldSendMissingEndpoints) { + super(nodeGroup, shouldSendMissingEndpoints); + } +} diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ControlPlane.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ControlPlane.kt index 4502d10d1..cc7bfdb57 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ControlPlane.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ControlPlane.kt @@ -2,9 +2,11 @@ package pl.allegro.tech.servicemesh.envoycontrol import io.envoyproxy.controlplane.cache.NodeGroup import io.envoyproxy.controlplane.cache.SnapshotCache +import io.envoyproxy.controlplane.cache.v3.Snapshot import io.envoyproxy.controlplane.server.DefaultExecutorGroup -import io.envoyproxy.controlplane.server.DiscoveryServer import io.envoyproxy.controlplane.server.ExecutorGroup +import io.envoyproxy.controlplane.server.V2DiscoveryServer +import io.envoyproxy.controlplane.server.V3DiscoveryServer import io.envoyproxy.controlplane.server.callback.SnapshotCollectingCallback import io.grpc.Server import io.grpc.netty.NettyServerBuilder @@ -16,6 +18,7 @@ import pl.allegro.tech.servicemesh.envoycontrol.groups.GroupChangeWatcher import pl.allegro.tech.servicemesh.envoycontrol.groups.MetadataNodeGroup import pl.allegro.tech.servicemesh.envoycontrol.groups.NodeMetadataValidator import pl.allegro.tech.servicemesh.envoycontrol.server.CachedProtoResourcesSerializer +import pl.allegro.tech.servicemesh.envoycontrol.server.ExecutorProperties import pl.allegro.tech.servicemesh.envoycontrol.server.ExecutorType import pl.allegro.tech.servicemesh.envoycontrol.server.ServerProperties import pl.allegro.tech.servicemesh.envoycontrol.server.callbacks.CompositeDiscoveryServerCallbacks @@ -29,12 +32,14 @@ import pl.allegro.tech.servicemesh.envoycontrol.snapshot.EnvoySnapshotFactory import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotUpdater import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotsVersions import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.endpoints.EnvoyEndpointsFactory -import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.EnvoyListenersFactory import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters.AccessLogFilterFactory import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters.EnvoyHttpFilters +import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.EnvoyListenersFactory import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes.ServiceTagMetadataGenerator import pl.allegro.tech.servicemesh.envoycontrol.utils.DirectScheduler import pl.allegro.tech.servicemesh.envoycontrol.utils.ParallelScheduler +import pl.allegro.tech.servicemesh.envoycontrol.utils.ParallelizableScheduler +import pl.allegro.tech.servicemesh.envoycontrol.v3.SimpleCache import reactor.core.Disposable import reactor.core.publisher.Flux import reactor.core.scheduler.Schedulers @@ -53,7 +58,7 @@ class ControlPlane private constructor( val grpcServer: Server, val snapshotUpdater: SnapshotUpdater, val nodeGroup: NodeGroup, - val cache: SnapshotCache, + val cache: SnapshotCache, private val changes: Flux ) : AutoCloseable { @@ -100,13 +105,7 @@ class ControlPlane private constructor( fun build(changes: Flux): ControlPlane { if (grpcServerExecutor == null) { - grpcServerExecutor = newMeteredThreadPoolExecutor( - properties.server.serverPoolSize, - properties.server.serverPoolSize, - properties.server.serverPoolKeepAlive.toMillis(), - LinkedBlockingQueue(), - "grpc-server-worker" - ) + grpcServerExecutor = buildThreadPoolExecutor() } if (nioEventLoopExecutor == null) { @@ -116,20 +115,7 @@ class ControlPlane private constructor( } if (executorGroup == null) { - executorGroup = when (properties.server.executorGroup.type) { - ExecutorType.DIRECT -> DefaultExecutorGroup() - ExecutorType.PARALLEL -> { - // TODO(https://github.com/allegro/envoy-control/issues/103) this implementation of parallel - // executor group is invalid, because it may lead to sending XDS responses out of order for - // given DiscoveryRequestStreamObserver. We should switch to multiple, single-threaded - // ThreadPoolExecutors. More info in linked task. - val executor = newMeteredFixedThreadPool( - "discovery-responses-executor", - properties.server.executorGroup.parallelPoolSize - ) - ExecutorGroup { executor } - } - } + executorGroup = buildExecutorGroup() } if (globalSnapshotExecutor == null) { @@ -141,54 +127,52 @@ class ControlPlane private constructor( val groupSnapshotProperties = properties.server.groupSnapshotUpdateScheduler - val groupSnapshotScheduler = when (groupSnapshotProperties.type) { - ExecutorType.DIRECT -> DirectScheduler - ExecutorType.PARALLEL -> ParallelScheduler( - scheduler = Schedulers.fromExecutor( - groupSnapshotParallelExecutorSupplier() - ?: newMeteredFixedThreadPool( - "group-snapshot", - groupSnapshotProperties.parallelPoolSize - ) - ), - parallelism = groupSnapshotProperties.parallelPoolSize - ) - } - + val groupSnapshotScheduler = buildGroupSnapshotScheduler(groupSnapshotProperties) val cache = SimpleCache(nodeGroup, properties.envoy.snapshot.shouldSendMissingEndpoints) - - val cleanupProperties = properties.server.snapshotCleanup - val groupChangeWatcher = GroupChangeWatcher(cache, metrics, meterRegistry) - - val discoveryServer = DiscoveryServer( - listOf( + val meteredConnectionsCallbacks = MeteredConnectionsCallbacks().also { + meterRegistry.gauge("grpc.all-connections", it.connections) + MeteredConnectionsCallbacks.MetricsStreamType.values().map { type -> + meterRegistry.gauge("grpc.connections.${type.name.toLowerCase()}", it.connections(type)) + } + } + val loggingDiscoveryServerCallbacks = LoggingDiscoveryServerCallbacks( + properties.server.logFullRequest, + properties.server.logFullResponse + ) + val snapshotCollectingCallback = buildSnapshotCollectingCallback(cache) + val cachedProtoResourcesSerializer = CachedProtoResourcesSerializer( + meterRegistry, + properties.server.reportProtobufCacheMetrics + ) + val compositeDiscoveryCallbacksV2 = listOf( CompositeDiscoveryServerCallbacks( - meterRegistry, - SnapshotCollectingCallback( - cache, - nodeGroup, - Clock.systemDefaultZone(), - emptySet(), - cleanupProperties.collectAfterMillis.toMillis(), - cleanupProperties.collectionIntervalMillis.toMillis() - ), - LoggingDiscoveryServerCallbacks( - properties.server.logFullRequest, - properties.server.logFullResponse - ), - MeteredConnectionsCallbacks().also { - meterRegistry.gauge("grpc.all-connections", it.connections) - MeteredConnectionsCallbacks.MetricsStreamType.values().map { type -> - meterRegistry.gauge("grpc.connections.${type.name.toLowerCase()}", it.connections(type)) - } - }, - NodeMetadataValidator(properties.envoy.snapshot) + meterRegistry, + snapshotCollectingCallback, + loggingDiscoveryServerCallbacks, + meteredConnectionsCallbacks, + NodeMetadataValidator(properties.envoy.snapshot) ) - ), + ) + val compositeDiscoveryCallbacksV3 = listOf( + CompositeDiscoveryServerCallbacks( + meterRegistry, + loggingDiscoveryServerCallbacks, + meteredConnectionsCallbacks, + NodeMetadataValidator(properties.envoy.snapshot) + ) + ) + val v2discoveryServer = V2DiscoveryServer( + compositeDiscoveryCallbacksV2, groupChangeWatcher, executorGroup, - CachedProtoResourcesSerializer(meterRegistry, properties.server.reportProtobufCacheMetrics) + cachedProtoResourcesSerializer + ) + val v3discoveryServer = V3DiscoveryServer( + compositeDiscoveryCallbacksV3, + groupChangeWatcher, + executorGroup, + cachedProtoResourcesSerializer ) val snapshotsVersions = SnapshotsVersions() @@ -211,7 +195,13 @@ class ControlPlane private constructor( ) return ControlPlane( - grpcServer(properties.server, discoveryServer, nioEventLoopExecutor!!, grpcServerExecutor!!), + grpcServer( + properties.server, + v2discoveryServer, + v3discoveryServer, + nioEventLoopExecutor!!, + grpcServerExecutor!! + ), SnapshotUpdater( cache, properties.envoy.snapshot, @@ -229,6 +219,63 @@ class ControlPlane private constructor( ) } + private fun buildSnapshotCollectingCallback( + cache: SimpleCache + ): SnapshotCollectingCallback { + val cleanupProperties = properties.server.snapshotCleanup + return SnapshotCollectingCallback( + cache, + nodeGroup, + Clock.systemDefaultZone(), + emptySet(), + cleanupProperties.collectAfterMillis.toMillis(), + cleanupProperties.collectionIntervalMillis.toMillis() + ) + } + + private fun buildGroupSnapshotScheduler(groupSnapshotProperties: ExecutorProperties): ParallelizableScheduler { + return when (groupSnapshotProperties.type) { + ExecutorType.DIRECT -> DirectScheduler + ExecutorType.PARALLEL -> ParallelScheduler( + scheduler = Schedulers.fromExecutor( + groupSnapshotParallelExecutorSupplier() + ?: newMeteredFixedThreadPool( + "group-snapshot", + groupSnapshotProperties.parallelPoolSize + ) + ), + parallelism = groupSnapshotProperties.parallelPoolSize + ) + } + } + + private fun buildExecutorGroup(): ExecutorGroup? { + return when (properties.server.executorGroup.type) { + ExecutorType.DIRECT -> DefaultExecutorGroup() + ExecutorType.PARALLEL -> { + // TODO(https://github.com/allegro/envoy-control/issues/103) this implementation of parallel + // executor group is invalid, because it may lead to sending XDS responses out of order for + // given DiscoveryRequestStreamObserver. We should switch to multiple, single-threaded + // ThreadPoolExecutors. More info in linked task. + val executor = newMeteredFixedThreadPool( + "discovery-responses-executor", + properties.server.executorGroup.parallelPoolSize + ) + ExecutorGroup { executor } + } + } + } + + private fun buildThreadPoolExecutor(): ThreadPoolExecutor { + return newMeteredThreadPoolExecutor( + properties.server.serverPoolSize, + properties.server.serverPoolSize, + properties.server.serverPoolKeepAlive.toMillis(), + LinkedBlockingQueue(), + "grpc-server-worker" + ) + } + fun withNodeGroup(nodeGroup: NodeGroup): ControlPlaneBuilder { this.nodeGroup = nodeGroup return this @@ -269,12 +316,21 @@ class ControlPlane private constructor( return this } - private fun NettyServerBuilder.withEnvoyServices(discoveryServer: DiscoveryServer): NettyServerBuilder = - this.addService(discoveryServer.aggregatedDiscoveryServiceImpl) + private fun NettyServerBuilder.withV2EnvoyServices(discoveryServer: V2DiscoveryServer): NettyServerBuilder { + return this.addService(discoveryServer.aggregatedDiscoveryServiceImpl) + .addService(discoveryServer.clusterDiscoveryServiceImpl) + .addService(discoveryServer.endpointDiscoveryServiceImpl) + .addService(discoveryServer.listenerDiscoveryServiceImpl) + .addService(discoveryServer.routeDiscoveryServiceImpl) + } + + private fun NettyServerBuilder.withV3EnvoyServices(discoveryServer: V3DiscoveryServer): NettyServerBuilder { + return this.addService(discoveryServer.aggregatedDiscoveryServiceImpl) .addService(discoveryServer.clusterDiscoveryServiceImpl) .addService(discoveryServer.endpointDiscoveryServiceImpl) .addService(discoveryServer.listenerDiscoveryServiceImpl) .addService(discoveryServer.routeDiscoveryServiceImpl) + } private class ThreadNamingThreadFactory(val threadNamePrefix: String) : ThreadFactory { private val counter = AtomicInteger() @@ -319,7 +375,8 @@ class ControlPlane private constructor( private fun grpcServer( config: ServerProperties, - discoveryServer: DiscoveryServer, + v2discoveryServer: V2DiscoveryServer, + v3discoveryServer: V3DiscoveryServer, nioEventLoopExecutor: Executor, grpcServerExecutor: Executor ): Server = NettyServerBuilder.forPort(config.port) @@ -333,7 +390,8 @@ class ControlPlane private constructor( .keepAliveTime(config.netty.keepAliveTime.toMillis(), TimeUnit.MILLISECONDS) .permitKeepAliveTime(config.netty.permitKeepAliveTime.toMillis(), TimeUnit.MILLISECONDS) .permitKeepAliveWithoutCalls(config.netty.permitKeepAliveWithoutCalls) - .withEnvoyServices(discoveryServer) + .withV2EnvoyServices(v2discoveryServer) + .withV3EnvoyServices(v3discoveryServer) .build() } } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/GroupChangeWatcher.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/GroupChangeWatcher.kt index d130b5e1f..12ba07f11 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/GroupChangeWatcher.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/GroupChangeWatcher.kt @@ -3,15 +3,15 @@ package pl.allegro.tech.servicemesh.envoycontrol.groups import io.envoyproxy.controlplane.cache.ConfigWatcher import io.envoyproxy.controlplane.cache.Response import io.envoyproxy.controlplane.cache.Watch -import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.controlplane.cache.XdsRequest import io.micrometer.core.instrument.MeterRegistry import pl.allegro.tech.servicemesh.envoycontrol.EnvoyControlMetrics -import pl.allegro.tech.servicemesh.envoycontrol.SimpleCache import pl.allegro.tech.servicemesh.envoycontrol.logger import pl.allegro.tech.servicemesh.envoycontrol.utils.measureBuffer import reactor.core.publisher.Flux import reactor.core.publisher.FluxSink import java.util.function.Consumer +import pl.allegro.tech.servicemesh.envoycontrol.v3.SimpleCache as SimpleCache /** * This class is needed to force snapshot creation in SnapshotUpdater when new group is added. @@ -41,12 +41,13 @@ internal class GroupChangeWatcher( override fun createWatch( ads: Boolean, - request: DiscoveryRequest, + request: XdsRequest, knownResourceNames: MutableSet, responseConsumer: Consumer, hasClusterChanged: Boolean ): Watch { val oldGroups = cache.groups() + val watch = cache.createWatch(ads, request, knownResourceNames, responseConsumer, hasClusterChanged) val groups = cache.groups() metrics.setCacheGroupsCount(groups.size) diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/Groups.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/Groups.kt index 94c05a29e..c4953be1e 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/Groups.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/Groups.kt @@ -5,20 +5,27 @@ sealed class Group { abstract val serviceName: String abstract val proxySettings: ProxySettings abstract val listenersConfig: ListenersConfig? + abstract val version: ResourceVersion +} + +enum class ResourceVersion { + V2, V3 } data class ServicesGroup( override val communicationMode: CommunicationMode, override val serviceName: String = "", override val proxySettings: ProxySettings = ProxySettings(), - override val listenersConfig: ListenersConfig? = null + override val listenersConfig: ListenersConfig? = null, + override val version: ResourceVersion = ResourceVersion.V2 ) : Group() data class AllServicesGroup( override val communicationMode: CommunicationMode, override val serviceName: String = "", override val proxySettings: ProxySettings = ProxySettings(), - override val listenersConfig: ListenersConfig? = null + override val listenersConfig: ListenersConfig? = null, + override val version: ResourceVersion = ResourceVersion.V2 ) : Group() data class ListenersConfig( diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroup.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroup.kt index a6bc5b4fa..53e328cc3 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroup.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroup.kt @@ -3,11 +3,11 @@ package pl.allegro.tech.servicemesh.envoycontrol.groups import com.google.protobuf.Struct import com.google.protobuf.Value import io.envoyproxy.controlplane.cache.NodeGroup -import io.envoyproxy.envoy.api.v2.core.Node - import pl.allegro.tech.servicemesh.envoycontrol.logger import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters.AccessLogFilterFactory +import io.envoyproxy.envoy.api.v2.core.Node as NodeV2 +import io.envoyproxy.envoy.config.core.v3.Node as NodeV3 class MetadataNodeGroup( val properties: SnapshotProperties, @@ -15,7 +15,13 @@ class MetadataNodeGroup( ) : NodeGroup { private val logger by logger() - override fun hash(node: Node): Group = createGroup(node) + override fun hash(node: NodeV2): Group { + return createV2Group(node) + } + + override fun hash(node: NodeV3): Group { + return createV3Group(node) + } @SuppressWarnings("ReturnCount") private fun metadataToListenersHostPort( @@ -121,26 +127,37 @@ class MetadataNodeGroup( ) } - private fun createGroup(node: Node): Group { + private fun createV3Group(node: NodeV3): Group { val metadata = NodeMetadata(node.metadata, properties) - val serviceName = serviceName(metadata) - val proxySettings = proxySettings(metadata) - val listenersConfig = createListenersConfig(node.id, node.metadata) + return createGroup(metadata, node.id, node.metadata, ResourceVersion.V3) + } + + private fun createV2Group(node: NodeV2): Group { + val metadata = NodeMetadata(node.metadata, properties) + return createGroup(metadata, node.id, node.metadata, ResourceVersion.V2) + } + + private fun createGroup(nodeMetadata: NodeMetadata, id: String, metadata: Struct, version: ResourceVersion): Group { + val serviceName = serviceName(nodeMetadata) + val proxySettings = proxySettings(nodeMetadata) + val listenersConfig = createListenersConfig(id, metadata) return when { - hasAllServicesDependencies(metadata) -> + hasAllServicesDependencies(nodeMetadata) -> AllServicesGroup( - metadata.communicationMode, + nodeMetadata.communicationMode, serviceName, proxySettings, - listenersConfig + listenersConfig, + version ) else -> ServicesGroup( - metadata.communicationMode, + nodeMetadata.communicationMode, serviceName, proxySettings, - listenersConfig + listenersConfig, + version ) } } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadata.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadata.kt index 2baa7243f..4fbac8e2f 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadata.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadata.kt @@ -5,7 +5,7 @@ import com.google.protobuf.Struct import com.google.protobuf.Value import com.google.protobuf.util.Durations import io.envoyproxy.controlplane.server.exception.RequestException -import io.envoyproxy.envoy.config.filter.accesslog.v2.ComparisonFilter +import io.envoyproxy.envoy.config.accesslog.v3.ComparisonFilter import io.grpc.Status import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters.AccessLogFilterFactory diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidator.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidator.kt index a1201eed4..a049edd87 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidator.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidator.kt @@ -2,6 +2,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.groups import io.envoyproxy.controlplane.server.DiscoveryServerCallbacks import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest as v3DiscoveryRequest import io.envoyproxy.envoy.api.v2.DiscoveryResponse import io.envoyproxy.envoy.api.v2.core.Node import pl.allegro.tech.servicemesh.envoycontrol.protocol.HttpMethod @@ -52,8 +53,12 @@ class NodeMetadataValidator( override fun onStreamOpen(streamId: Long, typeUrl: String?) {} - override fun onStreamRequest(streamId: Long, request: DiscoveryRequest?) { - request?.node?.let { validateMetadata(it) } + override fun onV3StreamRequest(streamId: Long, request: v3DiscoveryRequest?) { + request?.node?.let { validateV3Metadata(it) } + } + + override fun onV2StreamRequest(streamId: Long, request: DiscoveryRequest?) { + request?.node?.let { validateV2Metadata(it) } } override fun onStreamResponse( @@ -63,11 +68,23 @@ class NodeMetadataValidator( ) { } - private fun validateMetadata(node: Node) { + private fun validateV3Metadata(node: io.envoyproxy.envoy.config.core.v3.Node) { + // Some validation logic is executed when NodeMetadata is created. + // This may throw NodeMetadataValidationException + val metadata = NodeMetadata(node.metadata, properties) + + validateMetadata(metadata) + } + + private fun validateV2Metadata(node: Node) { // Some validation logic is executed when NodeMetadata is created. // This may throw NodeMetadataValidationException val metadata = NodeMetadata(node.metadata, properties) + validateMetadata(metadata) + } + + private fun validateMetadata(metadata: NodeMetadata) { validateDependencies(metadata) validateIncomingEndpoints(metadata) validateConfigurationMode(metadata) diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/CachedProtoResourcesSerializer.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/CachedProtoResourcesSerializer.kt index 75601c866..d84e4f652 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/CachedProtoResourcesSerializer.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/CachedProtoResourcesSerializer.kt @@ -4,52 +4,82 @@ import com.google.common.cache.Cache import com.google.common.cache.CacheBuilder import com.google.protobuf.Any import com.google.protobuf.Message -import io.envoyproxy.controlplane.server.serializer.ProtoResourcesSerializer +import io.envoyproxy.controlplane.cache.Resources import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.binder.cache.GuavaCacheMetrics import pl.allegro.tech.servicemesh.envoycontrol.utils.noopTimer import java.util.function.Supplier +import io.envoyproxy.controlplane.cache.Resources.ApiVersion.V2 +import io.envoyproxy.controlplane.cache.Resources.ApiVersion.V3 +import io.envoyproxy.controlplane.server.serializer.DefaultProtoResourcesSerializer +import io.micrometer.core.instrument.Timer + internal class CachedProtoResourcesSerializer( - meterRegistry: MeterRegistry, - reportMetrics: Boolean -) : ProtoResourcesSerializer { - - private val serializeTimer = if (reportMetrics) { - meterRegistry.timer("protobuf-cache.serialize.time") - } else { - noopTimer + private val meterRegistry: MeterRegistry, + private val reportMetrics: Boolean +) : DefaultProtoResourcesSerializer() { + + private fun createTimer(reportMetrics: Boolean, meterRegistry: MeterRegistry, timerName: String): Timer { + return if (reportMetrics) { + meterRegistry.timer(timerName) + } else { + noopTimer + } } - private val cache: Cache, MutableCollection> = if (reportMetrics) { - GuavaCacheMetrics - .monitor( - meterRegistry, - CacheBuilder.newBuilder() - .recordStats() + private val cacheV2: Cache, MutableCollection> = createCache("protobuf-cache-v2") + private val cacheV3: Cache, MutableCollection> = createCache("protobuf-cache-v3") + private val v2Timer = createTimer(reportMetrics, meterRegistry, "protobuf-cache-v2.serialize.time") + private val v3Timer = createTimer(reportMetrics, meterRegistry, "protobuf-cache-v3.serialize.time") + + private fun createCache(cacheName: String): Cache, MutableCollection> { + return if (reportMetrics) { + GuavaCacheMetrics + .monitor( + meterRegistry, + CacheBuilder.newBuilder() + .recordStats() + .weakValues() + .build, MutableCollection>(), + cacheName + ) + } else { + CacheBuilder.newBuilder() .weakValues() - .build, MutableCollection>(), - "protobuf-cache" - ) - } else { - CacheBuilder.newBuilder() - .weakValues() - .build, MutableCollection>() + .build, MutableCollection>() + } } - override fun serialize(resources: MutableCollection): MutableCollection = serializeTimer - .record(Supplier { getResources(resources) }) + override fun serialize( + resources: MutableCollection, + apiVersion: Resources.ApiVersion + ): MutableCollection { + return if (apiVersion == V2) { + v2Timer.record(Supplier { getResources(resources, apiVersion) }) + } else { + v3Timer.record(Supplier { getResources(resources, apiVersion) }) + } + } + + private fun getResources( + resources: MutableCollection, + apiVersion: Resources.ApiVersion + ): MutableCollection { + val cache = when (apiVersion) { + V2 -> cacheV2 + V3 -> cacheV3 + } - private fun getResources(resources: MutableCollection): MutableCollection { return cache.get(resources) { resources.asSequence() - .map { Any.pack(it) } + .map { super.maybeRewriteTypeUrl(Any.pack(it), apiVersion) } .toMutableList() } } @Suppress("NotImplementedDeclaration") - override fun serialize(resource: Message?): Any { + override fun serialize(resource: Message?, apiVersion: Resources.ApiVersion?): Any { throw NotImplementedError("Serializing single messages is not supported") } } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/CompositeDiscoveryServerCallbacks.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/CompositeDiscoveryServerCallbacks.kt index 2d8e9de09..ff043c991 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/CompositeDiscoveryServerCallbacks.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/CompositeDiscoveryServerCallbacks.kt @@ -3,6 +3,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.server.callbacks import io.envoyproxy.controlplane.server.DiscoveryServerCallbacks import io.envoyproxy.controlplane.server.exception.RequestException import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest as v3DiscoveryRequest import io.envoyproxy.envoy.api.v2.DiscoveryResponse import io.micrometer.core.instrument.MeterRegistry import pl.allegro.tech.servicemesh.envoycontrol.logger @@ -17,19 +18,33 @@ class CompositeDiscoveryServerCallbacks( private val logger by logger() override fun onStreamCloseWithError(streamId: Long, typeUrl: String?, error: Throwable?) { - runCallbacks { it.onStreamCloseWithError(streamId, typeUrl, error) } + runCallbacks { + it.onStreamCloseWithError(streamId, typeUrl, error) + } } override fun onStreamClose(streamId: Long, typeUrl: String?) { - runCallbacks { it.onStreamClose(streamId, typeUrl) } + runCallbacks { + it.onStreamClose(streamId, typeUrl) + } } override fun onStreamOpen(streamId: Long, typeUrl: String?) { - runCallbacks { it.onStreamOpen(streamId, typeUrl) } + runCallbacks { + it.onStreamOpen(streamId, typeUrl) + } + } + + override fun onV2StreamRequest(streamId: Long, request: DiscoveryRequest?) { + runCallbacks { + it.onV2StreamRequest(streamId, request) + } } - override fun onStreamRequest(streamId: Long, request: DiscoveryRequest?) { - runCallbacks { it.onStreamRequest(streamId, request) } + override fun onV3StreamRequest(streamId: Long, request: v3DiscoveryRequest?) { + runCallbacks { + it.onV3StreamRequest(streamId, request) + } } override fun onStreamResponse( @@ -37,7 +52,9 @@ class CompositeDiscoveryServerCallbacks( request: DiscoveryRequest?, response: DiscoveryResponse? ) { - runCallbacks { it.onStreamResponse(streamId, request, response) } + runCallbacks { + it.onStreamResponse(streamId, request, response) + } } private fun runCallbacks(fn: (DiscoveryServerCallbacks) -> Unit) { diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/LoggingDiscoveryServerCallbacks.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/LoggingDiscoveryServerCallbacks.kt index 9d758be48..7f68b16e0 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/LoggingDiscoveryServerCallbacks.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/LoggingDiscoveryServerCallbacks.kt @@ -2,6 +2,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.server.callbacks import io.envoyproxy.controlplane.server.DiscoveryServerCallbacks import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest as v3DiscoveryRequest import io.envoyproxy.envoy.api.v2.DiscoveryResponse import org.slf4j.LoggerFactory @@ -15,6 +16,10 @@ class LoggingDiscoveryServerCallbacks( logger.debug("onStreamClose streamId: {} typeUrl: {}", streamId, typeUrl) } + override fun onV3StreamRequest(streamId: Long, request: v3DiscoveryRequest?) { + logger.debug("onV3StreamRequest streamId: {} request: {}", streamId, requestData(request)) + } + override fun onStreamCloseWithError(streamId: Long, typeUrl: String?, error: Throwable?) { logger.debug("onStreamCloseWithError streamId: {}, typeUrl: {}", streamId, typeUrl, error) } @@ -23,8 +28,8 @@ class LoggingDiscoveryServerCallbacks( logger.debug("onStreamOpen streamId: {}, typeUrl: {}", streamId, typeUrl) } - override fun onStreamRequest(streamId: Long, request: DiscoveryRequest?) { - logger.debug("onStreamRequest streamId: {} request: {}", streamId, requestData(request)) + override fun onV2StreamRequest(streamId: Long, request: DiscoveryRequest?) { + logger.debug("onV2StreamRequest streamId: {} request: {}", streamId, requestData(request)) } override fun onStreamResponse( @@ -56,4 +61,13 @@ class LoggingDiscoveryServerCallbacks( "type: ${request?.typeUrl}, responseNonce: ${request?.responseNonce}" } } + + private fun requestData(request: io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest?): String { + return if (logFullRequest) { + "$request" + } else { + "version: ${request?.versionInfo}, id: ${request?.node?.id}, cluster: ${request?.node?.cluster}, " + + "type: ${request?.typeUrl}, responseNonce: ${request?.responseNonce}" + } + } } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/MeteredConnectionsCallbacks.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/MeteredConnectionsCallbacks.kt index 3ce889442..cca054de2 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/MeteredConnectionsCallbacks.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/server/callbacks/MeteredConnectionsCallbacks.kt @@ -2,8 +2,11 @@ package pl.allegro.tech.servicemesh.envoycontrol.server.callbacks import io.envoyproxy.controlplane.cache.Resources import io.envoyproxy.controlplane.server.DiscoveryServerCallbacks +import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest as v3DiscoveryRequest import java.util.concurrent.atomic.AtomicInteger +@Suppress("EmptyFunctionBlock") class MeteredConnectionsCallbacks( val connections: AtomicInteger = AtomicInteger() ) : DiscoveryServerCallbacks { @@ -30,6 +33,12 @@ class MeteredConnectionsCallbacks( connectionsByType(typeUrl).decrementAndGet() } + override fun onV3StreamRequest(streamId: Long, request: v3DiscoveryRequest?) { + } + + override fun onV2StreamRequest(p0: Long, p1: DiscoveryRequest?) { + } + override fun onStreamCloseWithError(streamId: Long, typeUrl: String?, error: Throwable?) { connections.decrementAndGet() connectionsByType(typeUrl).decrementAndGet() @@ -37,13 +46,19 @@ class MeteredConnectionsCallbacks( fun connections(type: MetricsStreamType): AtomicInteger = connectionsByType[type]!! + @Suppress("ComplexMethod") private fun connectionsByType(typeUrl: String?): AtomicInteger { val type = when (typeUrl) { - Resources.CLUSTER_TYPE_URL -> MetricsStreamType.CDS - Resources.ENDPOINT_TYPE_URL -> MetricsStreamType.EDS - Resources.LISTENER_TYPE_URL -> MetricsStreamType.LDS - Resources.ROUTE_TYPE_URL -> MetricsStreamType.RDS - Resources.SECRET_TYPE_URL -> MetricsStreamType.SDS + Resources.V2.CLUSTER_TYPE_URL -> MetricsStreamType.CDS + Resources.V2.ENDPOINT_TYPE_URL -> MetricsStreamType.EDS + Resources.V2.LISTENER_TYPE_URL -> MetricsStreamType.LDS + Resources.V2.ROUTE_TYPE_URL -> MetricsStreamType.RDS + Resources.V2.SECRET_TYPE_URL -> MetricsStreamType.SDS + Resources.V3.CLUSTER_TYPE_URL -> MetricsStreamType.CDS + Resources.V3.ENDPOINT_TYPE_URL -> MetricsStreamType.EDS + Resources.V3.LISTENER_TYPE_URL -> MetricsStreamType.LDS + Resources.V3.ROUTE_TYPE_URL -> MetricsStreamType.RDS + Resources.V3.SECRET_TYPE_URL -> MetricsStreamType.SDS "" -> MetricsStreamType.ADS // ads is when the type url is empty else -> MetricsStreamType.UNKNOWN } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/EnvoySnapshotFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/EnvoySnapshotFactory.kt index 895508dcf..3739c1ea6 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/EnvoySnapshotFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/EnvoySnapshotFactory.kt @@ -1,11 +1,11 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot -import io.envoyproxy.controlplane.cache.Snapshot -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.Listener -import io.envoyproxy.envoy.api.v2.RouteConfiguration -import io.envoyproxy.envoy.api.v2.auth.Secret +import io.envoyproxy.controlplane.cache.v3.Snapshot +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.listener.v3.Listener +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.Secret import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.Timer import pl.allegro.tech.servicemesh.envoycontrol.groups.AllServicesGroup @@ -42,6 +42,8 @@ class EnvoySnapshotFactory( val clusters = clustersFactory.getClustersForServices(clusterConfigurations.values, communicationMode) val securedClusters = clustersFactory.getSecuredClusters(clusters) + val v3Clusters = clustersFactory.mapToV3Cluster(clusters) + val v3SecuredClusters = clustersFactory.mapToV3Cluster(clusters) val endpoints: List = endpointsFactory.createLoadAssignment( clusters = clusterConfigurations.keys, @@ -53,7 +55,9 @@ class EnvoySnapshotFactory( clusters = clusters, securedClusters = securedClusters, endpoints = endpoints, - properties = properties.outgoingPermissions + properties = properties.outgoingPermissions, + v3Clusters = v3Clusters, + v3SecuredClusters = v3SecuredClusters ) sample.stop(meterRegistry.timer("snapshot-factory.new-snapshot.time")) diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/GlobalSnapshot.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/GlobalSnapshot.kt index a231d2787..8492c4aa9 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/GlobalSnapshot.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/GlobalSnapshot.kt @@ -1,26 +1,33 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot import io.envoyproxy.controlplane.cache.SnapshotResources -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment data class GlobalSnapshot( val clusters: SnapshotResources, val allServicesNames: Set, val endpoints: SnapshotResources, val clusterConfigurations: Map, - val securedClusters: SnapshotResources + val securedClusters: SnapshotResources, + val v3Clusters: SnapshotResources, + val v3SecuredClusters: SnapshotResources ) +@Suppress("LongParameterList") internal fun globalSnapshot( clusters: Iterable, endpoints: Iterable, properties: OutgoingPermissionsProperties = OutgoingPermissionsProperties(), clusterConfigurations: Map, - securedClusters: List + securedClusters: List, + v3Clusters: List, + v3SecuredClusters: List ): GlobalSnapshot { val clusters = SnapshotResources.create(clusters, "") val securedClusters = SnapshotResources.create(securedClusters, "") + val v3Clusters = SnapshotResources.create(v3Clusters, "") + val v3SecuredClusters = SnapshotResources.create(v3SecuredClusters, "") val allServicesNames = getClustersForAllServicesGroups(clusters.resources(), properties) val endpoints = SnapshotResources.create(endpoints, "") return GlobalSnapshot( @@ -28,7 +35,9 @@ internal fun globalSnapshot( securedClusters = securedClusters, endpoints = endpoints, allServicesNames = allServicesNames, - clusterConfigurations = clusterConfigurations + clusterConfigurations = clusterConfigurations, + v3Clusters = v3Clusters, + v3SecuredClusters = v3SecuredClusters ) } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotProperties.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotProperties.kt index e765ae2e2..0c0e4ddc6 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotProperties.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotProperties.kt @@ -2,8 +2,8 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.auth.TlsParameters +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.TlsParameters import pl.allegro.tech.servicemesh.envoycontrol.groups.PathMatchingType import java.time.Duration diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdater.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdater.kt index 3ae33a575..78211e910 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdater.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdater.kt @@ -1,6 +1,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot import io.envoyproxy.controlplane.cache.SnapshotCache +import io.envoyproxy.controlplane.cache.v3.Snapshot import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.Timer import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.ADS @@ -20,7 +21,7 @@ import reactor.core.publisher.Mono import reactor.core.scheduler.Scheduler class SnapshotUpdater( - private val cache: SnapshotCache, + private val cache: SnapshotCache, private val properties: SnapshotProperties, private val snapshotFactory: EnvoySnapshotFactory, private val globalSnapshotScheduler: Scheduler, diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersions.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersions.kt index 7dab586a5..8889a16d7 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersions.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersions.kt @@ -1,8 +1,8 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.Listener +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.listener.v3.Listener import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotsVersions.Companion.newVersion import java.util.UUID diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/clusters/EnvoyClustersFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/clusters/EnvoyClustersFactory.kt index 332b7a804..75dd87251 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/clusters/EnvoyClustersFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/clusters/EnvoyClustersFactory.kt @@ -5,35 +5,37 @@ import com.google.protobuf.Struct import com.google.protobuf.UInt32Value import com.google.protobuf.Value import com.google.protobuf.util.Durations -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.auth.CertificateValidationContext -import io.envoyproxy.envoy.api.v2.auth.CommonTlsContext -import io.envoyproxy.envoy.api.v2.auth.SdsSecretConfig -import io.envoyproxy.envoy.api.v2.auth.TlsParameters -import io.envoyproxy.envoy.api.v2.auth.UpstreamTlsContext -import io.envoyproxy.envoy.api.v2.cluster.CircuitBreakers -import io.envoyproxy.envoy.api.v2.cluster.OutlierDetection -import io.envoyproxy.envoy.api.v2.core.Address -import io.envoyproxy.envoy.api.v2.core.AggregatedConfigSource -import io.envoyproxy.envoy.api.v2.core.ApiConfigSource -import io.envoyproxy.envoy.api.v2.core.ConfigSource -import io.envoyproxy.envoy.api.v2.core.DataSource -import io.envoyproxy.envoy.api.v2.core.GrpcService -import io.envoyproxy.envoy.api.v2.core.Http2ProtocolOptions -import io.envoyproxy.envoy.api.v2.core.HttpProtocolOptions -import io.envoyproxy.envoy.api.v2.core.RoutingPriority -import io.envoyproxy.envoy.api.v2.core.SocketAddress -import io.envoyproxy.envoy.api.v2.core.TransportSocket -import io.envoyproxy.envoy.api.v2.core.UpstreamHttpProtocolOptions -import io.envoyproxy.envoy.api.v2.endpoint.Endpoint -import io.envoyproxy.envoy.api.v2.endpoint.LbEndpoint -import io.envoyproxy.envoy.api.v2.endpoint.LocalityLbEndpoints +import io.envoyproxy.envoy.config.cluster.v3.CircuitBreakers +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.cluster.v3.OutlierDetection +import io.envoyproxy.envoy.config.core.v3.Address +import io.envoyproxy.envoy.config.core.v3.AggregatedConfigSource +import io.envoyproxy.envoy.config.core.v3.ApiConfigSource +import io.envoyproxy.envoy.config.core.v3.ApiVersion +import io.envoyproxy.envoy.config.core.v3.ConfigSource +import io.envoyproxy.envoy.config.core.v3.DataSource +import io.envoyproxy.envoy.config.core.v3.GrpcService +import io.envoyproxy.envoy.config.core.v3.Http2ProtocolOptions +import io.envoyproxy.envoy.config.core.v3.HttpProtocolOptions +import io.envoyproxy.envoy.config.core.v3.RoutingPriority +import io.envoyproxy.envoy.config.core.v3.SocketAddress +import io.envoyproxy.envoy.config.core.v3.TransportSocket +import io.envoyproxy.envoy.config.core.v3.UpstreamHttpProtocolOptions +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.endpoint.v3.Endpoint +import io.envoyproxy.envoy.config.endpoint.v3.LbEndpoint +import io.envoyproxy.envoy.config.endpoint.v3.LocalityLbEndpoints +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.CertificateValidationContext +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.CommonTlsContext +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.SdsSecretConfig +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.TlsParameters +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.UpstreamTlsContext import pl.allegro.tech.servicemesh.envoycontrol.groups.AllServicesGroup import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.ADS import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.XDS import pl.allegro.tech.servicemesh.envoycontrol.groups.Group +import pl.allegro.tech.servicemesh.envoycontrol.groups.ResourceVersion import pl.allegro.tech.servicemesh.envoycontrol.groups.ServicesGroup import pl.allegro.tech.servicemesh.envoycontrol.snapshot.ClusterConfiguration import pl.allegro.tech.servicemesh.envoycontrol.snapshot.GlobalSnapshot @@ -45,7 +47,7 @@ class EnvoyClustersFactory( private val properties: SnapshotProperties ) { private val httpProtocolOptions: HttpProtocolOptions = HttpProtocolOptions.newBuilder().setIdleTimeout( - Durations.fromMillis(properties.egress.commonHttp.idleTimeout.toMillis()) + Durations.fromMillis(properties.egress.commonHttp.idleTimeout.toMillis()) ).build() private val thresholds: List = mapPropertiesToThresholds() @@ -53,15 +55,15 @@ class EnvoyClustersFactory( private val tlsProperties = properties.incomingPermissions.tlsAuthentication private val sanUriMatcher = SanUriMatcherFactory(tlsProperties) private val matchPlaintextContext = Cluster.TransportSocketMatch.newBuilder() - .setName("plaintext_match") - .setTransportSocket( - TransportSocket.newBuilder().setName("envoy.transport_sockets.raw_buffer").build() - ) - .build() + .setName("plaintext_match") + .setTransportSocket( + TransportSocket.newBuilder().setName("envoy.transport_sockets.raw_buffer").build() + ) + .build() private val tlsContextMatch = Struct.newBuilder() - .putFields(tlsProperties.tlsContextMetadataMatchKey, Value.newBuilder().setBoolValue(true).build()) - .build() + .putFields(tlsProperties.tlsContextMetadataMatchKey, Value.newBuilder().setBoolValue(true).build()) + .build() fun getClustersForServices( services: Collection, @@ -70,21 +72,27 @@ class EnvoyClustersFactory( return services.map { edsCluster(it, communicationMode) } } + fun mapToV3Cluster( + clusters: List + ): List { + return clusters.map { mapClusterToV3(it) } + } + fun getSecuredClusters(insecureClusters: List): List { return insecureClusters.map { cluster -> val upstreamTlsContext = createTlsContextWithSdsSecretConfig(cluster.name) val secureCluster = Cluster.newBuilder(cluster) val matchTlsContext = Cluster.TransportSocketMatch.newBuilder() - .setName("mtls_match") - .setMatch(tlsContextMatch) - .setTransportSocket(TransportSocket.newBuilder() - .setName("envoy.transport_sockets.tls") - .setTypedConfig(Any.pack(upstreamTlsContext))) - .build() + .setName("mtls_match") + .setMatch(tlsContextMatch) + .setTransportSocket(TransportSocket.newBuilder() + .setName("envoy.transport_sockets.tls") + .setTypedConfig(Any.pack(upstreamTlsContext))) + .build() secureCluster.addAllTransportSocketMatches(listOf(matchTlsContext, matchPlaintextContext)) - .build() + .build() } } @@ -92,53 +100,75 @@ class EnvoyClustersFactory( getEdsClustersForGroup(group, globalSnapshot) + getStrictDnsClustersForGroup(group) private fun getEdsClustersForGroup(group: Group, globalSnapshot: GlobalSnapshot): List { - val clusters = if (enableTlsForGroup(group)) { - globalSnapshot.securedClusters.resources() + val clusters: Map = if (enableTlsForGroup(group)) { + if (group.version == ResourceVersion.V3) { + globalSnapshot.v3SecuredClusters.resources() + } else { + globalSnapshot.securedClusters.resources() + } } else { - globalSnapshot.clusters.resources() + if (group.version == ResourceVersion.V3) { + globalSnapshot.v3Clusters.resources() + } else { + globalSnapshot.clusters.resources() + } } return when (group) { is ServicesGroup -> group.proxySettings.outgoing.getServiceDependencies().mapNotNull { - clusters.get(it.service) + clusters[it.service] } is AllServicesGroup -> globalSnapshot.allServicesNames.mapNotNull { - clusters.get(it) + clusters[it] } } } + private fun mapClusterToV3(cluster: Cluster): Cluster { + return cluster.let { + val v3Cluster = Cluster.newBuilder(it) + val v3EdsClusterConfig = Cluster.EdsClusterConfig.newBuilder(v3Cluster.edsClusterConfig) + + v3Cluster.setEdsClusterConfig( + v3EdsClusterConfig.setEdsConfig( + ConfigSource.newBuilder(v3EdsClusterConfig.edsConfig).setResourceApiVersion(ApiVersion.V3) + ) + ).build() + } + } + private fun enableTlsForGroup(group: Group): Boolean { return group.listenersConfig?.hasStaticSecretsDefined ?: false } private val commonTlsParams = TlsParameters.newBuilder() - .setTlsMinimumProtocolVersion(tlsProperties.protocol.minimumVersion) - .setTlsMaximumProtocolVersion(tlsProperties.protocol.maximumVersion) - .addAllCipherSuites(tlsProperties.protocol.cipherSuites) - .build() + .setTlsMinimumProtocolVersion(tlsProperties.protocol.minimumVersion) + .setTlsMaximumProtocolVersion(tlsProperties.protocol.maximumVersion) + .addAllCipherSuites(tlsProperties.protocol.cipherSuites) + .build() private val validationContextSecretConfig = SdsSecretConfig.newBuilder() - .setName(tlsProperties.validationContextSecretName).build() + .setName(tlsProperties.validationContextSecretName).build() private val tlsCertificateSecretConfig = SdsSecretConfig.newBuilder() - .setName(tlsProperties.tlsCertificateSecretName).build() + .setName(tlsProperties.tlsCertificateSecretName).build() private fun createTlsContextWithSdsSecretConfig(serviceName: String): UpstreamTlsContext { val sanMatch = sanUriMatcher.createSanUriMatcher(serviceName) return UpstreamTlsContext.newBuilder() - .setCommonTlsContext(CommonTlsContext.newBuilder() - .setTlsParams(commonTlsParams) - .setCombinedValidationContext(CommonTlsContext.CombinedCertificateValidationContext.newBuilder() - .setDefaultValidationContext(CertificateValidationContext.newBuilder() - .addAllMatchSubjectAltNames(listOf(sanMatch)).build()) - .setValidationContextSdsSecretConfig(validationContextSecretConfig) - .build() - ) - .addTlsCertificateSdsSecretConfigs(tlsCertificateSecretConfig) - .build() + .setCommonTlsContext(CommonTlsContext.newBuilder() + .setTlsParams(commonTlsParams) + .setCombinedValidationContext(CommonTlsContext.CombinedCertificateValidationContext.newBuilder() + .setDefaultValidationContext(CertificateValidationContext.newBuilder() + .addAllMatchSubjectAltNames(listOf(sanMatch)) + .build()) + .setValidationContextSdsSecretConfig(validationContextSecretConfig) + .build() ) + .addTlsCertificateSdsSecretConfigs(tlsCertificateSecretConfig) .build() + ) + .build() } private fun getStrictDnsClustersForGroup(group: Group): List { @@ -181,31 +211,26 @@ class EnvoyClustersFactory( if (ssl) { val commonTlsContext = CommonTlsContext.newBuilder() - .setValidationContext( - CertificateValidationContext.newBuilder() - .setTrustedCa( - // TODO: https://github.com/allegro/envoy-control/issues/5 - DataSource.newBuilder().setFilename(properties.trustedCaFile).build() - ).build() - ).build() - - val upstreamTlsContext = UpstreamTlsContext.newBuilder().setCommonTlsContext(commonTlsContext) - // for envoy >= 1.14.0-dev it will be overridden by setAutoSni below - // TODO(https://github.com/allegro/envoy-control/issues/97) - // remove when envoy < 1.14.0-dev will be not supported - .setSni(host) - .build() + .setValidationContext( + CertificateValidationContext.newBuilder() + .setTrustedCa( + // TODO: https://github.com/allegro/envoy-control/issues/5 + DataSource.newBuilder().setFilename(properties.trustedCaFile).build() + ).build() + ).build() + + val upstreamTlsContext = UpstreamTlsContext.newBuilder().setCommonTlsContext(commonTlsContext).build() val transportSocket = TransportSocket.newBuilder() - .setTypedConfig(Any.pack( - upstreamTlsContext - )) - .setName("envoy.transport_sockets.tls").build() + .setTypedConfig(Any.pack( + upstreamTlsContext + )) + .setName("envoy.transport_sockets.tls").build() clusterBuilder - .setTransportSocket(transportSocket) - .setUpstreamHttpProtocolOptions( - UpstreamHttpProtocolOptions.newBuilder().setAutoSanValidation(true).setAutoSni(true).build() - ) + .setTransportSocket(transportSocket) + .setUpstreamHttpProtocolOptions( + UpstreamHttpProtocolOptions.newBuilder().setAutoSanValidation(true).setAutoSni(true).build() + ) } return clusterBuilder.build() @@ -228,16 +253,17 @@ class EnvoyClustersFactory( .setEdsClusterConfig( Cluster.EdsClusterConfig.newBuilder().setEdsConfig( when (communicationMode) { + // here we do not have group information ADS -> ConfigSource.newBuilder().setAds(AggregatedConfigSource.newBuilder()) XDS -> - ConfigSource.newBuilder().setApiConfigSource( - ApiConfigSource.newBuilder().setApiType(ApiConfigSource.ApiType.GRPC) - .addGrpcServices(0, GrpcService.newBuilder().setEnvoyGrpc( - GrpcService.EnvoyGrpc.newBuilder() - .setClusterName(properties.xdsClusterName) - ) + ConfigSource.newBuilder().setApiConfigSource( + ApiConfigSource.newBuilder().setApiType(ApiConfigSource.ApiType.GRPC) + .addGrpcServices(0, GrpcService.newBuilder().setEnvoyGrpc( + GrpcService.EnvoyGrpc.newBuilder() + .setClusterName(properties.xdsClusterName) + ) + ) ) - ) } ).setServiceName(clusterConfiguration.serviceName) ) diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/endpoints/EnvoyEndpointsFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/endpoints/EnvoyEndpointsFactory.kt index 6abe64b9a..b8d881c91 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/endpoints/EnvoyEndpointsFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/endpoints/EnvoyEndpointsFactory.kt @@ -4,14 +4,14 @@ import com.google.protobuf.ListValue import com.google.protobuf.Struct import com.google.protobuf.UInt32Value import com.google.protobuf.Value -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.core.Address -import io.envoyproxy.envoy.api.v2.core.Locality -import io.envoyproxy.envoy.api.v2.core.Metadata -import io.envoyproxy.envoy.api.v2.core.SocketAddress -import io.envoyproxy.envoy.api.v2.endpoint.Endpoint -import io.envoyproxy.envoy.api.v2.endpoint.LbEndpoint -import io.envoyproxy.envoy.api.v2.endpoint.LocalityLbEndpoints +import io.envoyproxy.envoy.config.core.v3.Address +import io.envoyproxy.envoy.config.core.v3.Metadata +import io.envoyproxy.envoy.config.core.v3.Locality +import io.envoyproxy.envoy.config.core.v3.SocketAddress +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.endpoint.v3.Endpoint +import io.envoyproxy.envoy.config.endpoint.v3.LbEndpoint +import io.envoyproxy.envoy.config.endpoint.v3.LocalityLbEndpoints import pl.allegro.tech.servicemesh.envoycontrol.services.MultiClusterState import pl.allegro.tech.servicemesh.envoycontrol.services.ServiceInstance import pl.allegro.tech.servicemesh.envoycontrol.services.ServiceInstances diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/EnvoyListenersFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/EnvoyListenersFactory.kt index 670be667a..a73cb9eab 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/EnvoyListenersFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/EnvoyListenersFactory.kt @@ -5,38 +5,40 @@ import com.google.protobuf.Duration import com.google.protobuf.Struct import com.google.protobuf.Value import com.google.protobuf.util.Durations -import io.envoyproxy.envoy.api.v2.Listener -import io.envoyproxy.envoy.api.v2.auth.CommonTlsContext -import io.envoyproxy.envoy.api.v2.auth.DownstreamTlsContext -import io.envoyproxy.envoy.api.v2.auth.SdsSecretConfig -import io.envoyproxy.envoy.api.v2.auth.TlsParameters -import io.envoyproxy.envoy.api.v2.core.RuntimeUInt32 -import io.envoyproxy.envoy.api.v2.core.Address -import io.envoyproxy.envoy.api.v2.core.AggregatedConfigSource -import io.envoyproxy.envoy.api.v2.core.ApiConfigSource -import io.envoyproxy.envoy.api.v2.core.ConfigSource -import io.envoyproxy.envoy.api.v2.core.GrpcService -import io.envoyproxy.envoy.api.v2.core.Http1ProtocolOptions -import io.envoyproxy.envoy.api.v2.core.HttpProtocolOptions -import io.envoyproxy.envoy.api.v2.core.SocketAddress -import io.envoyproxy.envoy.api.v2.core.TransportSocket -import io.envoyproxy.envoy.api.v2.listener.Filter -import io.envoyproxy.envoy.api.v2.listener.FilterChain -import io.envoyproxy.envoy.api.v2.listener.FilterChainMatch -import io.envoyproxy.envoy.config.accesslog.v2.FileAccessLog -import io.envoyproxy.envoy.config.filter.accesslog.v2.AccessLog -import io.envoyproxy.envoy.config.filter.accesslog.v2.AccessLogFilter -import io.envoyproxy.envoy.config.filter.accesslog.v2.ComparisonFilter -import io.envoyproxy.envoy.config.filter.accesslog.v2.StatusCodeFilter -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpConnectionManager -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpFilter -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.Rds +import io.envoyproxy.envoy.config.accesslog.v3.AccessLog +import io.envoyproxy.envoy.config.accesslog.v3.AccessLogFilter +import io.envoyproxy.envoy.config.accesslog.v3.ComparisonFilter +import io.envoyproxy.envoy.config.accesslog.v3.StatusCodeFilter +import io.envoyproxy.envoy.config.core.v3.Address +import io.envoyproxy.envoy.config.core.v3.AggregatedConfigSource +import io.envoyproxy.envoy.config.core.v3.ApiConfigSource +import io.envoyproxy.envoy.config.core.v3.ApiVersion +import io.envoyproxy.envoy.config.core.v3.ConfigSource +import io.envoyproxy.envoy.config.core.v3.GrpcService +import io.envoyproxy.envoy.config.core.v3.Http1ProtocolOptions +import io.envoyproxy.envoy.config.core.v3.HttpProtocolOptions +import io.envoyproxy.envoy.config.core.v3.RuntimeUInt32 +import io.envoyproxy.envoy.config.core.v3.SocketAddress +import io.envoyproxy.envoy.config.core.v3.TransportSocket +import io.envoyproxy.envoy.config.listener.v3.Filter +import io.envoyproxy.envoy.config.listener.v3.FilterChain +import io.envoyproxy.envoy.config.listener.v3.FilterChainMatch +import io.envoyproxy.envoy.config.listener.v3.Listener +import io.envoyproxy.envoy.extensions.access_loggers.file.v3.FileAccessLog +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpConnectionManager +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpFilter +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.Rds +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.CommonTlsContext +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.DownstreamTlsContext +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.SdsSecretConfig +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.TlsParameters import pl.allegro.tech.servicemesh.envoycontrol.groups.AccessLogFilterSettings import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.ADS import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.XDS import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.groups.ListenersConfig +import pl.allegro.tech.servicemesh.envoycontrol.groups.ResourceVersion import pl.allegro.tech.servicemesh.envoycontrol.snapshot.GlobalSnapshot import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters.EnvoyHttpFilters @@ -205,7 +207,7 @@ class EnvoyListenersFactory( ): Filter { val connectionManagerBuilder = HttpConnectionManager.newBuilder() .setStatPrefix("egress_http") - .setRds(egressRds(group.communicationMode)) + .setRds(egressRds(group.communicationMode, group.version)) .setHttpProtocolOptions(egressHttp1ProtocolOptions()) addHttpFilters(connectionManagerBuilder, egressFilters, group, globalSnapshot) @@ -239,7 +241,7 @@ class EnvoyListenersFactory( } return Filter.newBuilder() - .setName("envoy.http_connection_manager") + .setName("envoy.filters.network.http_connection_manager") .setTypedConfig(ProtobufAny.pack( connectionManagerBuilder.build() )) @@ -252,10 +254,14 @@ class EnvoyListenersFactory( .build() } - private fun egressRds(communicationMode: CommunicationMode): Rds { + private fun egressRds(communicationMode: CommunicationMode, version: ResourceVersion): Rds { val configSource = ConfigSource.newBuilder() .setInitialFetchTimeout(egressRdsInitialFetchTimeout) + if (version == ResourceVersion.V3) { + configSource.setResourceApiVersion(ApiVersion.V3) + } + when (communicationMode) { ADS -> configSource.setAds(AggregatedConfigSource.getDefaultInstance()) XDS -> configSource.setApiConfigSource(defaultApiConfigSource) @@ -284,7 +290,7 @@ class EnvoyListenersFactory( .setDelayedCloseTimeout(durationInSeconds(0)) .setCommonHttpProtocolOptions(httpProtocolOptions) .setCodecType(HttpConnectionManager.CodecType.AUTO) - .setRds(ingressRds(group.communicationMode)) + .setRds(ingressRds(group.communicationMode, group.version)) .setHttpProtocolOptions(ingressHttp1ProtocolOptions(group.serviceName)) if (listenersConfig.useRemoteAddress) { @@ -305,10 +311,14 @@ class EnvoyListenersFactory( .build() } - private fun ingressRds(communicationMode: CommunicationMode): Rds { + private fun ingressRds(communicationMode: CommunicationMode, version: ResourceVersion): Rds { val configSource = ConfigSource.newBuilder() .setInitialFetchTimeout(ingressRdsInitialFetchTimeout) + if (version == ResourceVersion.V3) { + configSource.setResourceApiVersion(ApiVersion.V3) + } + when (communicationMode) { ADS -> configSource.setAds(AggregatedConfigSource.getDefaultInstance()) XDS -> configSource.setApiConfigSource(defaultApiConfigSource) diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/AccessLogFilterFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/AccessLogFilterFactory.kt index 54353fa4c..4a7e00732 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/AccessLogFilterFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/AccessLogFilterFactory.kt @@ -1,7 +1,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters import com.google.re2j.Pattern -import io.envoyproxy.envoy.config.filter.accesslog.v2.ComparisonFilter +import io.envoyproxy.envoy.config.accesslog.v3.ComparisonFilter import pl.allegro.tech.servicemesh.envoycontrol.groups.AccessLogFilterSettings import pl.allegro.tech.servicemesh.envoycontrol.groups.NodeMetadataValidationException diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyDefaultFilters.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyDefaultFilters.kt index 12b726234..3f1541520 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyDefaultFilters.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyDefaultFilters.kt @@ -1,9 +1,9 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters import com.google.protobuf.Any -import io.envoyproxy.envoy.api.v2.core.Metadata -import io.envoyproxy.envoy.config.filter.http.header_to_metadata.v2.Config -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpFilter +import io.envoyproxy.envoy.config.core.v3.Metadata +import io.envoyproxy.envoy.extensions.filters.http.header_to_metadata.v3.Config +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpFilter import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.snapshot.GlobalSnapshot import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties @@ -75,7 +75,7 @@ class EnvoyDefaultFilters( private fun envoyRouterHttpFilter(): HttpFilter = HttpFilter .newBuilder() - .setName("envoy.router") + .setName("envoy.filters.http.router") .build() private fun headerToMetadataHttpFilter(headerToMetadataConfig: Config.Builder): HttpFilter { diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyHttpFilters.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyHttpFilters.kt index 0e1a0570c..3e7f0d9a5 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyHttpFilters.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/EnvoyHttpFilters.kt @@ -1,6 +1,6 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters -import io.envoyproxy.envoy.api.v2.core.Metadata +import io.envoyproxy.envoy.config.core.v3.Metadata import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.HttpFilterFactory diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/LuaFilterFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/LuaFilterFactory.kt index 2ee127d0b..b615d310c 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/LuaFilterFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/LuaFilterFactory.kt @@ -1,12 +1,12 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters import com.google.protobuf.Any +import io.envoyproxy.envoy.config.core.v3.Metadata +import io.envoyproxy.envoy.extensions.filters.http.lua.v3.Lua +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpFilter import com.google.protobuf.ListValue import com.google.protobuf.Struct import com.google.protobuf.Value -import io.envoyproxy.envoy.api.v2.core.Metadata -import io.envoyproxy.envoy.config.filter.http.lua.v2.Lua -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpFilter import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.snapshot.IncomingPermissionsProperties diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactory.kt index bb3a6aa25..ba24bc521 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactory.kt @@ -2,13 +2,13 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.fil import com.google.protobuf.Any import com.google.protobuf.UInt32Value -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.core.CidrRange -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpFilter -import io.envoyproxy.envoy.config.rbac.v2.Policy -import io.envoyproxy.envoy.config.rbac.v2.Principal -import io.envoyproxy.envoy.config.rbac.v2.RBAC +import io.envoyproxy.envoy.config.core.v3.CidrRange +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.rbac.v3.Policy +import io.envoyproxy.envoy.config.rbac.v3.Principal +import io.envoyproxy.envoy.config.rbac.v3.RBAC +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpFilter import pl.allegro.tech.servicemesh.envoycontrol.groups.ClientWithSelector import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.groups.Incoming @@ -20,7 +20,7 @@ import pl.allegro.tech.servicemesh.envoycontrol.snapshot.GlobalSnapshot import pl.allegro.tech.servicemesh.envoycontrol.snapshot.IncomingPermissionsProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SelectorMatching import pl.allegro.tech.servicemesh.envoycontrol.snapshot.StatusRouteProperties -import io.envoyproxy.envoy.config.filter.http.rbac.v2.RBAC as RBACFilter +import io.envoyproxy.envoy.extensions.filters.http.rbac.v3.RBAC as RBACFilter class RBACFilterFactory( private val incomingPermissionsProperties: IncomingPermissionsProperties, diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissions.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissions.kt index 3b2e03806..fb5ef436d 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissions.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissions.kt @@ -1,10 +1,10 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher -import io.envoyproxy.envoy.config.rbac.v2.Permission -import io.envoyproxy.envoy.type.matcher.PathMatcher -import io.envoyproxy.envoy.type.matcher.RegexMatcher -import io.envoyproxy.envoy.type.matcher.StringMatcher +import io.envoyproxy.envoy.config.rbac.v3.Permission +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher +import io.envoyproxy.envoy.type.matcher.v3.PathMatcher +import io.envoyproxy.envoy.type.matcher.v3.RegexMatcher +import io.envoyproxy.envoy.type.matcher.v3.StringMatcher import pl.allegro.tech.servicemesh.envoycontrol.groups.IncomingEndpoint import pl.allegro.tech.servicemesh.envoycontrol.groups.PathMatchingType diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/SanUriMatcherFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/SanUriMatcherFactory.kt index d557e7794..e83edb75f 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/SanUriMatcherFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/SanUriMatcherFactory.kt @@ -1,7 +1,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters -import io.envoyproxy.envoy.type.matcher.RegexMatcher -import io.envoyproxy.envoy.type.matcher.StringMatcher +import io.envoyproxy.envoy.type.matcher.v3.RegexMatcher +import io.envoyproxy.envoy.type.matcher.v3.StringMatcher import pl.allegro.tech.servicemesh.envoycontrol.snapshot.TlsAuthenticationProperties import java.lang.IllegalArgumentException diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/ServiceTagFilter.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/ServiceTagFilter.kt index 892f6e57d..5c803cac6 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/ServiceTagFilter.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/ServiceTagFilter.kt @@ -1,6 +1,6 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters -import io.envoyproxy.envoy.config.filter.http.header_to_metadata.v2.Config +import io.envoyproxy.envoy.extensions.filters.http.header_to_metadata.v3.Config class ServiceTagFilter private constructor() { companion object { diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AdminRoutesFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AdminRoutesFactory.kt index 7047d65b0..a0f11e2dd 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AdminRoutesFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AdminRoutesFactory.kt @@ -1,12 +1,12 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes -import io.envoyproxy.envoy.api.v2.core.DataSource -import io.envoyproxy.envoy.api.v2.route.DirectResponseAction -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher -import io.envoyproxy.envoy.api.v2.route.RedirectAction -import io.envoyproxy.envoy.api.v2.route.Route -import io.envoyproxy.envoy.api.v2.route.RouteAction -import io.envoyproxy.envoy.api.v2.route.RouteMatch +import io.envoyproxy.envoy.config.core.v3.DataSource +import io.envoyproxy.envoy.config.route.v3.DirectResponseAction +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher +import io.envoyproxy.envoy.config.route.v3.RedirectAction +import io.envoyproxy.envoy.config.route.v3.Route +import io.envoyproxy.envoy.config.route.v3.RouteAction +import io.envoyproxy.envoy.config.route.v3.RouteMatch import pl.allegro.tech.servicemesh.envoycontrol.protocol.HttpMethod import pl.allegro.tech.servicemesh.envoycontrol.snapshot.RoutesProperties diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AuthorizationRoute.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AuthorizationRoute.kt index 59d14153c..2f9f6cd23 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AuthorizationRoute.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/AuthorizationRoute.kt @@ -1,6 +1,6 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes -import io.envoyproxy.envoy.api.v2.route.Route +import io.envoyproxy.envoy.config.route.v3.Route class AuthorizationRoute( val authorized: Route, diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactory.kt index 13b4291ae..77ffc6bab 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactory.kt @@ -2,14 +2,14 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes import com.google.protobuf.util.Durations import io.envoyproxy.controlplane.cache.TestResources -import io.envoyproxy.envoy.api.v2.RouteConfiguration -import io.envoyproxy.envoy.api.v2.core.HeaderValue -import io.envoyproxy.envoy.api.v2.core.HeaderValueOption -import io.envoyproxy.envoy.api.v2.route.DirectResponseAction -import io.envoyproxy.envoy.api.v2.route.Route -import io.envoyproxy.envoy.api.v2.route.RouteAction -import io.envoyproxy.envoy.api.v2.route.RouteMatch -import io.envoyproxy.envoy.api.v2.route.VirtualHost +import io.envoyproxy.envoy.config.core.v3.HeaderValue +import io.envoyproxy.envoy.config.core.v3.HeaderValueOption +import io.envoyproxy.envoy.config.route.v3.DirectResponseAction +import io.envoyproxy.envoy.config.route.v3.Route +import io.envoyproxy.envoy.config.route.v3.RouteAction +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration +import io.envoyproxy.envoy.config.route.v3.RouteMatch +import io.envoyproxy.envoy.config.route.v3.VirtualHost import pl.allegro.tech.servicemesh.envoycontrol.snapshot.RouteSpecification import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties @@ -78,10 +78,11 @@ class EnvoyEgressRoutesFactory( .setMatch( RouteMatch.newBuilder() .setPrefix("/") + .build() ) .setRoute( createRouteAction(routeSpecification) - ) + ).build() ) .build() } @@ -123,7 +124,7 @@ class EnvoyEgressRoutesFactory( } if (properties.egress.hostHeaderRewriting.enabled && routeSpecification.settings.rewriteHostHeader) { - routeAction.autoHostRewriteHeader = properties.egress.hostHeaderRewriting.customHostHeader + routeAction.hostRewriteHeader = properties.egress.hostHeaderRewriting.customHostHeader } return routeAction diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyIngressRoutesFactory.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyIngressRoutesFactory.kt index 309539363..c8bc176de 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyIngressRoutesFactory.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyIngressRoutesFactory.kt @@ -3,15 +3,15 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes import com.google.protobuf.Duration import com.google.protobuf.UInt32Value import com.google.protobuf.util.Durations -import io.envoyproxy.envoy.api.v2.RouteConfiguration -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher -import io.envoyproxy.envoy.api.v2.route.RetryPolicy -import io.envoyproxy.envoy.api.v2.route.Route -import io.envoyproxy.envoy.api.v2.route.RouteAction -import io.envoyproxy.envoy.api.v2.route.RouteMatch -import io.envoyproxy.envoy.api.v2.route.VirtualCluster -import io.envoyproxy.envoy.api.v2.route.VirtualHost -import io.envoyproxy.envoy.type.matcher.RegexMatcher +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher +import io.envoyproxy.envoy.config.route.v3.RetryPolicy +import io.envoyproxy.envoy.config.route.v3.Route +import io.envoyproxy.envoy.config.route.v3.RouteAction +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration +import io.envoyproxy.envoy.config.route.v3.RouteMatch +import io.envoyproxy.envoy.config.route.v3.VirtualCluster +import io.envoyproxy.envoy.config.route.v3.VirtualHost +import io.envoyproxy.envoy.type.matcher.v3.RegexMatcher import pl.allegro.tech.servicemesh.envoycontrol.groups.PathMatchingType import pl.allegro.tech.servicemesh.envoycontrol.groups.ProxySettings import pl.allegro.tech.servicemesh.envoycontrol.protocol.HttpMethod @@ -113,8 +113,7 @@ class EnvoyIngressRoutesFactory( .setRoute(clusterRouteActionWithRetryPolicy(retryPolicy, localRouteAction)) } return (retryRoutes + nonRetryRoute).map { builder -> - builder.setMetadata(filterMetadata) - .build() + builder.setMetadata(filterMetadata).build() } } diff --git a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/RoutesMatchers.kt b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/RoutesMatchers.kt index df5f5c3d9..191ae064c 100644 --- a/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/RoutesMatchers.kt +++ b/envoy-control-core/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/RoutesMatchers.kt @@ -1,6 +1,6 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.routes -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher import pl.allegro.tech.servicemesh.envoycontrol.protocol.HttpMethod fun httpMethodMatcher(method: HttpMethod): HeaderMatcher = exactHeader(":method", method.name) diff --git a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheWithMissingEndpointsTest.java b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheWithMissingEndpointsTest.java deleted file mode 100644 index a2bda07c5..000000000 --- a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheWithMissingEndpointsTest.java +++ /dev/null @@ -1,63 +0,0 @@ -package pl.allegro.tech.servicemesh.envoycontrol; - -import com.google.common.collect.ImmutableList; -import io.envoyproxy.controlplane.cache.Resources; -import io.envoyproxy.controlplane.cache.Snapshot; -import io.envoyproxy.controlplane.cache.Watch; -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment; -import io.envoyproxy.envoy.api.v2.DiscoveryRequest; -import io.envoyproxy.envoy.api.v2.core.Node; -import org.junit.Ignore; -import org.junit.jupiter.api.Test; - -import java.util.Collections; - -import static java.util.Collections.emptyList; - -public class SimpleCacheWithMissingEndpointsTest extends SimpleCacheTest { - - @Override - protected boolean shouldSendMissingEndpoints() { - return true; - } - - protected static final Snapshot SNAPSHOT_WITH_MISSING_RESOURCES = Snapshot.create( - emptyList(), - ImmutableList.of( - ClusterLoadAssignment.newBuilder().setClusterName("none").build(), - ClusterLoadAssignment.newBuilder().setClusterName(CLUSTER_NAME).build() - ), - emptyList(), - emptyList(), - emptyList(), - VERSION2 - ); - - @Ignore - @Override - public void invalidNamesListShouldReturnWatcherWithNoResponseInAdsMode() { - } - - @Test - public void missingNamesListShouldReturnWatcherWithResponseInAdsMode() { - SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); - - cache.setSnapshot(SingleNodeGroup.GROUP, MULTIPLE_RESOURCES_SNAPSHOT2); - - ResponseTracker responseTracker = new ResponseTracker(); - - Watch watch = cache.createWatch( - true, - DiscoveryRequest.newBuilder() - .setNode(Node.getDefaultInstance()) - .setTypeUrl(Resources.ENDPOINT_TYPE_URL) - .addResourceNames("none") - .addResourceNames(CLUSTER_NAME) - .build(), - Collections.emptySet(), - responseTracker, - false); - - assertThatWatchReceivesSnapshot(new WatchAndTracker(watch, responseTracker), SNAPSHOT_WITH_MISSING_RESOURCES); - } -} diff --git a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheTest.java b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheTest.java similarity index 82% rename from envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheTest.java rename to envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheTest.java index a5dd041a4..2a9885649 100644 --- a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/SimpleCacheTest.java +++ b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheTest.java @@ -1,4 +1,7 @@ -package pl.allegro.tech.servicemesh.envoycontrol; +package pl.allegro.tech.servicemesh.envoycontrol.v2; +import static io.envoyproxy.controlplane.cache.Resources.V2.CLUSTER_TYPE_URL; +import static io.envoyproxy.controlplane.cache.Resources.V2.ROUTE_TYPE_URL; +import static org.assertj.core.api.Assertions.assertThat; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; @@ -6,9 +9,10 @@ import io.envoyproxy.controlplane.cache.NodeGroup; import io.envoyproxy.controlplane.cache.Resources; import io.envoyproxy.controlplane.cache.Response; -import io.envoyproxy.controlplane.cache.Snapshot; import io.envoyproxy.controlplane.cache.StatusInfo; import io.envoyproxy.controlplane.cache.Watch; +import io.envoyproxy.controlplane.cache.XdsRequest; +import io.envoyproxy.controlplane.cache.v2.Snapshot; import io.envoyproxy.envoy.api.v2.Cluster; import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment; import io.envoyproxy.envoy.api.v2.DiscoveryRequest; @@ -16,8 +20,8 @@ import io.envoyproxy.envoy.api.v2.RouteConfiguration; import io.envoyproxy.envoy.api.v2.auth.Secret; import io.envoyproxy.envoy.api.v2.core.Node; -import org.junit.jupiter.api.Test; - +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -26,12 +30,10 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.function.Consumer; import java.util.stream.Collectors; - -import static io.envoyproxy.controlplane.cache.Resources.ROUTE_TYPE_URL; -import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Test; /** - * This class is copy of {@link io.envoyproxy.controlplane.cache.SimpleCacheTest} + * This class is copy of {@link io.envoyproxy.controlplane.cache.v2.SimpleCacheTest} */ public class SimpleCacheTest { @@ -50,7 +52,7 @@ public class SimpleCacheTest { ImmutableList.of(ClusterLoadAssignment.getDefaultInstance()), ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), - ImmutableList.of(Secret.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), VERSION1); private static final Snapshot SNAPSHOT2 = Snapshot.create( @@ -58,7 +60,7 @@ public class SimpleCacheTest { ImmutableList.of(ClusterLoadAssignment.getDefaultInstance()), ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), - ImmutableList.of(Secret.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), VERSION2); protected static final Snapshot MULTIPLE_RESOURCES_SNAPSHOT2 = Snapshot.create( @@ -68,7 +70,7 @@ public class SimpleCacheTest { ClusterLoadAssignment.newBuilder().setClusterName(SECONDARY_CLUSTER_NAME).build()), ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), - ImmutableList.of(Secret.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), VERSION2); protected boolean shouldSendMissingEndpoints() { @@ -85,11 +87,11 @@ public void invalidNamesListShouldReturnWatcherWithNoResponseInAdsMode() { Watch watch = cache.createWatch( true, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) - .setTypeUrl(Resources.ENDPOINT_TYPE_URL) + .setTypeUrl(Resources.V2.ENDPOINT_TYPE_URL) .addResourceNames("none") - .build(), + .build()), Collections.emptySet(), responseTracker, false); @@ -107,11 +109,11 @@ public void invalidNamesListShouldReturnWatcherWithResponseInXdsMode() { Watch watch = cache.createWatch( false, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) - .setTypeUrl(Resources.ENDPOINT_TYPE_URL) + .setTypeUrl(Resources.V2.ENDPOINT_TYPE_URL) .addResourceNames("none") - .build(), + .build()), Collections.emptySet(), responseTracker, false); @@ -126,16 +128,16 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatch() { cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { ResponseTracker responseTracker = new ResponseTracker(); Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) - .build(), + .build()), Collections.emptySet(), responseTracker, false); @@ -158,19 +160,19 @@ public void shouldSendEdsWhenClusterChangedButEdsVersionDidnt() { Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setVersionInfo(VERSION1) - .setTypeUrl(Resources.ENDPOINT_TYPE_URL) - .addAllResourceNames(SNAPSHOT1.resources(Resources.ENDPOINT_TYPE_URL).keySet()) - .build(), + .setTypeUrl(Resources.V2.ENDPOINT_TYPE_URL) + .addAllResourceNames(SNAPSHOT1.resources(Resources.V2.ENDPOINT_TYPE_URL).keySet()) + .build()), Sets.newHashSet(""), responseTracker, true); - assertThat(watch.request().getTypeUrl()).isEqualTo(Resources.ENDPOINT_TYPE_URL); + assertThat(watch.request().getTypeUrl()).isEqualTo(Resources.V2.ENDPOINT_TYPE_URL); assertThat(watch.request().getResourceNamesList()).containsExactlyElementsOf( - SNAPSHOT1.resources(Resources.ENDPOINT_TYPE_URL).keySet()); + SNAPSHOT1.resources(Resources.V2.ENDPOINT_TYPE_URL).keySet()); assertThatWatchReceivesSnapshot(new WatchAndTracker(watch, responseTracker), SNAPSHOT1); } @@ -179,7 +181,7 @@ public void shouldSendEdsWhenClusterChangedButEdsVersionDidnt() { public void successfullyWatchAllResourceTypesWithSetAfterWatch() { SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); - Map watches = Resources.TYPE_URLS.stream() + Map watches = Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -187,11 +189,11 @@ public void successfullyWatchAllResourceTypesWithSetAfterWatch() { Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) - .build(), + .build()), Collections.emptySet(), responseTracker, false); @@ -201,7 +203,7 @@ public void successfullyWatchAllResourceTypesWithSetAfterWatch() { cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { assertThatWatchReceivesSnapshot(watches.get(typeUrl), SNAPSHOT1); } } @@ -217,7 +219,7 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithRequestVersio HashMap watches = new HashMap<>(); for (int i = 0; i < 2; ++i) { - watches.putAll(Resources.TYPE_URLS.stream() + watches.putAll(Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -225,12 +227,12 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithRequestVersio Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .setVersionInfo(SNAPSHOT1.version(typeUrl)) .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) - .build(), + .build()), SNAPSHOT2.resources(typeUrl).keySet(), r -> { responseTracker.accept(r); @@ -244,21 +246,21 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithRequestVersio } // The request version matches the current snapshot version, so the watches shouldn't receive any responses. - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); } cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT2); - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { assertThatWatchReceivesSnapshot(watches.get(typeUrl), SNAPSHOT2); } // Verify that CDS and LDS always get triggered before EDS and RDS respectively. - assertThat(responseOrderTracker.responseTypes).containsExactly(Resources.CLUSTER_TYPE_URL, - Resources.CLUSTER_TYPE_URL, Resources.ENDPOINT_TYPE_URL, Resources.ENDPOINT_TYPE_URL, - Resources.LISTENER_TYPE_URL, Resources.LISTENER_TYPE_URL, Resources.ROUTE_TYPE_URL, - Resources.ROUTE_TYPE_URL, Resources.SECRET_TYPE_URL, Resources.SECRET_TYPE_URL); + assertThat(responseOrderTracker.responseTypes).containsExactly(Resources.V2.CLUSTER_TYPE_URL, + Resources.V2.CLUSTER_TYPE_URL, Resources.V2.ENDPOINT_TYPE_URL, Resources.V2.ENDPOINT_TYPE_URL, + Resources.V2.LISTENER_TYPE_URL, Resources.V2.LISTENER_TYPE_URL, Resources.V2.ROUTE_TYPE_URL, + Resources.V2.ROUTE_TYPE_URL, Resources.V2.SECRET_TYPE_URL, Resources.V2.SECRET_TYPE_URL); } @Test @@ -272,7 +274,7 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVe // // Note how we're requesting the resources from MULTIPLE_RESOURCE_SNAPSHOT2 while claiming we // only know about the ones from SNAPSHOT2 - Map watches = Resources.TYPE_URLS.stream() + Map watches = Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -280,12 +282,12 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVe Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .setVersionInfo(MULTIPLE_RESOURCES_SNAPSHOT2.version(typeUrl)) .addAllResourceNames(MULTIPLE_RESOURCES_SNAPSHOT2.resources(typeUrl).keySet()) - .build(), + .build()), SNAPSHOT2.resources(typeUrl).keySet(), responseTracker, false); @@ -295,8 +297,8 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVe // The snapshot version matches for all resources, but for eds and cds there are new resources present // for the same version, so we expect the watches to trigger. - assertThatWatchReceivesSnapshot(watches.remove(Resources.CLUSTER_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); - assertThatWatchReceivesSnapshot(watches.remove(Resources.ENDPOINT_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); + assertThatWatchReceivesSnapshot(watches.remove(Resources.V2.CLUSTER_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); + assertThatWatchReceivesSnapshot(watches.remove(Resources.V2.ENDPOINT_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); // Remaining watches should not trigger for (WatchAndTracker watchAndTracker : watches.values()) { @@ -317,7 +319,7 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVe // while we only know about the resources found in SNAPSHOT2. Since SNAPSHOT2 is the current // snapshot, we have nothing to respond with for the new resources so we should not trigger // the watch. - Map watches = Resources.TYPE_URLS.stream() + Map watches = Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -325,12 +327,12 @@ public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVe Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .setVersionInfo(SNAPSHOT2.version(typeUrl)) .addAllResourceNames(MULTIPLE_RESOURCES_SNAPSHOT2.resources(typeUrl).keySet()) - .build(), + .build()), SNAPSHOT2.resources(typeUrl).keySet(), responseTracker, false); @@ -350,7 +352,7 @@ public void setSnapshotWithVersionMatchingRequestShouldLeaveWatchOpenWithoutAddi cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); - Map watches = Resources.TYPE_URLS.stream() + Map watches = Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -358,12 +360,12 @@ public void setSnapshotWithVersionMatchingRequestShouldLeaveWatchOpenWithoutAddi Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .setVersionInfo(SNAPSHOT1.version(typeUrl)) .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) - .build(), + .build()), SNAPSHOT1.resources(typeUrl).keySet(), responseTracker, false); @@ -372,14 +374,14 @@ public void setSnapshotWithVersionMatchingRequestShouldLeaveWatchOpenWithoutAddi })); // The request version matches the current snapshot version, so the watches shouldn't receive any responses. - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); } cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); // The request version still matches the current snapshot version, so the watches shouldn't receive any responses. - for (String typeUrl : Resources.TYPE_URLS) { + for (String typeUrl : Resources.V2.TYPE_URLS) { assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); } } @@ -388,7 +390,7 @@ public void setSnapshotWithVersionMatchingRequestShouldLeaveWatchOpenWithoutAddi public void watchesAreReleasedAfterCancel() { SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); - Map watches = Resources.TYPE_URLS.stream() + Map watches = Resources.V2.TYPE_URLS.stream() .collect(Collectors.toMap( typeUrl -> typeUrl, typeUrl -> { @@ -396,11 +398,11 @@ public void watchesAreReleasedAfterCancel() { Watch watch = cache.createWatch( ADS, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(typeUrl) .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) - .build(), + .build()), Collections.emptySet(), responseTracker, false); @@ -428,11 +430,11 @@ public void watchIsLeftOpenIfNotRespondedImmediately() { ResponseTracker responseTracker = new ResponseTracker(); Watch watch = cache.createWatch( true, - DiscoveryRequest.newBuilder() + XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) .setTypeUrl(ROUTE_TYPE_URL) .addAllResourceNames(Collections.singleton(ROUTE_NAME)) - .build(), + .build()), Collections.singleton(ROUTE_NAME), responseTracker, false); @@ -466,10 +468,13 @@ public void clearSnapshotWithWatches() { cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); - final Watch watch = cache.createWatch(ADS, DiscoveryRequest.newBuilder() + // Create a watch with an arbitrary type URL and a versionInfo that matches the saved + // snapshot, so the watch doesn't immediately close. + final Watch watch = cache.createWatch(ADS, XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) - .setTypeUrl("") - .build(), + .setTypeUrl(CLUSTER_TYPE_URL) + .setVersionInfo(SNAPSHOT1.version(CLUSTER_TYPE_URL)) + .build()), Collections.emptySet(), r -> { }, false); @@ -493,10 +498,10 @@ public void groups() { assertThat(cache.groups()).isEmpty(); - cache.createWatch(ADS, DiscoveryRequest.newBuilder() + cache.createWatch(ADS, XdsRequest.create(DiscoveryRequest.newBuilder() .setNode(Node.getDefaultInstance()) - .setTypeUrl("") - .build(), + .setTypeUrl(CLUSTER_TYPE_URL) + .build()), Collections.emptySet(), r -> { }, false); @@ -509,6 +514,7 @@ private static void assertThatWatchIsOpenWithNoResponses(WatchAndTracker watchAn assertThat(watchAndTracker.tracker.responses).isEmpty(); } + protected static void assertThatWatchReceivesSnapshot(WatchAndTracker watchAndTracker, Snapshot snapshot) { assertThat(watchAndTracker.tracker.responses).isNotEmpty(); @@ -517,7 +523,7 @@ protected static void assertThatWatchReceivesSnapshot(WatchAndTracker watchAndTr assertThat(response).isNotNull(); assertThat(response.version()).isEqualTo(snapshot.version(watchAndTracker.watch.request().getTypeUrl())); assertThat(response.resources().toArray(new Message[0])) - .containsExactlyElementsOf(snapshot.resources(watchAndTracker.watch.request().getTypeUrl()).values()); + .containsExactlyElementsOf(snapshot.resources(watchAndTracker.watch.request().getTypeUrl()).values()); } protected static class ResponseTracker implements Consumer { @@ -529,14 +535,16 @@ public void accept(Response response) { responses.add(response); } + public LinkedList getResponses() { + return responses; + } } private static class ResponseOrderTracker implements Consumer { private final LinkedList responseTypes = new LinkedList<>(); - @Override - public void accept(Response response) { + @Override public void accept(Response response) { responseTypes.add(response.request().getTypeUrl()); } } @@ -553,6 +561,11 @@ public String hash(Node node) { return GROUP; } + + @Override + public String hash(io.envoyproxy.envoy.config.core.v3.Node node) { + throw new IllegalStateException("should not have received a v3 Node in a v2 Test"); + } } protected static class WatchAndTracker { diff --git a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheWithMissingEndpointsTest.java b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheWithMissingEndpointsTest.java new file mode 100644 index 000000000..9463db3b6 --- /dev/null +++ b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v2/SimpleCacheWithMissingEndpointsTest.java @@ -0,0 +1,85 @@ +package pl.allegro.tech.servicemesh.envoycontrol.v2; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Message; +import io.envoyproxy.controlplane.cache.Resources; +import io.envoyproxy.controlplane.cache.Response; +import io.envoyproxy.controlplane.cache.Watch; +import io.envoyproxy.controlplane.cache.XdsRequest; +import io.envoyproxy.controlplane.cache.v2.Snapshot; +import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment; +import io.envoyproxy.envoy.api.v2.DiscoveryRequest; +import io.envoyproxy.envoy.api.v2.core.Node; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Ignore; +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static java.util.Collections.emptyList; + +public class SimpleCacheWithMissingEndpointsTest extends SimpleCacheTest { + + @Override + protected boolean shouldSendMissingEndpoints() { + return true; + } + + protected static final Snapshot SNAPSHOT_WITH_MISSING_RESOURCES = Snapshot.create( + emptyList(), + ImmutableList.of( + ClusterLoadAssignment.newBuilder().setClusterName("none").build(), + ClusterLoadAssignment.newBuilder().setClusterName(CLUSTER_NAME).build() + ), + emptyList(), + emptyList(), + emptyList(), + VERSION2 + ); + + @Ignore + @Override + public void invalidNamesListShouldReturnWatcherWithNoResponseInAdsMode() { + } + + @Test + public void missingNamesListShouldReturnWatcherWithResponseInAdsMode() { + pl.allegro.tech.servicemesh.envoycontrol.v2.SimpleCache cache = new pl.allegro.tech.servicemesh.envoycontrol.v2.SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, MULTIPLE_RESOURCES_SNAPSHOT2); + + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + true, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(Resources.V2.ENDPOINT_TYPE_URL) + .addResourceNames("none") + .addResourceNames(CLUSTER_NAME) + .build()), + Collections.emptySet(), + responseTracker, + false); + + assertThatWatchReceivesSnapshotWithMissingResources(new WatchAndTracker(watch, responseTracker), SNAPSHOT_WITH_MISSING_RESOURCES); + } + + private static void assertThatWatchReceivesSnapshotWithMissingResources(WatchAndTracker watchAndTracker, Snapshot snapshot) { + assertThat(watchAndTracker.tracker.getResponses()).isNotEmpty(); + + Response response = watchAndTracker.tracker.getResponses().getFirst(); + + assertThat(response).isNotNull(); + assertThat(response.version()).isEqualTo(snapshot.version(watchAndTracker.watch.request().getTypeUrl())); + Message[] responseValues = response.resources().toArray(new Message[0]); + Message[] snapshotValues = snapshot.resources(watchAndTracker.watch.request().getTypeUrl()).values().toArray(new Message[0]); + + assertThat(responseValues.length).isEqualTo(2); + assertThat(responseValues.length).isEqualTo(snapshotValues.length); + assertThat(responseValues[0]).isEqualToComparingFieldByFieldRecursively(snapshotValues[0]); + assertThat(responseValues[1]).isEqualToComparingFieldByFieldRecursively(snapshotValues[1]); + } +} diff --git a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheTest.java b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheTest.java new file mode 100644 index 000000000..6f85b7128 --- /dev/null +++ b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheTest.java @@ -0,0 +1,573 @@ +package pl.allegro.tech.servicemesh.envoycontrol.v3; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.protobuf.Message; +import io.envoyproxy.controlplane.cache.*; +import io.envoyproxy.controlplane.cache.v3.Snapshot; +import io.envoyproxy.envoy.config.cluster.v3.Cluster; +import io.envoyproxy.envoy.config.core.v3.Node; +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment; +import io.envoyproxy.envoy.config.listener.v3.Listener; +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration; +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.Secret; +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest; +import org.junit.Test; +import pl.allegro.tech.servicemesh.envoycontrol.v3.SimpleCache; + +import java.util.*; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static io.envoyproxy.controlplane.cache.Resources.V3.CLUSTER_TYPE_URL; +import static io.envoyproxy.controlplane.cache.Resources.V3.ROUTE_TYPE_URL; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * This class is copy of {@link io.envoyproxy.controlplane.cache.v3.SimpleCacheTest} + */ +public class SimpleCacheTest { + + private static final boolean ADS = ThreadLocalRandom.current().nextBoolean(); + protected static final String CLUSTER_NAME = "cluster0"; + private static final String SECONDARY_CLUSTER_NAME = "cluster1"; + private static final String LISTENER_NAME = "listener0"; + private static final String ROUTE_NAME = "route0"; + private static final String SECRET_NAME = "secret0"; + + private static final String VERSION1 = UUID.randomUUID().toString(); + protected static final String VERSION2 = UUID.randomUUID().toString(); + + private static final Snapshot SNAPSHOT1 = Snapshot.create( + ImmutableList.of(Cluster.newBuilder().setName(CLUSTER_NAME).build()), + ImmutableList.of(ClusterLoadAssignment.getDefaultInstance()), + ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), + ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), + VERSION1); + + private static final Snapshot SNAPSHOT2 = Snapshot.create( + ImmutableList.of(Cluster.newBuilder().setName(CLUSTER_NAME).build()), + ImmutableList.of(ClusterLoadAssignment.getDefaultInstance()), + ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), + ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), + VERSION2); + + protected static final Snapshot MULTIPLE_RESOURCES_SNAPSHOT2 = Snapshot.create( + ImmutableList.of(Cluster.newBuilder().setName(CLUSTER_NAME).build(), + Cluster.newBuilder().setName(SECONDARY_CLUSTER_NAME).build()), + ImmutableList.of(ClusterLoadAssignment.newBuilder().setClusterName(CLUSTER_NAME).build(), + ClusterLoadAssignment.newBuilder().setClusterName(SECONDARY_CLUSTER_NAME).build()), + ImmutableList.of(Listener.newBuilder().setName(LISTENER_NAME).build()), + ImmutableList.of(RouteConfiguration.newBuilder().setName(ROUTE_NAME).build()), + ImmutableList.of(Secret.newBuilder().setName(SECRET_NAME).build()), + VERSION2); + + protected boolean shouldSendMissingEndpoints() { + return false; + } + + @Test + public void invalidNamesListShouldReturnWatcherWithNoResponseInAdsMode() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + true, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(Resources.V3.ENDPOINT_TYPE_URL) + .addResourceNames("none") + .build()), + Collections.emptySet(), + responseTracker, + false); + + assertThatWatchIsOpenWithNoResponses(new WatchAndTracker(watch, responseTracker)); + } + + @Test + public void invalidNamesListShouldReturnWatcherWithResponseInXdsMode() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + false, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(Resources.V3.ENDPOINT_TYPE_URL) + .addResourceNames("none") + .build()), + Collections.emptySet(), + responseTracker, + false); + + assertThat(watch.isCancelled()).isFalse(); + assertThat(responseTracker.responses).isNotEmpty(); + } + + @Test + public void successfullyWatchAllResourceTypesWithSetBeforeWatch() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + for (String typeUrl : Resources.V3.TYPE_URLS) { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) + .build()), + Collections.emptySet(), + responseTracker, + false); + + assertThat(watch.request().getTypeUrl()).isEqualTo(typeUrl); + assertThat(watch.request().getResourceNamesList()).containsExactlyElementsOf( + SNAPSHOT1.resources(typeUrl).keySet()); + + assertThatWatchReceivesSnapshot(new WatchAndTracker(watch, responseTracker), SNAPSHOT1); + } + } + + @Test + public void shouldSendEdsWhenClusterChangedButEdsVersionDidnt() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setVersionInfo(VERSION1) + .setTypeUrl(Resources.V3.ENDPOINT_TYPE_URL) + .addAllResourceNames(SNAPSHOT1.resources(Resources.V3.ENDPOINT_TYPE_URL).keySet()) + .build()), + Sets.newHashSet(""), + responseTracker, + true); + + assertThat(watch.request().getTypeUrl()).isEqualTo(Resources.V3.ENDPOINT_TYPE_URL); + assertThat(watch.request().getResourceNamesList()).containsExactlyElementsOf( + SNAPSHOT1.resources(Resources.V3.ENDPOINT_TYPE_URL).keySet()); + + assertThatWatchReceivesSnapshot(new WatchAndTracker(watch, responseTracker), SNAPSHOT1); + } + + @Test + public void successfullyWatchAllResourceTypesWithSetAfterWatch() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + Map watches = Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) + .build()), + Collections.emptySet(), + responseTracker, + false); + + return new WatchAndTracker(watch, responseTracker); + })); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + for (String typeUrl : Resources.V3.TYPE_URLS) { + assertThatWatchReceivesSnapshot(watches.get(typeUrl), SNAPSHOT1); + } + } + + @Test + public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithRequestVersion() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + ResponseOrderTracker responseOrderTracker = new ResponseOrderTracker(); + + HashMap watches = new HashMap<>(); + + for (int i = 0; i < 2; ++i) { + watches.putAll(Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .setVersionInfo(SNAPSHOT1.version(typeUrl)) + .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) + .build()), + SNAPSHOT2.resources(typeUrl).keySet(), + r -> { + responseTracker.accept(r); + responseOrderTracker.accept(r); + }, + false); + + return new WatchAndTracker(watch, responseTracker); + })) + ); + } + + // The request version matches the current snapshot version, so the watches shouldn't receive any responses. + for (String typeUrl : Resources.V3.TYPE_URLS) { + assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); + } + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT2); + + for (String typeUrl : Resources.V3.TYPE_URLS) { + assertThatWatchReceivesSnapshot(watches.get(typeUrl), SNAPSHOT2); + } + + // Verify that CDS and LDS always get triggered before EDS and RDS respectively. + assertThat(responseOrderTracker.responseTypes).containsExactly(Resources.V3.CLUSTER_TYPE_URL, + Resources.V3.CLUSTER_TYPE_URL, Resources.V3.ENDPOINT_TYPE_URL, Resources.V3.ENDPOINT_TYPE_URL, + Resources.V3.LISTENER_TYPE_URL, Resources.V3.LISTENER_TYPE_URL, Resources.V3.ROUTE_TYPE_URL, + Resources.V3.ROUTE_TYPE_URL, Resources.V3.SECRET_TYPE_URL, Resources.V3.SECRET_TYPE_URL); + } + + @Test + public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVersionNewResourceHints() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, MULTIPLE_RESOURCES_SNAPSHOT2); + + // Set a watch for the current snapshot with the same version but with resource hints present + // in the snapshot that the watch creator does not currently know about. + // + // Note how we're requesting the resources from MULTIPLE_RESOURCE_SNAPSHOT2 while claiming we + // only know about the ones from SNAPSHOT2 + Map watches = Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .setVersionInfo(MULTIPLE_RESOURCES_SNAPSHOT2.version(typeUrl)) + .addAllResourceNames(MULTIPLE_RESOURCES_SNAPSHOT2.resources(typeUrl).keySet()) + .build()), + SNAPSHOT2.resources(typeUrl).keySet(), + responseTracker, + false); + + return new WatchAndTracker(watch, responseTracker); + })); + + // The snapshot version matches for all resources, but for eds and cds there are new resources present + // for the same version, so we expect the watches to trigger. + assertThatWatchReceivesSnapshot(watches.remove(Resources.V3.CLUSTER_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); + assertThatWatchReceivesSnapshot(watches.remove(Resources.V3.ENDPOINT_TYPE_URL), MULTIPLE_RESOURCES_SNAPSHOT2); + + // Remaining watches should not trigger + for (WatchAndTracker watchAndTracker : watches.values()) { + assertThatWatchIsOpenWithNoResponses(watchAndTracker); + } + } + + @Test + public void successfullyWatchAllResourceTypesWithSetBeforeWatchWithSameRequestVersionNewResourceHintsNoChange() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT2); + + // Set a watch for the current snapshot for the same version but with new resource hints not + // present in the snapshot that the watch creator does not know about. + // + // Note that we're requesting the additional resources found in MULTIPLE_RESOURCE_SNAPSHOT2 + // while we only know about the resources found in SNAPSHOT2. Since SNAPSHOT2 is the current + // snapshot, we have nothing to respond with for the new resources so we should not trigger + // the watch. + Map watches = Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .setVersionInfo(SNAPSHOT2.version(typeUrl)) + .addAllResourceNames(MULTIPLE_RESOURCES_SNAPSHOT2.resources(typeUrl).keySet()) + .build()), + SNAPSHOT2.resources(typeUrl).keySet(), + responseTracker, + false); + + return new WatchAndTracker(watch, responseTracker); + })); + + // No watches should trigger since no new information will be returned + for (WatchAndTracker watchAndTracker : watches.values()) { + assertThatWatchIsOpenWithNoResponses(watchAndTracker); + } + } + + @Test + public void setSnapshotWithVersionMatchingRequestShouldLeaveWatchOpenWithoutAdditionalResponse() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + Map watches = Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .setVersionInfo(SNAPSHOT1.version(typeUrl)) + .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) + .build()), + SNAPSHOT1.resources(typeUrl).keySet(), + responseTracker, + false); + + return new WatchAndTracker(watch, responseTracker); + })); + + // The request version matches the current snapshot version, so the watches shouldn't receive any responses. + for (String typeUrl : Resources.V3.TYPE_URLS) { + assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); + } + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + // The request version still matches the current snapshot version, so the watches shouldn't receive any responses. + for (String typeUrl : Resources.V3.TYPE_URLS) { + assertThatWatchIsOpenWithNoResponses(watches.get(typeUrl)); + } + } + + @Test + public void watchesAreReleasedAfterCancel() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + Map watches = Resources.V3.TYPE_URLS.stream() + .collect(Collectors.toMap( + typeUrl -> typeUrl, + typeUrl -> { + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + ADS, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(typeUrl) + .addAllResourceNames(SNAPSHOT1.resources(typeUrl).keySet()) + .build()), + Collections.emptySet(), + responseTracker, + false); + + return new WatchAndTracker(watch, responseTracker); + })); + + StatusInfo statusInfo = cache.statusInfo(SingleNodeGroup.GROUP); + + assertThat(statusInfo.numWatches()).isEqualTo(watches.size()); + + watches.values().forEach(w -> w.watch.cancel()); + + assertThat(statusInfo.numWatches()).isZero(); + + watches.values().forEach(w -> assertThat(w.watch.isCancelled()).isTrue()); + } + + @Test + public void watchIsLeftOpenIfNotRespondedImmediately() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + cache.setSnapshot(SingleNodeGroup.GROUP, Snapshot.create( + ImmutableList.of(), ImmutableList.of(), ImmutableList.of(), ImmutableList.of(), ImmutableList.of(), VERSION1)); + + ResponseTracker responseTracker = new ResponseTracker(); + Watch watch = cache.createWatch( + true, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(ROUTE_TYPE_URL) + .addAllResourceNames(Collections.singleton(ROUTE_NAME)) + .build()), + Collections.singleton(ROUTE_NAME), + responseTracker, + false); + + assertThatWatchIsOpenWithNoResponses(new WatchAndTracker(watch, responseTracker)); + } + + @Test + public void getSnapshot() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + assertThat(cache.getSnapshot(SingleNodeGroup.GROUP)).isEqualTo(SNAPSHOT1); + } + + @Test + public void clearSnapshot() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + assertThat(cache.clearSnapshot(SingleNodeGroup.GROUP)).isTrue(); + + assertThat(cache.getSnapshot(SingleNodeGroup.GROUP)).isNull(); + } + + @Test + public void clearSnapshotWithWatches() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, SNAPSHOT1); + + // Create a watch with an arbitrary type URL and a versionInfo that matches the saved + // snapshot, so the watch doesn't immediately close. + final Watch watch = cache.createWatch(ADS, XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(CLUSTER_TYPE_URL) + .setVersionInfo(SNAPSHOT1.version(CLUSTER_TYPE_URL)) + .build()), + Collections.emptySet(), + r -> { }, + false); + + // clearSnapshot should fail and the snapshot should be left untouched + assertThat(cache.clearSnapshot(SingleNodeGroup.GROUP)).isFalse(); + assertThat(cache.getSnapshot(SingleNodeGroup.GROUP)).isEqualTo(SNAPSHOT1); + assertThat(cache.statusInfo(SingleNodeGroup.GROUP)).isNotNull(); + + watch.cancel(); + + // now that the watch is gone we should be able to clear it + assertThat(cache.clearSnapshot(SingleNodeGroup.GROUP)).isTrue(); + assertThat(cache.getSnapshot(SingleNodeGroup.GROUP)).isNull(); + assertThat(cache.statusInfo(SingleNodeGroup.GROUP)).isNull(); + } + + @Test + public void groups() { + SimpleCache cache = new SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + assertThat(cache.groups()).isEmpty(); + + cache.createWatch(ADS, XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(CLUSTER_TYPE_URL) + .build()), + Collections.emptySet(), + r -> { }, + false); + + assertThat(cache.groups()).containsExactly(SingleNodeGroup.GROUP); + } + + private static void assertThatWatchIsOpenWithNoResponses(WatchAndTracker watchAndTracker) { + assertThat(watchAndTracker.watch.isCancelled()).isFalse(); + assertThat(watchAndTracker.tracker.responses).isEmpty(); + } + + + protected static void assertThatWatchReceivesSnapshot(WatchAndTracker watchAndTracker, Snapshot snapshot) { + assertThat(watchAndTracker.tracker.responses).isNotEmpty(); + + Response response = watchAndTracker.tracker.responses.getFirst(); + + assertThat(response).isNotNull(); + assertThat(response.version()).isEqualTo(snapshot.version(watchAndTracker.watch.request().getTypeUrl())); + assertThat(response.resources().toArray(new Message[0])) + .containsExactlyElementsOf(snapshot.resources(watchAndTracker.watch.request().getTypeUrl()).values()); + } + + protected static class ResponseTracker implements Consumer { + + private final LinkedList responses = new LinkedList<>(); + + @Override + public void accept(Response response) { + responses.add(response); + } + + public LinkedList getResponses() { + return responses; + } + } + + private static class ResponseOrderTracker implements Consumer { + + private final LinkedList responseTypes = new LinkedList<>(); + + @Override public void accept(Response response) { + responseTypes.add(response.request().getTypeUrl()); + } + } + + protected static class SingleNodeGroup implements NodeGroup { + + protected static final String GROUP = "node"; + + @Override + public String hash(Node node) { + if (node == null) { + throw new IllegalArgumentException("node"); + } + + return GROUP; + } + + @Override + public String hash(io.envoyproxy.envoy.api.v2.core.Node node) { + throw new IllegalStateException("should not have received a v3 Node in a v2 Test"); + } + } + + protected static class WatchAndTracker { + + final Watch watch; + final ResponseTracker tracker; + + WatchAndTracker(Watch watch, ResponseTracker tracker) { + this.watch = watch; + this.tracker = tracker; + } + } +} diff --git a/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheWithMissingEndpointsTest.java b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheWithMissingEndpointsTest.java new file mode 100644 index 000000000..2b82ec6fe --- /dev/null +++ b/envoy-control-core/src/test/java/pl/allegro/tech/servicemesh/envoycontrol/v3/SimpleCacheWithMissingEndpointsTest.java @@ -0,0 +1,85 @@ +package pl.allegro.tech.servicemesh.envoycontrol.v3; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Message; +import io.envoyproxy.controlplane.cache.Resources; +import io.envoyproxy.controlplane.cache.Response; +import io.envoyproxy.controlplane.cache.Watch; +import io.envoyproxy.controlplane.cache.XdsRequest; +import io.envoyproxy.controlplane.cache.v3.Snapshot; +import io.envoyproxy.envoy.config.core.v3.Node; +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment; +import io.envoyproxy.envoy.service.discovery.v3.DiscoveryRequest; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Ignore; +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static java.util.Collections.emptyList; + +public class SimpleCacheWithMissingEndpointsTest extends SimpleCacheTest { + + @Override + protected boolean shouldSendMissingEndpoints() { + return true; + } + + protected static final Snapshot SNAPSHOT_WITH_MISSING_RESOURCES = Snapshot.create( + emptyList(), + ImmutableList.of( + ClusterLoadAssignment.newBuilder().setClusterName("none").build(), + ClusterLoadAssignment.newBuilder().setClusterName(CLUSTER_NAME).build() + ), + emptyList(), + emptyList(), + emptyList(), + VERSION2 + ); + + @Ignore + @Override + public void invalidNamesListShouldReturnWatcherWithNoResponseInAdsMode() { + } + + @Test + public void missingNamesListShouldReturnWatcherWithResponseInAdsMode() { + pl.allegro.tech.servicemesh.envoycontrol.v3.SimpleCache cache = new pl.allegro.tech.servicemesh.envoycontrol.v3.SimpleCache<>(new SingleNodeGroup(), shouldSendMissingEndpoints()); + + cache.setSnapshot(SingleNodeGroup.GROUP, MULTIPLE_RESOURCES_SNAPSHOT2); + + ResponseTracker responseTracker = new ResponseTracker(); + + Watch watch = cache.createWatch( + true, + XdsRequest.create(DiscoveryRequest.newBuilder() + .setNode(Node.getDefaultInstance()) + .setTypeUrl(Resources.V3.ENDPOINT_TYPE_URL) + .addResourceNames("none") + .addResourceNames(CLUSTER_NAME) + .build()), + Collections.emptySet(), + responseTracker, + false); + + assertThatWatchReceivesSnapshotWithMissingResources(new WatchAndTracker(watch, responseTracker), SNAPSHOT_WITH_MISSING_RESOURCES); + } + + private static void assertThatWatchReceivesSnapshotWithMissingResources(WatchAndTracker watchAndTracker, Snapshot snapshot) { + assertThat(watchAndTracker.tracker.getResponses()).isNotEmpty(); + + Response response = watchAndTracker.tracker.getResponses().getFirst(); + + assertThat(response).isNotNull(); + assertThat(response.version()).isEqualTo(snapshot.version(watchAndTracker.watch.request().getTypeUrl())); + Message[] responseValues = response.resources().toArray(new Message[0]); + Message[] snapshotValues = snapshot.resources(watchAndTracker.watch.request().getTypeUrl()).values().toArray(new Message[0]); + + assertThat(responseValues.length).isEqualTo(2); + assertThat(responseValues.length).isEqualTo(snapshotValues.length); + assertThat(responseValues[0]).isEqualToComparingFieldByFieldRecursively(snapshotValues[0]); + assertThat(responseValues[1]).isEqualToComparingFieldByFieldRecursively(snapshotValues[1]); + } +} diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoySnapshotFactoryTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoySnapshotFactoryTest.kt index 7ede769b7..3bc9bf7ff 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoySnapshotFactoryTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoySnapshotFactoryTest.kt @@ -2,11 +2,11 @@ package pl.allegro.tech.servicemesh.envoycontrol import com.google.protobuf.util.Durations import io.envoyproxy.controlplane.cache.SnapshotResources -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.Listener -import io.envoyproxy.envoy.api.v2.core.AggregatedConfigSource -import io.envoyproxy.envoy.api.v2.core.ConfigSource -import io.envoyproxy.envoy.api.v2.core.Metadata +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.core.v3.AggregatedConfigSource +import io.envoyproxy.envoy.config.core.v3.ConfigSource +import io.envoyproxy.envoy.config.core.v3.Metadata +import io.envoyproxy.envoy.config.listener.v3.Listener import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.assertj.core.api.Assertions.assertThat @@ -69,10 +69,10 @@ class EnvoySnapshotFactoryTest { assertThat(ingressSocket?.address).isEqualTo(INGRESS_HOST) assertThat(ingressSocket?.portValue).isEqualTo(INGRESS_PORT) - assertThat(ingressFilterChain?.filtersList?.get(0)?.name).isEqualTo("envoy.http_connection_manager") + assertThat(ingressFilterChain?.filtersList?.get(0)?.name).isEqualTo("envoy.filters.network.http_connection_manager") assertThat(egressSocket?.address).isEqualTo(EGRESS_HOST) assertThat(egressSocket?.portValue).isEqualTo(EGRESS_PORT) - assertThat(egressFilterChain?.filtersList?.get(0)?.name).isEqualTo("envoy.http_connection_manager") + assertThat(egressFilterChain?.filtersList?.get(0)?.name).isEqualTo("envoy.filters.network.http_connection_manager") } @Test @@ -111,10 +111,12 @@ class EnvoySnapshotFactoryTest { assertThat(listeners.size).isEqualTo(0) } - private fun createServicesGroup(mode: CommunicationMode = CommunicationMode.XDS, - serviceName: String = DEFAULT_SERVICE_NAME, - dependencies: Array = emptyArray(), - listenersConfigExists: Boolean = true): ServicesGroup { + private fun createServicesGroup( + mode: CommunicationMode = CommunicationMode.XDS, + serviceName: String = DEFAULT_SERVICE_NAME, + dependencies: Array = emptyArray(), + listenersConfigExists: Boolean = true + ): ServicesGroup { val listenersConfig = when (listenersConfigExists) { true -> createListenersConfig() false -> null @@ -140,8 +142,10 @@ class EnvoySnapshotFactoryTest { fun createSnapshotFactory(properties: SnapshotProperties): EnvoySnapshotFactory { val ingressRoutesFactory = EnvoyIngressRoutesFactory( SnapshotProperties(), - EnvoyHttpFilters(emptyList(), emptyList(), - Metadata.getDefaultInstance()) + EnvoyHttpFilters( + emptyList(), emptyList(), + Metadata.getDefaultInstance() + ) ) val egressRoutesFactory = EnvoyEgressRoutesFactory(properties) val clustersFactory = EnvoyClustersFactory(properties) @@ -151,21 +155,25 @@ class EnvoySnapshotFactoryTest { val snapshotsVersions = SnapshotsVersions() val meterRegistry: MeterRegistry = SimpleMeterRegistry() - return EnvoySnapshotFactory(ingressRoutesFactory, + return EnvoySnapshotFactory( + ingressRoutesFactory, egressRoutesFactory, clustersFactory, endpointsFactory, listenersFactory, snapshotsVersions, properties, - meterRegistry) + meterRegistry + ) } private fun createGlobalSnapshot(cluster: Cluster?): GlobalSnapshot { return GlobalSnapshot( - SnapshotResources.create(emptyList(), "v2"), emptySet(), + SnapshotResources.create(emptyList(), "pl/allegro/tech/servicemesh/envoycontrol/v3"), emptySet(), SnapshotResources.create(emptyList(), "v1"), emptyMap(), - SnapshotResources.create(listOf(cluster), "v3") + SnapshotResources.create(listOf(cluster), "v3"), + SnapshotResources.create(emptyList(), "pl/allegro/tech/servicemesh/envoycontrol/v3"), + SnapshotResources.create(emptyList(), "pl/allegro/tech/servicemesh/envoycontrol/v3") ) } diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroupTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroupTest.kt index 7eb115741..4ec0798bc 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroupTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/MetadataNodeGroupTest.kt @@ -10,7 +10,7 @@ import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.ADS import pl.allegro.tech.servicemesh.envoycontrol.groups.CommunicationMode.XDS import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotProperties import pl.allegro.tech.servicemesh.envoycontrol.snapshot.serviceDependencies -import io.envoyproxy.envoy.config.filter.accesslog.v2.ComparisonFilter.Op.EQ +import io.envoyproxy.envoy.config.accesslog.v3.ComparisonFilter import io.grpc.Status import org.junit.jupiter.api.assertThrows import org.junit.jupiter.params.ParameterizedTest @@ -242,8 +242,9 @@ class MetadataNodeGroupTest { val group = nodeGroup.hash(Node.newBuilder().setMetadata(metadata.build()).build()) // then - assertThat(group.listenersConfig!!.accessLogFilterSettings.statusCodeFilterSettings!!.comparisonCode).isEqualTo(400) - assertThat(group.listenersConfig!!.accessLogFilterSettings.statusCodeFilterSettings!!.comparisonOperator).isEqualTo(EQ) + val statusCodeFilterSettings = group.listenersConfig!!.accessLogFilterSettings.statusCodeFilterSettings!! + assertThat(statusCodeFilterSettings.comparisonCode).isEqualTo(400) + assertThat(statusCodeFilterSettings.comparisonOperator).isEqualTo(ComparisonFilter.Op.EQ) } @ParameterizedTest diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataTest.kt index 92e1f984b..2c5e1f3ff 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataTest.kt @@ -2,7 +2,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.groups import com.google.protobuf.Value import com.google.protobuf.util.Durations -import io.envoyproxy.envoy.config.filter.accesslog.v2.ComparisonFilter +import io.envoyproxy.envoy.config.accesslog.v3.ComparisonFilter import io.grpc.Status import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.ObjectAssert diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidatorTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidatorTest.kt index c39606e2f..f88b75d4b 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidatorTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/NodeMetadataValidatorTest.kt @@ -37,7 +37,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // when - val exception = catchThrowable { validator.onStreamRequest(streamId = 123, request = request) } + val exception = catchThrowable { validator.onV2StreamRequest(streamId = 123, request = request) } // then assertThat(exception).isInstanceOf(WildcardPrincipalValidationException::class.java) @@ -59,7 +59,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // when - val exception = catchThrowable { validator.onStreamRequest(streamId = 123, request = request) } + val exception = catchThrowable { validator.onV2StreamRequest(streamId = 123, request = request) } // expects assertThat(exception).isInstanceOf(WildcardPrincipalMixedWithOthersValidationException::class.java) @@ -80,7 +80,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // when - val exception = catchThrowable { validator.onStreamRequest(streamId = 123, request = request) } + val exception = catchThrowable { validator.onV2StreamRequest(streamId = 123, request = request) } // expects assertThat(exception).isInstanceOf(AllDependenciesValidationException::class.java) @@ -103,7 +103,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // then - assertDoesNotThrow { validator.onStreamRequest(123, request = request) } + assertDoesNotThrow { validator.onV2StreamRequest(123, request = request) } } @Test @@ -117,7 +117,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // then - assertDoesNotThrow { validator.onStreamRequest(123, request = request) } + assertDoesNotThrow { validator.onV2StreamRequest(123, request = request) } } @Test @@ -133,7 +133,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // then - assertDoesNotThrow { permissionsDisabledValidator.onStreamRequest(123, request = request) } + assertDoesNotThrow { permissionsDisabledValidator.onV2StreamRequest(123, request = request) } } @ParameterizedTest @@ -162,7 +162,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // when - val exception = catchThrowable { configurationModeValidator.onStreamRequest(streamId = 123, request = request) } + val exception = catchThrowable { configurationModeValidator.onV2StreamRequest(streamId = 123, request = request) } // expects assertThat(exception).isInstanceOf(ConfigurationModeNotSupportedException::class.java) @@ -198,7 +198,7 @@ class NodeMetadataValidatorTest { val request = DiscoveryRequest.newBuilder().setNode(node).build() // then - assertDoesNotThrow { configurationModeValidator.onStreamRequest(123, request = request) } + assertDoesNotThrow { configurationModeValidator.onV2StreamRequest(123, request = request) } } private fun createIncomingPermissions( diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/RoutesAssertions.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/RoutesAssertions.kt index d39ce47c7..99bbdcdee 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/RoutesAssertions.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/groups/RoutesAssertions.kt @@ -1,15 +1,15 @@ package pl.allegro.tech.servicemesh.envoycontrol.groups import com.google.protobuf.Duration -import io.envoyproxy.envoy.api.v2.RouteConfiguration -import io.envoyproxy.envoy.api.v2.route.DirectResponseAction -import io.envoyproxy.envoy.api.v2.route.HeaderMatcher -import io.envoyproxy.envoy.api.v2.route.RedirectAction -import io.envoyproxy.envoy.api.v2.route.RetryPolicy -import io.envoyproxy.envoy.api.v2.route.Route -import io.envoyproxy.envoy.api.v2.route.RouteAction -import io.envoyproxy.envoy.api.v2.route.VirtualCluster -import io.envoyproxy.envoy.api.v2.route.VirtualHost +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration +import io.envoyproxy.envoy.config.route.v3.DirectResponseAction +import io.envoyproxy.envoy.config.route.v3.HeaderMatcher +import io.envoyproxy.envoy.config.route.v3.RedirectAction +import io.envoyproxy.envoy.config.route.v3.RetryPolicy +import io.envoyproxy.envoy.config.route.v3.Route +import io.envoyproxy.envoy.config.route.v3.RouteAction +import io.envoyproxy.envoy.config.route.v3.VirtualCluster +import io.envoyproxy.envoy.config.route.v3.VirtualHost import org.assertj.core.api.Assertions.assertThat import pl.allegro.tech.servicemesh.envoycontrol.snapshot.RetryPolicyProperties @@ -50,13 +50,13 @@ fun RouteAction.hasCustomRequestTimeout(requestTimeout: Duration): RouteAction { return this } -fun RouteAction.autoHostRewriteHeaderIsEmpty(): RouteAction { - assertThat(this.autoHostRewriteHeader).isEmpty() +fun RouteAction.hostRewriteHeaderIsEmpty(): RouteAction { + assertThat(this.hostRewriteHeader).isEmpty() return this } -fun RouteAction.hasAutoHostRewriteHeader(header: String): RouteAction { - assertThat(this.autoHostRewriteHeader).isEqualTo(header) +fun RouteAction.hasHostRewriteHeader(header: String): RouteAction { + assertThat(this.hostRewriteHeader).isEqualTo(header) return this } diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdaterTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdaterTest.kt index d56d055f3..072dbb0ab 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdaterTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotUpdaterTest.kt @@ -1,12 +1,14 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot import com.google.protobuf.util.Durations +import io.envoyproxy.controlplane.cache.Resources import io.envoyproxy.controlplane.cache.Response -import io.envoyproxy.controlplane.cache.Snapshot import io.envoyproxy.controlplane.cache.SnapshotCache import io.envoyproxy.controlplane.cache.StatusInfo import io.envoyproxy.controlplane.cache.Watch -import io.envoyproxy.envoy.api.v2.DiscoveryRequest +import io.envoyproxy.controlplane.cache.XdsRequest +import io.envoyproxy.controlplane.cache.v3.Snapshot +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.assertj.core.api.Assertions.assertThat @@ -580,7 +582,7 @@ class SnapshotUpdaterTest { class FailingMockCacheException : RuntimeException() - open class MockCache : SnapshotCache { + open class MockCache : SnapshotCache { val groups: MutableMap = mutableMapOf() private var concurrentSetSnapshotCounter: CountDownLatch? = null @@ -603,7 +605,7 @@ class SnapshotUpdaterTest { override fun createWatch( ads: Boolean, - request: DiscoveryRequest, + request: XdsRequest, knownResourceNames: MutableSet, responseConsumer: Consumer, hasClusterChanged: Boolean @@ -629,32 +631,33 @@ class SnapshotUpdaterTest { } } - private fun hasSnapshot(cache: SnapshotCache, group: Group): Snapshot { + private fun hasSnapshot(cache: SnapshotCache, group: Group): Snapshot { val snapshot = cache.getSnapshot(group) assertThat(snapshot).isNotNull return snapshot } private fun Snapshot.hasOnlyClustersFor(vararg expected: String): Snapshot { - assertThat(this.clusters().resources().keys.toSet()) + assertThat(this.resources(Resources.ResourceType.CLUSTER).keys.toSet()) .isEqualTo(expected.toSet()) return this } private fun Snapshot.hasOnlyEndpointsFor(vararg expected: String): Snapshot { - assertThat(this.endpoints().resources().keys.toSet()) + assertThat(this.resources(Resources.ResourceType.ENDPOINT).keys.toSet()) .isEqualTo(expected.toSet()) return this } private fun Snapshot.hasOnlyEgressRoutesForClusters(vararg expected: String): Snapshot { - assertThat(this.routes().resources()["default_routes"]!!.virtualHostsList.flatMap { it.domainsList }.toSet()) + val routes = this.resources(Resources.ResourceType.ROUTE)["default_routes"] as RouteConfiguration + assertThat(routes.virtualHostsList.flatMap { it.domainsList }.toSet()) .isEqualTo(expected.toSet() + setOf("envoy-original-destination", "*")) return this } private fun Snapshot.withoutClusters() { - assertThat(this.clusters().resources().keys).isEmpty() + assertThat(this.resources(Resources.ResourceType.CLUSTER).keys).isEmpty() } private fun Snapshot.hasVirtualHostConfig(name: String, idleTimeout: String, requestTimeout: String): Snapshot { @@ -740,7 +743,7 @@ class SnapshotUpdaterTest { ) private fun snapshotUpdater( - cache: SnapshotCache, + cache: SnapshotCache, properties: SnapshotProperties = SnapshotProperties(), groups: List = emptyList(), groupSnapshotScheduler: ParallelizableScheduler = DirectScheduler diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersionsTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersionsTest.kt index 5e2da3f7d..e3a9def2a 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersionsTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/SnapshotsVersionsTest.kt @@ -1,12 +1,12 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.core.Address -import io.envoyproxy.envoy.api.v2.core.SocketAddress -import io.envoyproxy.envoy.api.v2.endpoint.Endpoint -import io.envoyproxy.envoy.api.v2.endpoint.LbEndpoint -import io.envoyproxy.envoy.api.v2.endpoint.LocalityLbEndpoints +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.core.v3.Address +import io.envoyproxy.envoy.config.core.v3.SocketAddress +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.endpoint.v3.Endpoint +import io.envoyproxy.envoy.config.endpoint.v3.LbEndpoint +import io.envoyproxy.envoy.config.endpoint.v3.LocalityLbEndpoints import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.Test import pl.allegro.tech.servicemesh.envoycontrol.groups.AllServicesGroup diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactoryTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactoryTest.kt index 7e3cf8478..dde981914 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactoryTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterFactoryTest.kt @@ -3,14 +3,14 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.fil import com.google.protobuf.Any import com.google.protobuf.util.JsonFormat import io.envoyproxy.controlplane.cache.SnapshotResources -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.core.Address -import io.envoyproxy.envoy.api.v2.core.SocketAddress -import io.envoyproxy.envoy.api.v2.endpoint.Endpoint -import io.envoyproxy.envoy.api.v2.endpoint.LbEndpoint -import io.envoyproxy.envoy.api.v2.endpoint.LocalityLbEndpoints -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpFilter -import io.envoyproxy.envoy.config.filter.http.rbac.v2.RBAC as RBACFilter +import io.envoyproxy.envoy.config.core.v3.Address +import io.envoyproxy.envoy.config.core.v3.SocketAddress +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.endpoint.v3.Endpoint +import io.envoyproxy.envoy.config.endpoint.v3.LbEndpoint +import io.envoyproxy.envoy.config.endpoint.v3.LocalityLbEndpoints +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpFilter +import io.envoyproxy.envoy.extensions.filters.http.rbac.v3.RBAC as RBACFilter import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.Test import pl.allegro.tech.servicemesh.envoycontrol.groups.ClientWithSelector @@ -65,28 +65,30 @@ internal class RBACFilterFactoryTest { StatusRouteProperties() ) private val rbacFilterFactoryWithSourceIpWithSelectorAuth = RBACFilterFactory( - IncomingPermissionsProperties().also { - it.enabled = true - it.sourceIpAuthentication = SourceIpAuthenticationProperties().also { ipProperties -> - ipProperties.ipFromServiceDiscovery.enabledForIncomingServices = listOf("client1") - ipProperties.ipFromRange = mutableMapOf( - "client2" to setOf("192.168.1.0/24", "192.168.2.0/28") - ) - } - it.selectorMatching = mutableMapOf( - "client1" to SelectorMatching().also { it.header = "x-secret-header" }, - "client2" to SelectorMatching().also { it.header = "x-secret-header" } + IncomingPermissionsProperties().also { + it.enabled = true + it.sourceIpAuthentication = SourceIpAuthenticationProperties().also { ipProperties -> + ipProperties.ipFromServiceDiscovery.enabledForIncomingServices = listOf("client1") + ipProperties.ipFromRange = mutableMapOf( + "client2" to setOf("192.168.1.0/24", "192.168.2.0/28") ) - }, - StatusRouteProperties() + } + it.selectorMatching = mutableMapOf( + "client1" to SelectorMatching().also { it.header = "x-secret-header" }, + "client2" to SelectorMatching().also { it.header = "x-secret-header" } + ) + }, + StatusRouteProperties() ) val snapshot = GlobalSnapshot( - SnapshotResources.create(listOf(), ""), - setOf(), - SnapshotResources.create(listOf(), ""), - mapOf(), - SnapshotResources.create(listOf(), "") + SnapshotResources.create(listOf(), ""), + setOf(), + SnapshotResources.create(listOf(), ""), + mapOf(), + SnapshotResources.create(listOf(), ""), + SnapshotResources.create(listOf(), ""), + SnapshotResources.create(listOf(), "") ) val clusterLoadAssignment = ClusterLoadAssignment.newBuilder() @@ -104,11 +106,13 @@ internal class RBACFilterFactoryTest { ).build() val snapshotForSourceIpAuth = GlobalSnapshot( - SnapshotResources.create(listOf(), ""), - setOf(), - SnapshotResources.create(listOf(clusterLoadAssignment), ""), - mapOf(), - SnapshotResources.create(listOf(), "") + SnapshotResources.create(listOf(), ""), + setOf(), + SnapshotResources.create(listOf(clusterLoadAssignment), ""), + mapOf(), + SnapshotResources.create(listOf(), ""), + SnapshotResources.create(listOf(), ""), + SnapshotResources.create(listOf(), "") ) @Test diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissionsTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissionsTest.kt index 3c76a3cfe..600e91bbe 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissionsTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/listeners/filters/RBACFilterPermissionsTest.kt @@ -1,7 +1,7 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.resource.listeners.filters import com.google.protobuf.util.JsonFormat -import io.envoyproxy.envoy.config.rbac.v2.Permission +import io.envoyproxy.envoy.config.rbac.v3.Permission import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.Test import pl.allegro.tech.servicemesh.envoycontrol.groups.IncomingEndpoint diff --git a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactoryTest.kt b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactoryTest.kt index 5f85ebf2b..249911d31 100644 --- a/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactoryTest.kt +++ b/envoy-control-core/src/test/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/resource/routes/EnvoyEgressRoutesFactoryTest.kt @@ -6,8 +6,8 @@ import pl.allegro.tech.servicemesh.envoycontrol.groups.DependencySettings import pl.allegro.tech.servicemesh.envoycontrol.groups.Outgoing import pl.allegro.tech.servicemesh.envoycontrol.groups.hasCustomIdleTimeout import pl.allegro.tech.servicemesh.envoycontrol.groups.hasCustomRequestTimeout -import pl.allegro.tech.servicemesh.envoycontrol.groups.autoHostRewriteHeaderIsEmpty -import pl.allegro.tech.servicemesh.envoycontrol.groups.hasAutoHostRewriteHeader +import pl.allegro.tech.servicemesh.envoycontrol.groups.hostRewriteHeaderIsEmpty +import pl.allegro.tech.servicemesh.envoycontrol.groups.hasHostRewriteHeader import pl.allegro.tech.servicemesh.envoycontrol.groups.hasRequestHeaderToAdd import pl.allegro.tech.servicemesh.envoycontrol.groups.hasNoRequestHeaderToAdd import pl.allegro.tech.servicemesh.envoycontrol.groups.hasResponseHeaderToAdd @@ -116,7 +116,7 @@ internal class EnvoyEgressRoutesFactoryTest { .virtualHostsList[0] .routesList[0] .route - .autoHostRewriteHeaderIsEmpty() + .hostRewriteHeaderIsEmpty() } @Test @@ -136,6 +136,6 @@ internal class EnvoyEgressRoutesFactoryTest { .virtualHostsList[0] .routesList[0] .route - .hasAutoHostRewriteHeader(snapshotProperties.egress.hostHeaderRewriting.customHostHeader) + .hasHostRewriteHeader(snapshotProperties.egress.hostHeaderRewriting.customHostHeader) } } diff --git a/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugController.kt b/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugController.kt index e122fe7e0..dd84d64af 100644 --- a/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugController.kt +++ b/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugController.kt @@ -13,13 +13,12 @@ import com.google.protobuf.util.JsonFormat import com.google.protobuf.util.JsonFormat.TypeRegistry import io.envoyproxy.controlplane.cache.NodeGroup import io.envoyproxy.controlplane.cache.SnapshotCache -import io.envoyproxy.envoy.api.v2.auth.UpstreamTlsContext -import io.envoyproxy.envoy.api.v2.core.Node -import io.envoyproxy.envoy.config.filter.http.header_to_metadata.v2.Config -import io.envoyproxy.envoy.config.filter.http.lua.v2.Lua -import io.envoyproxy.envoy.config.filter.http.rbac.v2.RBAC as FilterRBAC -import io.envoyproxy.envoy.config.rbac.v2.RBAC -import io.envoyproxy.envoy.config.filter.network.http_connection_manager.v2.HttpConnectionManager +import io.envoyproxy.controlplane.cache.v3.Snapshot +import io.envoyproxy.envoy.config.rbac.v3.RBAC +import io.envoyproxy.envoy.extensions.filters.http.header_to_metadata.v3.Config +import io.envoyproxy.envoy.extensions.filters.http.lua.v3.Lua +import io.envoyproxy.envoy.extensions.filters.network.http_connection_manager.v3.HttpConnectionManager +import io.envoyproxy.envoy.extensions.transport_sockets.tls.v3.UpstreamTlsContext import io.envoyproxy.envoy.type.matcher.PathMatcher import io.envoyproxy.envoy.type.matcher.StringMatcher import org.springframework.boot.jackson.JsonComponent @@ -35,10 +34,13 @@ import org.springframework.web.bind.annotation.RestController import pl.allegro.tech.servicemesh.envoycontrol.ControlPlane import pl.allegro.tech.servicemesh.envoycontrol.groups.Group import pl.allegro.tech.servicemesh.envoycontrol.snapshot.SnapshotUpdater +import io.envoyproxy.envoy.api.v2.core.Node as NodeV2 +import io.envoyproxy.envoy.config.core.v3.Node as NodeV3 +import io.envoyproxy.envoy.extensions.filters.http.rbac.v3.RBAC as RBACFilter @RestController class SnapshotDebugController(controlPlane: ControlPlane) { - val cache: SnapshotCache = controlPlane.cache + val cache: SnapshotCache = controlPlane.cache val nodeGroup: NodeGroup = controlPlane.nodeGroup val snapshotUpdater: SnapshotUpdater = controlPlane.snapshotUpdater @@ -47,8 +49,23 @@ class SnapshotDebugController(controlPlane: ControlPlane) { * It contains the versions of XDS resources and the contents for a provided node JSON * extracted from Envoy's config_dump endpoint. */ - @PostMapping("/snapshot") - fun snapshot(@RequestBody node: Node): ResponseEntity { + @PostMapping(value = ["/snapshot"], consumes = ["application/json"], produces = ["application/v2+json", + "application/json"]) + fun snapshot(@RequestBody node: NodeV2): ResponseEntity { + val nodeHash = nodeGroup.hash(node) + val snapshot = cache.getSnapshot(nodeHash) + return if (snapshot == null) { + throw SnapshotNotFoundException() + } else { + ResponseEntity( + SnapshotDebugInfo(snapshot), + HttpStatus.OK + ) + } + } + + @PostMapping(value = ["/snapshot"], consumes = ["application/json"], produces = ["application/v3+json"]) + fun snapshot(@RequestBody node: NodeV3): ResponseEntity { val nodeHash = nodeGroup.hash(node) val snapshot = cache.getSnapshot(nodeHash) return if (snapshot == null) { @@ -94,7 +111,7 @@ class SnapshotDebugController(controlPlane: ControlPlane) { .add(Struct.getDescriptor()) .add(Value.getDescriptor()) .add(RBAC.getDescriptor()) - .add(FilterRBAC.getDescriptor()) + .add(RBACFilter.getDescriptor()) .add(Any.getDescriptor()) .add(PathMatcher.getDescriptor()) .add(StringMatcher.getDescriptor()) diff --git a/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugInfo.kt b/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugInfo.kt index fa78d8ff9..c554d8c32 100644 --- a/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugInfo.kt +++ b/envoy-control-runner/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/snapshot/debug/SnapshotDebugInfo.kt @@ -1,9 +1,9 @@ package pl.allegro.tech.servicemesh.envoycontrol.snapshot.debug -import io.envoyproxy.envoy.api.v2.Cluster -import io.envoyproxy.envoy.api.v2.ClusterLoadAssignment -import io.envoyproxy.envoy.api.v2.Listener -import io.envoyproxy.envoy.api.v2.RouteConfiguration +import io.envoyproxy.envoy.config.cluster.v3.Cluster +import io.envoyproxy.envoy.config.endpoint.v3.ClusterLoadAssignment +import io.envoyproxy.envoy.config.listener.v3.Listener +import io.envoyproxy.envoy.config.route.v3.RouteConfiguration import net.openhft.hashing.LongHashFunction import pl.allegro.tech.servicemesh.envoycontrol.snapshot.GlobalSnapshot @@ -28,8 +28,8 @@ data class Versions( ) data class Snapshot( - val clusters: Map, - val endpoints: Map, + val clusters: Map = emptyMap(), + val endpoints: Map = emptyMap(), val listeners: Map = emptyMap(), val routes: Map = emptyMap() ) @@ -38,7 +38,7 @@ data class SnapshotDebugInfo( val snapshot: Snapshot, val versions: Versions ) { - constructor(snapshot: io.envoyproxy.controlplane.cache.Snapshot) : this( + constructor(snapshot: io.envoyproxy.controlplane.cache.v3.Snapshot) : this( snapshot = Snapshot( clusters = snapshot.clusters().resources(), endpoints = snapshot.endpoints().resources(), diff --git a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoyControlV2toV3MigrationSmokeTest.kt b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoyControlV2toV3MigrationSmokeTest.kt new file mode 100644 index 000000000..7f47fb7ec --- /dev/null +++ b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/EnvoyControlV2toV3MigrationSmokeTest.kt @@ -0,0 +1,57 @@ +package pl.allegro.tech.servicemesh.envoycontrol + +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.extension.RegisterExtension +import pl.allegro.tech.servicemesh.envoycontrol.assertions.isFrom +import pl.allegro.tech.servicemesh.envoycontrol.assertions.isOk +import pl.allegro.tech.servicemesh.envoycontrol.assertions.untilAsserted +import pl.allegro.tech.servicemesh.envoycontrol.config.Ads +import pl.allegro.tech.servicemesh.envoycontrol.config.AdsV3 +import pl.allegro.tech.servicemesh.envoycontrol.config.consul.ConsulExtension +import pl.allegro.tech.servicemesh.envoycontrol.config.envoy.EnvoyExtension +import pl.allegro.tech.servicemesh.envoycontrol.config.envoycontrol.EnvoyControlExtension +import pl.allegro.tech.servicemesh.envoycontrol.config.service.EchoServiceExtension + +class EnvoyControlV2toV3MigrationSmokeTest { + + companion object { + + @JvmField + @RegisterExtension + val consul = ConsulExtension() + + @JvmField + @RegisterExtension + val envoyControl = EnvoyControlExtension(consul) + + @JvmField + @RegisterExtension + val serviceEnvoyV2 = EchoServiceExtension() + + @JvmField + @RegisterExtension + val serviceEnvoyV3 = EchoServiceExtension() + + @JvmField + @RegisterExtension + val envoyV2 = EnvoyExtension(envoyControl, serviceEnvoyV2, Ads) + + @JvmField + @RegisterExtension + val envoyV3 = EnvoyExtension(envoyControl, serviceEnvoyV3, AdsV3) + } + + @Test + fun `should create a server listening on a port`() { + untilAsserted { + // when + val ingressRootEnvoyV2 = envoyV2.ingressOperations.callLocalService("") + val ingressRootEnvoyV3 = envoyV3.ingressOperations.callLocalService("") + + // then + assertThat(ingressRootEnvoyV2).isFrom(serviceEnvoyV2).isOk() + assertThat(ingressRootEnvoyV3).isFrom(serviceEnvoyV3).isOk() + } + } +} diff --git a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/SnapshotDebugTest.kt b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/SnapshotDebugTest.kt index 316bb7f66..f9fe4bfa1 100644 --- a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/SnapshotDebugTest.kt +++ b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/SnapshotDebugTest.kt @@ -84,8 +84,7 @@ open class SnapshotDebugTest : EnvoyControlTestConfiguration() { }, "locality": { "zone": "dev-dc4" - }, - "build_version": "b7bef67c256090919a4585a1a06c42f15d640a09/1.13.0-dev/Clean/RELEASE/BoringSSL" + } } """.trim() diff --git a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/EnvoyControlTestConfiguration.kt b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/EnvoyControlTestConfiguration.kt index c523d5cfc..97ecf2ee5 100644 --- a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/EnvoyControlTestConfiguration.kt +++ b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/EnvoyControlTestConfiguration.kt @@ -37,6 +37,7 @@ val AdsAllDependencies = EnvoyConfig("envoy/config_ads_all_dependencies.yaml") val AdsCustomHealthCheck = EnvoyConfig("envoy/config_ads_custom_health_check.yaml") val FaultyConfig = EnvoyConfig("envoy/bad_config.yaml") val Ads = EnvoyConfig("envoy/config_ads.yaml") +val AdsV3 = EnvoyConfig("envoy/config_ads_v3.yaml") val Echo1EnvoyAuthConfig = EnvoyConfig("envoy/config_auth.yaml") val Echo2EnvoyAuthConfig = Echo1EnvoyAuthConfig.copy( serviceName = "echo2", diff --git a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/envoycontrol/EnvoyControlTestApp.kt b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/envoycontrol/EnvoyControlTestApp.kt index 9c63f0f41..25d0c9cb3 100644 --- a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/envoycontrol/EnvoyControlTestApp.kt +++ b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/config/envoycontrol/EnvoyControlTestApp.kt @@ -109,6 +109,7 @@ class EnvoyControlRunnerTestApp( override fun getSnapshot(nodeJson: String): SnapshotDebugResponse { val response = httpClient.newCall( Request.Builder() + .addHeader("Accept", "application/v2+json") .post(RequestBody.create(MediaType.get("application/json"), nodeJson)) .url("http://localhost:$appPort/snapshot") .build() diff --git a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ssl/EnvoyHttpsDependencyTest.kt b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ssl/EnvoyHttpsDependencyTest.kt index 4d6e2e0d4..d3a3e7178 100644 --- a/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ssl/EnvoyHttpsDependencyTest.kt +++ b/envoy-control-tests/src/main/kotlin/pl/allegro/tech/servicemesh/envoycontrol/ssl/EnvoyHttpsDependencyTest.kt @@ -21,22 +21,6 @@ class EnvoyCurrentVersionHttpsDependencyTest : EnvoyHttpsDependencyTest() { } } -// TODO(https://github.com/allegro/envoy-control/issues/97) - remove when envoy < 1.14.0-dev will be not supported -class EnvoyCompatibleVersionHttpsDependencyTest : EnvoyHttpsDependencyTest() { - companion object { - @JvmStatic - @BeforeAll - fun setupTest() { - setup( - appFactoryForEc1 = { consulPort -> EnvoyControlRunnerTestApp(properties, consulPort) }, - // 1.13.0-dev - envoyImage = "envoyproxy/envoy-alpine-dev:b7bef67c256090919a4585a1a06c42f15d640a09" - ) - setupTestCommon() - } - } -} - abstract class EnvoyHttpsDependencyTest : EnvoyControlTestConfiguration() { companion object { @JvmStatic diff --git a/envoy-control-tests/src/main/resources/envoy/bad_config.yaml b/envoy-control-tests/src/main/resources/envoy/bad_config.yaml index f1c1accab..ea9c65a3a 100644 --- a/envoy-control-tests/src/main/resources/envoy/bad_config.yaml +++ b/envoy-control-tests/src/main/resources/envoy/bad_config.yaml @@ -87,7 +87,7 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router + - name: envoy.filters.http.router - name: ingress_listener address: socket_address: @@ -104,4 +104,4 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router \ No newline at end of file + - name: envoy.filters.http.router \ No newline at end of file diff --git a/envoy-control-tests/src/main/resources/envoy/config_ads_custom_health_check.yaml b/envoy-control-tests/src/main/resources/envoy/config_ads_custom_health_check.yaml index f5f608982..b180b6deb 100644 --- a/envoy-control-tests/src/main/resources/envoy/config_ads_custom_health_check.yaml +++ b/envoy-control-tests/src/main/resources/envoy/config_ads_custom_health_check.yaml @@ -97,7 +97,7 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router + - name: envoy.filters.http.router - name: ingress_listener address: socket_address: @@ -114,4 +114,4 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router \ No newline at end of file + - name: envoy.filters.http.router \ No newline at end of file diff --git a/envoy-control-tests/src/main/resources/envoy/config_ads_static_listeners.yaml b/envoy-control-tests/src/main/resources/envoy/config_ads_static_listeners.yaml index 6a5d3b752..dc13b700b 100644 --- a/envoy-control-tests/src/main/resources/envoy/config_ads_static_listeners.yaml +++ b/envoy-control-tests/src/main/resources/envoy/config_ads_static_listeners.yaml @@ -82,7 +82,7 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router + - name: envoy.filters.http.router - name: ingress_listener address: socket_address: @@ -99,4 +99,4 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router \ No newline at end of file + - name: envoy.filters.http.router \ No newline at end of file diff --git a/envoy-control-tests/src/main/resources/envoy/config_ads_v3.yaml b/envoy-control-tests/src/main/resources/envoy/config_ads_v3.yaml new file mode 100644 index 000000000..aafca81f1 --- /dev/null +++ b/envoy-control-tests/src/main/resources/envoy/config_ads_v3.yaml @@ -0,0 +1,116 @@ +admin: + access_log_path: /dev/null + address: + socket_address: { address: 0.0.0.0, port_value: 10000 } +dynamic_resources: + lds_config: + resource_api_version: V3 + ads: {} + cds_config: + resource_api_version: V3 + ads: {} + ads_config: + transport_api_version: V3 + api_type: GRPC + grpc_services: + envoy_grpc: + cluster_name: envoy-control-xds +node: + cluster: test-cluster + id: test-id + metadata: + service_name: "echo2" + ads: true + ingress_host: "0.0.0.0" + ingress_port: 5001 + egress_host: "0.0.0.0" + egress_port: 5000 + use_remote_address: true + access_log_enabled: false + add_upstream_external_address_header: true + resources_dir: "/etc/envoy/extra" + proxy_settings: + incoming: + endpoints: + - path: "/endpoint" + clients: ["authorizedClient"] + - path: "/secured_endpoint" + clients: ["echo"] + outgoing: + dependencies: + - service: "service-1" + - service: "service-2" + - service: "service-3" + - service: "service-4" + - service: "service-5" + - service: "echo" + timeoutPolicy: + requestTimeout: "15s" + - service: "consul" + timeoutPolicy: + requestTimeout: "15s" + - service: "proxy1" + - service: "proxy2" + - service: "service-redirect" + handleInternalRedirect: true + - service: "host-rewrite-service" + rewriteHostHeader: true + - domain: "https://my.example.com" + - domain: "https://bad.host.example.com" + - domain: "https://www.example.com" + - domain: "https://www.example-redirect.com" + handleInternalRedirect: true + +static_resources: + clusters: + - connect_timeout: 1s + load_assignment: + cluster_name: envoy-control-xds + endpoints: + - lb_endpoints: + - endpoint: + address: + socket_address: + address: HOST_IP + port_value: HOST_PORT + - endpoint: + address: + socket_address: + address: HOST_IP + port_value: HOST2_PORT + http2_protocol_options: {} + name: envoy-control-xds + - name: envoy-original-destination + type: ORIGINAL_DST + lb_policy: CLUSTER_PROVIDED + original_dst_lb_config: + use_http_header: true + connect_timeout: + seconds: 1 + http_protocol_options: + allow_absolute_url: true + - name: local_service + type: STATIC + load_assignment: + cluster_name: local_service + endpoints: + - lb_endpoints: + - endpoint: + address: + socket_address: + address: LOCAL_SERVICE_IP + port_value: 5678 + connect_timeout: 1s + - name: this_admin + type: STATIC + load_assignment: + cluster_name: this_admin + endpoints: + - lb_endpoints: + - endpoint: + address: + socket_address: + address: 127.0.0.1 + port_value: 10000 + connect_timeout: + seconds: 1 \ No newline at end of file diff --git a/tools/envoy/envoy-template.yaml b/tools/envoy/envoy-template.yaml index 8e5f1c7cf..76ce3d0ca 100644 --- a/tools/envoy/envoy-template.yaml +++ b/tools/envoy/envoy-template.yaml @@ -27,7 +27,7 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router + - name: envoy.filters.http.router http_protocol_options: allow_absolute_url: true access_log: @@ -52,7 +52,7 @@ static_resources: config_source: ads: {} http_filters: - - name: envoy.router + - name: envoy.filters.http.router clusters: - name: envoy-control-xds