proxy: rename console -> control_plane, rename web -> console_redirect (#9266)

rename console -> control_plane
rename web -> console_redirect

I think these names are a little more representative.
This commit is contained in:
Conrad Ludgate
2024-10-07 14:09:54 +01:00
committed by GitHub
parent 47c3c9a413
commit 8cd7b5bf54
28 changed files with 186 additions and 172 deletions

View File

@@ -18,7 +18,7 @@ pub(crate) use flow::*;
use tokio::time::error::Elapsed;
use crate::{
console,
control_plane,
error::{ReportableError, UserFacingError},
};
use std::{io, net::IpAddr};
@@ -34,7 +34,7 @@ pub(crate) enum AuthErrorImpl {
Web(#[from] backend::WebAuthError),
#[error(transparent)]
GetAuthInfo(#[from] console::errors::GetAuthInfoError),
GetAuthInfo(#[from] control_plane::errors::GetAuthInfoError),
/// SASL protocol errors (includes [SCRAM](crate::scram)).
#[error(transparent)]

View File

@@ -1,27 +1,27 @@
mod classic;
mod console_redirect;
mod hacks;
pub mod jwt;
pub mod local;
mod web;
use std::net::IpAddr;
use std::sync::Arc;
use std::time::Duration;
pub(crate) use console_redirect::WebAuthError;
use ipnet::{Ipv4Net, Ipv6Net};
use local::LocalBackend;
use tokio::io::{AsyncRead, AsyncWrite};
use tokio_postgres::config::AuthKeys;
use tracing::{info, warn};
pub(crate) use web::WebAuthError;
use crate::auth::credentials::check_peer_addr_is_in_list;
use crate::auth::{validate_password_and_exchange, AuthError};
use crate::cache::Cached;
use crate::console::errors::GetAuthInfoError;
use crate::console::provider::{CachedRoleSecret, ConsoleBackend};
use crate::console::{AuthSecret, NodeInfo};
use crate::context::RequestMonitoring;
use crate::control_plane::errors::GetAuthInfoError;
use crate::control_plane::provider::{CachedRoleSecret, ControlPlaneBackend};
use crate::control_plane::{AuthSecret, NodeInfo};
use crate::intern::EndpointIdInt;
use crate::metrics::Metrics;
use crate::proxy::connect_compute::ComputeConnectBackend;
@@ -31,7 +31,7 @@ use crate::stream::Stream;
use crate::{
auth::{self, ComputeUserInfoMaybeEndpoint},
config::AuthenticationConfig,
console::{
control_plane::{
self,
provider::{CachedAllowedIps, CachedNodeInfo},
Api,
@@ -67,19 +67,19 @@ impl<T> std::ops::Deref for MaybeOwned<'_, T> {
/// backends which require them for the authentication process.
pub enum Backend<'a, T, D> {
/// Cloud API (V2).
Console(MaybeOwned<'a, ConsoleBackend>, T),
ControlPlane(MaybeOwned<'a, ControlPlaneBackend>, T),
/// Authentication via a web browser.
Web(MaybeOwned<'a, url::ApiUrl>, D),
ConsoleRedirect(MaybeOwned<'a, url::ApiUrl>, D),
/// Local proxy uses configured auth credentials and does not wake compute
Local(MaybeOwned<'a, LocalBackend>),
}
#[cfg(test)]
pub(crate) trait TestBackend: Send + Sync + 'static {
fn wake_compute(&self) -> Result<CachedNodeInfo, console::errors::WakeComputeError>;
fn wake_compute(&self) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError>;
fn get_allowed_ips_and_secret(
&self,
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), console::errors::GetAuthInfoError>;
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), control_plane::errors::GetAuthInfoError>;
fn dyn_clone(&self) -> Box<dyn TestBackend>;
}
@@ -93,18 +93,23 @@ impl Clone for Box<dyn TestBackend> {
impl std::fmt::Display for Backend<'_, (), ()> {
fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Console(api, ()) => match &**api {
ConsoleBackend::Console(endpoint) => {
fmt.debug_tuple("Console").field(&endpoint.url()).finish()
}
Self::ControlPlane(api, ()) => match &**api {
ControlPlaneBackend::Management(endpoint) => fmt
.debug_tuple("ControlPlane::Management")
.field(&endpoint.url())
.finish(),
#[cfg(any(test, feature = "testing"))]
ConsoleBackend::Postgres(endpoint) => {
fmt.debug_tuple("Postgres").field(&endpoint.url()).finish()
}
ControlPlaneBackend::PostgresMock(endpoint) => fmt
.debug_tuple("ControlPlane::PostgresMock")
.field(&endpoint.url())
.finish(),
#[cfg(test)]
ConsoleBackend::Test(_) => fmt.debug_tuple("Test").finish(),
ControlPlaneBackend::Test(_) => fmt.debug_tuple("ControlPlane::Test").finish(),
},
Self::Web(url, ()) => fmt.debug_tuple("Web").field(&url.as_str()).finish(),
Self::ConsoleRedirect(url, ()) => fmt
.debug_tuple("ConsoleRedirect")
.field(&url.as_str())
.finish(),
Self::Local(_) => fmt.debug_tuple("Local").finish(),
}
}
@@ -115,8 +120,8 @@ impl<T, D> Backend<'_, T, D> {
/// This helps us pass structured config to async tasks.
pub(crate) fn as_ref(&self) -> Backend<'_, &T, &D> {
match self {
Self::Console(c, x) => Backend::Console(MaybeOwned::Borrowed(c), x),
Self::Web(c, x) => Backend::Web(MaybeOwned::Borrowed(c), x),
Self::ControlPlane(c, x) => Backend::ControlPlane(MaybeOwned::Borrowed(c), x),
Self::ConsoleRedirect(c, x) => Backend::ConsoleRedirect(MaybeOwned::Borrowed(c), x),
Self::Local(l) => Backend::Local(MaybeOwned::Borrowed(l)),
}
}
@@ -128,8 +133,8 @@ impl<'a, T, D> Backend<'a, T, D> {
/// a function to a contained value.
pub(crate) fn map<R>(self, f: impl FnOnce(T) -> R) -> Backend<'a, R, D> {
match self {
Self::Console(c, x) => Backend::Console(c, f(x)),
Self::Web(c, x) => Backend::Web(c, x),
Self::ControlPlane(c, x) => Backend::ControlPlane(c, f(x)),
Self::ConsoleRedirect(c, x) => Backend::ConsoleRedirect(c, x),
Self::Local(l) => Backend::Local(l),
}
}
@@ -139,8 +144,8 @@ impl<'a, T, D, E> Backend<'a, Result<T, E>, D> {
/// This is most useful for error handling.
pub(crate) fn transpose(self) -> Result<Backend<'a, T, D>, E> {
match self {
Self::Console(c, x) => x.map(|x| Backend::Console(c, x)),
Self::Web(c, x) => Ok(Backend::Web(c, x)),
Self::ControlPlane(c, x) => x.map(|x| Backend::ControlPlane(c, x)),
Self::ConsoleRedirect(c, x) => Ok(Backend::ConsoleRedirect(c, x)),
Self::Local(l) => Ok(Backend::Local(l)),
}
}
@@ -290,7 +295,7 @@ impl AuthenticationConfig {
/// All authentication flows will emit an AuthenticationOk message if successful.
async fn auth_quirks(
ctx: &RequestMonitoring,
api: &impl console::Api,
api: &impl control_plane::Api,
user_info: ComputeUserInfoMaybeEndpoint,
client: &mut stream::PqStream<Stream<impl AsyncRead + AsyncWrite + Unpin>>,
allow_cleartext: bool,
@@ -412,8 +417,8 @@ impl<'a> Backend<'a, ComputeUserInfoMaybeEndpoint, &()> {
/// Get username from the credentials.
pub(crate) fn get_user(&self) -> &str {
match self {
Self::Console(_, user_info) => &user_info.user,
Self::Web(_, ()) => "web",
Self::ControlPlane(_, user_info) => &user_info.user,
Self::ConsoleRedirect(_, ()) => "web",
Self::Local(_) => "local",
}
}
@@ -429,7 +434,7 @@ impl<'a> Backend<'a, ComputeUserInfoMaybeEndpoint, &()> {
endpoint_rate_limiter: Arc<EndpointRateLimiter>,
) -> auth::Result<Backend<'a, ComputeCredentials, NodeInfo>> {
let res = match self {
Self::Console(api, user_info) => {
Self::ControlPlane(api, user_info) => {
info!(
user = &*user_info.user,
project = user_info.endpoint(),
@@ -446,15 +451,15 @@ impl<'a> Backend<'a, ComputeUserInfoMaybeEndpoint, &()> {
endpoint_rate_limiter,
)
.await?;
Backend::Console(api, credentials)
Backend::ControlPlane(api, credentials)
}
// NOTE: this auth backend doesn't use client credentials.
Self::Web(url, ()) => {
Self::ConsoleRedirect(url, ()) => {
info!("performing web authentication");
let info = web::authenticate(ctx, config, &url, client).await?;
let info = console_redirect::authenticate(ctx, config, &url, client).await?;
Backend::Web(url, info)
Backend::ConsoleRedirect(url, info)
}
Self::Local(_) => {
return Err(auth::AuthError::bad_auth_method("invalid for local proxy"))
@@ -472,8 +477,8 @@ impl Backend<'_, ComputeUserInfo, &()> {
ctx: &RequestMonitoring,
) -> Result<CachedRoleSecret, GetAuthInfoError> {
match self {
Self::Console(api, user_info) => api.get_role_secret(ctx, user_info).await,
Self::Web(_, ()) => Ok(Cached::new_uncached(None)),
Self::ControlPlane(api, user_info) => api.get_role_secret(ctx, user_info).await,
Self::ConsoleRedirect(_, ()) => Ok(Cached::new_uncached(None)),
Self::Local(_) => Ok(Cached::new_uncached(None)),
}
}
@@ -483,8 +488,10 @@ impl Backend<'_, ComputeUserInfo, &()> {
ctx: &RequestMonitoring,
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), GetAuthInfoError> {
match self {
Self::Console(api, user_info) => api.get_allowed_ips_and_secret(ctx, user_info).await,
Self::Web(_, ()) => Ok((Cached::new_uncached(Arc::new(vec![])), None)),
Self::ControlPlane(api, user_info) => {
api.get_allowed_ips_and_secret(ctx, user_info).await
}
Self::ConsoleRedirect(_, ()) => Ok((Cached::new_uncached(Arc::new(vec![])), None)),
Self::Local(_) => Ok((Cached::new_uncached(Arc::new(vec![])), None)),
}
}
@@ -495,18 +502,18 @@ impl ComputeConnectBackend for Backend<'_, ComputeCredentials, NodeInfo> {
async fn wake_compute(
&self,
ctx: &RequestMonitoring,
) -> Result<CachedNodeInfo, console::errors::WakeComputeError> {
) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError> {
match self {
Self::Console(api, creds) => api.wake_compute(ctx, &creds.info).await,
Self::Web(_, info) => Ok(Cached::new_uncached(info.clone())),
Self::ControlPlane(api, creds) => api.wake_compute(ctx, &creds.info).await,
Self::ConsoleRedirect(_, info) => Ok(Cached::new_uncached(info.clone())),
Self::Local(local) => Ok(Cached::new_uncached(local.node_info.clone())),
}
}
fn get_keys(&self) -> &ComputeCredentialKeys {
match self {
Self::Console(_, creds) => &creds.keys,
Self::Web(_, _) => &ComputeCredentialKeys::None,
Self::ControlPlane(_, creds) => &creds.keys,
Self::ConsoleRedirect(_, _) => &ComputeCredentialKeys::None,
Self::Local(_) => &ComputeCredentialKeys::None,
}
}
@@ -517,10 +524,10 @@ impl ComputeConnectBackend for Backend<'_, ComputeCredentials, &()> {
async fn wake_compute(
&self,
ctx: &RequestMonitoring,
) -> Result<CachedNodeInfo, console::errors::WakeComputeError> {
) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError> {
match self {
Self::Console(api, creds) => api.wake_compute(ctx, &creds.info).await,
Self::Web(_, ()) => {
Self::ControlPlane(api, creds) => api.wake_compute(ctx, &creds.info).await,
Self::ConsoleRedirect(_, ()) => {
unreachable!("web auth flow doesn't support waking the compute")
}
Self::Local(local) => Ok(Cached::new_uncached(local.node_info.clone())),
@@ -529,8 +536,8 @@ impl ComputeConnectBackend for Backend<'_, ComputeCredentials, &()> {
fn get_keys(&self) -> &ComputeCredentialKeys {
match self {
Self::Console(_, creds) => &creds.keys,
Self::Web(_, ()) => &ComputeCredentialKeys::None,
Self::ControlPlane(_, creds) => &creds.keys,
Self::ConsoleRedirect(_, ()) => &ComputeCredentialKeys::None,
Self::Local(_) => &ComputeCredentialKeys::None,
}
}
@@ -553,12 +560,12 @@ mod tests {
use crate::{
auth::{backend::MaskedIp, ComputeUserInfoMaybeEndpoint, IpPattern},
config::AuthenticationConfig,
console::{
context::RequestMonitoring,
control_plane::{
self,
provider::{self, CachedAllowedIps, CachedRoleSecret},
CachedNodeInfo,
},
context::RequestMonitoring,
proxy::NeonOptions,
rate_limiter::{EndpointRateLimiter, RateBucketInfo},
scram::{threadpool::ThreadPool, ServerSecret},
@@ -572,12 +579,12 @@ mod tests {
secret: AuthSecret,
}
impl console::Api for Auth {
impl control_plane::Api for Auth {
async fn get_role_secret(
&self,
_ctx: &RequestMonitoring,
_user_info: &super::ComputeUserInfo,
) -> Result<CachedRoleSecret, console::errors::GetAuthInfoError> {
) -> Result<CachedRoleSecret, control_plane::errors::GetAuthInfoError> {
Ok(CachedRoleSecret::new_uncached(Some(self.secret.clone())))
}
@@ -585,8 +592,10 @@ mod tests {
&self,
_ctx: &RequestMonitoring,
_user_info: &super::ComputeUserInfo,
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), console::errors::GetAuthInfoError>
{
) -> Result<
(CachedAllowedIps, Option<CachedRoleSecret>),
control_plane::errors::GetAuthInfoError,
> {
Ok((
CachedAllowedIps::new_uncached(Arc::new(self.ips.clone())),
Some(CachedRoleSecret::new_uncached(Some(self.secret.clone()))),
@@ -605,7 +614,7 @@ mod tests {
&self,
_ctx: &RequestMonitoring,
_user_info: &super::ComputeUserInfo,
) -> Result<CachedNodeInfo, console::errors::WakeComputeError> {
) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError> {
unimplemented!()
}
}

View File

@@ -3,8 +3,8 @@ use crate::{
auth::{self, backend::ComputeCredentialKeys, AuthFlow},
compute,
config::AuthenticationConfig,
console::AuthSecret,
context::RequestMonitoring,
control_plane::AuthSecret,
sasl,
stream::{PqStream, Stream},
};

View File

@@ -1,8 +1,8 @@
use crate::{
auth, compute,
config::AuthenticationConfig,
console::{self, provider::NodeInfo},
context::RequestMonitoring,
control_plane::{self, provider::NodeInfo},
error::{ReportableError, UserFacingError},
stream::PqStream,
waiters,
@@ -70,7 +70,7 @@ pub(super) async fn authenticate(
let (psql_session_id, waiter) = loop {
let psql_session_id = new_psql_session_id();
match console::mgmt::get_waiter(&psql_session_id) {
match control_plane::mgmt::get_waiter(&psql_session_id) {
Ok(waiter) => break (psql_session_id, waiter),
Err(_e) => continue,
}

View File

@@ -2,8 +2,8 @@ use super::{ComputeCredentials, ComputeUserInfo, ComputeUserInfoNoEndpoint};
use crate::{
auth::{self, AuthFlow},
config::AuthenticationConfig,
console::AuthSecret,
context::RequestMonitoring,
control_plane::AuthSecret,
intern::EndpointIdInt,
sasl,
stream::{self, Stream},

View File

@@ -5,11 +5,11 @@ use arc_swap::ArcSwapOption;
use crate::{
compute::ConnCfg,
console::{
context::RequestMonitoring,
control_plane::{
messages::{ColdStartInfo, EndpointJwksResponse, MetricsAuxInfo},
NodeInfo,
},
context::RequestMonitoring,
intern::{BranchIdTag, EndpointIdTag, InternId, ProjectIdTag},
EndpointId,
};

View File

@@ -3,8 +3,8 @@
use super::{backend::ComputeCredentialKeys, AuthErrorImpl, PasswordHackPayload};
use crate::{
config::TlsServerEndPoint,
console::AuthSecret,
context::RequestMonitoring,
control_plane::AuthSecret,
intern::EndpointIdInt,
sasl,
scram::{self, threadpool::ThreadPool},

View File

@@ -12,7 +12,7 @@ use proxy::{
},
cancellation::CancellationHandlerMain,
config::{self, AuthenticationConfig, HttpConfig, ProxyConfig, RetryConfig},
console::{
control_plane::{
locks::ApiLocks,
messages::{EndpointJwksResponse, JwksSettings},
},

View File

@@ -19,8 +19,8 @@ use proxy::config::CacheOptions;
use proxy::config::HttpConfig;
use proxy::config::ProjectInfoCacheOptions;
use proxy::config::ProxyProtocolV2;
use proxy::console;
use proxy::context::parquet::ParquetUploadArgs;
use proxy::control_plane;
use proxy::http;
use proxy::http::health_server::AppMetrics;
use proxy::metrics::Metrics;
@@ -495,7 +495,7 @@ async fn main() -> anyhow::Result<()> {
proxy: proxy::metrics::Metrics::get(),
},
));
maintenance_tasks.spawn(console::mgmt::task_main(mgmt_listener));
maintenance_tasks.spawn(control_plane::mgmt::task_main(mgmt_listener));
if let Some(metrics_config) = &config.metric_collection {
// TODO: Add gc regardles of the metric collection being enabled.
@@ -506,8 +506,8 @@ async fn main() -> anyhow::Result<()> {
));
}
if let auth::Backend::Console(api, _) = &config.auth_backend {
if let proxy::console::provider::ConsoleBackend::Console(api) = &**api {
if let auth::Backend::ControlPlane(api, _) = &config.auth_backend {
if let proxy::control_plane::provider::ControlPlaneBackend::Management(api) = &**api {
match (redis_notifications_client, regional_redis_client.clone()) {
(None, None) => {}
(client1, client2) => {
@@ -623,7 +623,7 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> {
"Using AllowedIpsCache (wake_compute) with options={project_info_cache_config:?}"
);
info!("Using EndpointCacheConfig with options={endpoint_cache_config:?}");
let caches = Box::leak(Box::new(console::caches::ApiCaches::new(
let caches = Box::leak(Box::new(control_plane::caches::ApiCaches::new(
wake_compute_cache_config,
project_info_cache_config,
endpoint_cache_config,
@@ -636,7 +636,7 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> {
timeout,
} = args.wake_compute_lock.parse()?;
info!(?limiter, shards, ?epoch, "Using NodeLocks (wake_compute)");
let locks = Box::leak(Box::new(console::locks::ApiLocks::new(
let locks = Box::leak(Box::new(control_plane::locks::ApiLocks::new(
"wake_compute_lock",
limiter,
shards,
@@ -653,27 +653,27 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> {
RateBucketInfo::validate(&mut wake_compute_rps_limit)?;
let wake_compute_endpoint_rate_limiter =
Arc::new(WakeComputeRateLimiter::new(wake_compute_rps_limit));
let api = console::provider::neon::Api::new(
let api = control_plane::provider::neon::Api::new(
endpoint,
caches,
locks,
wake_compute_endpoint_rate_limiter,
);
let api = console::provider::ConsoleBackend::Console(api);
auth::Backend::Console(MaybeOwned::Owned(api), ())
let api = control_plane::provider::ControlPlaneBackend::Management(api);
auth::Backend::ControlPlane(MaybeOwned::Owned(api), ())
}
AuthBackendType::Web => {
let url = args.uri.parse()?;
auth::Backend::Web(MaybeOwned::Owned(url), ())
auth::Backend::ConsoleRedirect(MaybeOwned::Owned(url), ())
}
#[cfg(feature = "testing")]
AuthBackendType::Postgres => {
let url = args.auth_endpoint.parse()?;
let api = console::provider::mock::Api::new(url, !args.is_private_access_proxy);
let api = console::provider::ConsoleBackend::Postgres(api);
auth::Backend::Console(MaybeOwned::Owned(api), ())
let api = control_plane::provider::mock::Api::new(url, !args.is_private_access_proxy);
let api = control_plane::provider::ControlPlaneBackend::PostgresMock(api);
auth::Backend::ControlPlane(MaybeOwned::Owned(api), ())
}
};
@@ -689,7 +689,7 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> {
?epoch,
"Using NodeLocks (connect_compute)"
);
let connect_compute_locks = console::locks::ApiLocks::new(
let connect_compute_locks = control_plane::locks::ApiLocks::new(
"connect_compute_lock",
limiter,
shards,

View File

@@ -16,7 +16,7 @@ use tracing::{debug, info};
use crate::{
auth::IpPattern,
config::ProjectInfoCacheOptions,
console::AuthSecret,
control_plane::AuthSecret,
intern::{EndpointIdInt, ProjectIdInt, RoleNameInt},
EndpointId, RoleName,
};

View File

@@ -1,8 +1,8 @@
use crate::{
auth::parse_endpoint_param,
cancellation::CancelClosure,
console::{errors::WakeComputeError, messages::MetricsAuxInfo, provider::ApiLockError},
context::RequestMonitoring,
control_plane::{errors::WakeComputeError, messages::MetricsAuxInfo, provider::ApiLockError},
error::{ReportableError, UserFacingError},
metrics::{Metrics, NumDbConnectionsGuard},
proxy::neon_option,

View File

@@ -3,7 +3,7 @@ use crate::{
self,
backend::{jwt::JwkCache, AuthRateLimiter},
},
console::locks::ApiLocks,
control_plane::locks::ApiLocks,
rate_limiter::{RateBucketInfo, RateLimitAlgorithm, RateLimiterConfig},
scram::threadpool::ThreadPool,
serverless::{cancel_set::CancelSet, GlobalConnPoolOptions},
@@ -372,7 +372,7 @@ pub struct EndpointCacheConfig {
}
impl EndpointCacheConfig {
/// Default options for [`crate::console::provider::NodeInfoCache`].
/// Default options for [`crate::control_plane::provider::NodeInfoCache`].
/// Notice that by default the limiter is empty, which means that cache is disabled.
pub const CACHE_DEFAULT_OPTIONS: &'static str =
"initial_batch_size=1000,default_batch_size=10,xread_timeout=5m,stream_name=controlPlane,disable_cache=true,limiter_info=1000@1s,retry_interval=1s";
@@ -447,7 +447,7 @@ pub struct CacheOptions {
}
impl CacheOptions {
/// Default options for [`crate::console::provider::NodeInfoCache`].
/// Default options for [`crate::control_plane::provider::NodeInfoCache`].
pub const CACHE_DEFAULT_OPTIONS: &'static str = "size=4000,ttl=4m";
/// Parse cache options passed via cmdline.
@@ -503,7 +503,7 @@ pub struct ProjectInfoCacheOptions {
}
impl ProjectInfoCacheOptions {
/// Default options for [`crate::console::provider::NodeInfoCache`].
/// Default options for [`crate::control_plane::provider::NodeInfoCache`].
pub const CACHE_DEFAULT_OPTIONS: &'static str =
"size=10000,ttl=4m,max_roles=10,gc_interval=60m";
@@ -622,9 +622,9 @@ pub struct ConcurrencyLockOptions {
}
impl ConcurrencyLockOptions {
/// Default options for [`crate::console::provider::ApiLocks`].
/// Default options for [`crate::control_plane::provider::ApiLocks`].
pub const DEFAULT_OPTIONS_WAKE_COMPUTE_LOCK: &'static str = "permits=0";
/// Default options for [`crate::console::provider::ApiLocks`].
/// Default options for [`crate::control_plane::provider::ApiLocks`].
pub const DEFAULT_OPTIONS_CONNECT_COMPUTE_LOCK: &'static str =
"shards=64,permits=100,epoch=10m,timeout=10ms";

View File

@@ -11,7 +11,7 @@ use try_lock::TryLock;
use uuid::Uuid;
use crate::{
console::messages::{ColdStartInfo, MetricsAuxInfo},
control_plane::messages::{ColdStartInfo, MetricsAuxInfo},
error::ErrorKind,
intern::{BranchIdInt, ProjectIdInt},
metrics::{ConnectOutcome, InvalidEndpointsGroup, LatencyTimer, Metrics, Protocol, Waiting},

View File

@@ -10,14 +10,14 @@ use crate::proxy::retry::CouldRetry;
/// Generic error response with human-readable description.
/// Note that we can't always present it to user as is.
#[derive(Debug, Deserialize, Clone)]
pub(crate) struct ConsoleError {
pub(crate) struct ControlPlaneError {
pub(crate) error: Box<str>,
#[serde(skip)]
pub(crate) http_status_code: http::StatusCode,
pub(crate) status: Option<Status>,
}
impl ConsoleError {
impl ControlPlaneError {
pub(crate) fn get_reason(&self) -> Reason {
self.status
.as_ref()
@@ -51,7 +51,7 @@ impl ConsoleError {
}
}
impl Display for ConsoleError {
impl Display for ControlPlaneError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let msg: &str = self
.status
@@ -62,7 +62,7 @@ impl Display for ConsoleError {
}
}
impl CouldRetry for ConsoleError {
impl CouldRetry for ControlPlaneError {
fn could_retry(&self) -> bool {
// If the error message does not have a status,
// the error is unknown and probably should not retry automatically

View File

@@ -1,5 +1,5 @@
use crate::{
console::messages::{DatabaseInfo, KickSession},
control_plane::messages::{DatabaseInfo, KickSession},
waiters::{self, Waiter, Waiters},
};
use anyhow::Context;

View File

@@ -2,7 +2,7 @@
pub mod mock;
pub mod neon;
use super::messages::{ConsoleError, MetricsAuxInfo};
use super::messages::{ControlPlaneError, MetricsAuxInfo};
use crate::{
auth::{
backend::{
@@ -28,7 +28,7 @@ use tracing::info;
pub(crate) mod errors {
use crate::{
console::messages::{self, ConsoleError, Reason},
control_plane::messages::{self, ControlPlaneError, Reason},
error::{io_error, ErrorKind, ReportableError, UserFacingError},
proxy::retry::CouldRetry,
};
@@ -44,7 +44,7 @@ pub(crate) mod errors {
pub(crate) enum ApiError {
/// Error returned by the console itself.
#[error("{REQUEST_FAILED} with {0}")]
Console(ConsoleError),
ControlPlane(ControlPlaneError),
/// Various IO errors like broken pipe or malformed payload.
#[error("{REQUEST_FAILED}: {0}")]
@@ -55,7 +55,7 @@ pub(crate) mod errors {
/// Returns HTTP status code if it's the reason for failure.
pub(crate) fn get_reason(&self) -> messages::Reason {
match self {
ApiError::Console(e) => e.get_reason(),
ApiError::ControlPlane(e) => e.get_reason(),
ApiError::Transport(_) => messages::Reason::Unknown,
}
}
@@ -65,7 +65,7 @@ pub(crate) mod errors {
fn to_string_client(&self) -> String {
match self {
// To minimize risks, only select errors are forwarded to users.
ApiError::Console(c) => c.get_user_facing_message(),
ApiError::ControlPlane(c) => c.get_user_facing_message(),
ApiError::Transport(_) => REQUEST_FAILED.to_owned(),
}
}
@@ -74,7 +74,7 @@ pub(crate) mod errors {
impl ReportableError for ApiError {
fn get_error_kind(&self) -> crate::error::ErrorKind {
match self {
ApiError::Console(e) => match e.get_reason() {
ApiError::ControlPlane(e) => match e.get_reason() {
Reason::RoleProtected => ErrorKind::User,
Reason::ResourceNotFound => ErrorKind::User,
Reason::ProjectNotFound => ErrorKind::User,
@@ -91,12 +91,12 @@ pub(crate) mod errors {
Reason::LockAlreadyTaken => ErrorKind::ControlPlane,
Reason::RunningOperations => ErrorKind::ControlPlane,
Reason::Unknown => match &e {
ConsoleError {
ControlPlaneError {
http_status_code:
http::StatusCode::NOT_FOUND | http::StatusCode::NOT_ACCEPTABLE,
..
} => crate::error::ErrorKind::User,
ConsoleError {
ControlPlaneError {
http_status_code: http::StatusCode::UNPROCESSABLE_ENTITY,
error,
..
@@ -105,7 +105,7 @@ pub(crate) mod errors {
{
crate::error::ErrorKind::User
}
ConsoleError {
ControlPlaneError {
http_status_code: http::StatusCode::LOCKED,
error,
..
@@ -114,11 +114,11 @@ pub(crate) mod errors {
{
crate::error::ErrorKind::User
}
ConsoleError {
ControlPlaneError {
http_status_code: http::StatusCode::TOO_MANY_REQUESTS,
..
} => crate::error::ErrorKind::ServiceRateLimit,
ConsoleError { .. } => crate::error::ErrorKind::ControlPlane,
ControlPlaneError { .. } => crate::error::ErrorKind::ControlPlane,
},
},
ApiError::Transport(_) => crate::error::ErrorKind::ControlPlane,
@@ -131,7 +131,7 @@ pub(crate) mod errors {
match self {
// retry some transport errors
Self::Transport(io) => io.could_retry(),
Self::Console(e) => e.could_retry(),
Self::ControlPlane(e) => e.could_retry(),
}
}
}
@@ -314,7 +314,8 @@ impl NodeInfo {
}
}
pub(crate) type NodeInfoCache = TimedLru<EndpointCacheKey, Result<NodeInfo, Box<ConsoleError>>>;
pub(crate) type NodeInfoCache =
TimedLru<EndpointCacheKey, Result<NodeInfo, Box<ControlPlaneError>>>;
pub(crate) type CachedNodeInfo = Cached<&'static NodeInfoCache, NodeInfo>;
pub(crate) type CachedRoleSecret = Cached<&'static ProjectInfoCacheImpl, Option<AuthSecret>>;
pub(crate) type CachedAllowedIps = Cached<&'static ProjectInfoCacheImpl, Arc<Vec<IpPattern>>>;
@@ -353,28 +354,28 @@ pub(crate) trait Api {
#[non_exhaustive]
#[derive(Clone)]
pub enum ConsoleBackend {
/// Current Cloud API (V2).
Console(neon::Api),
/// Local mock of Cloud API (V2).
pub enum ControlPlaneBackend {
/// Current Management API (V2).
Management(neon::Api),
/// Local mock control plane.
#[cfg(any(test, feature = "testing"))]
Postgres(mock::Api),
PostgresMock(mock::Api),
/// Internal testing
#[cfg(test)]
#[allow(private_interfaces)]
Test(Box<dyn crate::auth::backend::TestBackend>),
}
impl Api for ConsoleBackend {
impl Api for ControlPlaneBackend {
async fn get_role_secret(
&self,
ctx: &RequestMonitoring,
user_info: &ComputeUserInfo,
) -> Result<CachedRoleSecret, errors::GetAuthInfoError> {
match self {
Self::Console(api) => api.get_role_secret(ctx, user_info).await,
Self::Management(api) => api.get_role_secret(ctx, user_info).await,
#[cfg(any(test, feature = "testing"))]
Self::Postgres(api) => api.get_role_secret(ctx, user_info).await,
Self::PostgresMock(api) => api.get_role_secret(ctx, user_info).await,
#[cfg(test)]
Self::Test(_) => {
unreachable!("this function should never be called in the test backend")
@@ -388,9 +389,9 @@ impl Api for ConsoleBackend {
user_info: &ComputeUserInfo,
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), errors::GetAuthInfoError> {
match self {
Self::Console(api) => api.get_allowed_ips_and_secret(ctx, user_info).await,
Self::Management(api) => api.get_allowed_ips_and_secret(ctx, user_info).await,
#[cfg(any(test, feature = "testing"))]
Self::Postgres(api) => api.get_allowed_ips_and_secret(ctx, user_info).await,
Self::PostgresMock(api) => api.get_allowed_ips_and_secret(ctx, user_info).await,
#[cfg(test)]
Self::Test(api) => api.get_allowed_ips_and_secret(),
}
@@ -402,9 +403,9 @@ impl Api for ConsoleBackend {
endpoint: EndpointId,
) -> anyhow::Result<Vec<AuthRule>> {
match self {
Self::Console(api) => api.get_endpoint_jwks(ctx, endpoint).await,
Self::Management(api) => api.get_endpoint_jwks(ctx, endpoint).await,
#[cfg(any(test, feature = "testing"))]
Self::Postgres(api) => api.get_endpoint_jwks(ctx, endpoint).await,
Self::PostgresMock(api) => api.get_endpoint_jwks(ctx, endpoint).await,
#[cfg(test)]
Self::Test(_api) => Ok(vec![]),
}
@@ -416,16 +417,16 @@ impl Api for ConsoleBackend {
user_info: &ComputeUserInfo,
) -> Result<CachedNodeInfo, errors::WakeComputeError> {
match self {
Self::Console(api) => api.wake_compute(ctx, user_info).await,
Self::Management(api) => api.wake_compute(ctx, user_info).await,
#[cfg(any(test, feature = "testing"))]
Self::Postgres(api) => api.wake_compute(ctx, user_info).await,
Self::PostgresMock(api) => api.wake_compute(ctx, user_info).await,
#[cfg(test)]
Self::Test(api) => api.wake_compute(),
}
}
}
/// Various caches for [`console`](super).
/// Various caches for [`control_plane`](super).
pub struct ApiCaches {
/// Cache for the `wake_compute` API method.
pub(crate) node_info: NodeInfoCache,
@@ -454,7 +455,7 @@ impl ApiCaches {
}
}
/// Various caches for [`console`](super).
/// Various caches for [`control_plane`](super).
pub struct ApiLocks<K> {
name: &'static str,
node_locks: DashMap<K, Arc<DynamicLimiter>>,
@@ -577,7 +578,7 @@ impl WakeComputePermit {
}
}
impl FetchAuthRules for ConsoleBackend {
impl FetchAuthRules for ControlPlaneBackend {
async fn fetch_auth_rules(
&self,
ctx: &RequestMonitoring,

View File

@@ -10,7 +10,7 @@ use crate::{
use crate::{auth::backend::ComputeUserInfo, compute, error::io_error, scram, url::ApiUrl};
use crate::{auth::IpPattern, cache::Cached};
use crate::{
console::{
control_plane::{
messages::MetricsAuxInfo,
provider::{CachedAllowedIps, CachedRoleSecret},
},
@@ -166,7 +166,7 @@ impl Api {
endpoint_id: (&EndpointId::from("endpoint")).into(),
project_id: (&ProjectId::from("project")).into(),
branch_id: (&BranchId::from("branch")).into(),
cold_start_info: crate::console::messages::ColdStartInfo::Warm,
cold_start_info: crate::control_plane::messages::ColdStartInfo::Warm,
},
allow_self_signed_compute: false,
};

View File

@@ -1,7 +1,7 @@
//! Production console backend.
use super::{
super::messages::{ConsoleError, GetRoleSecret, WakeCompute},
super::messages::{ControlPlaneError, GetRoleSecret, WakeCompute},
errors::{ApiError, GetAuthInfoError, WakeComputeError},
ApiCaches, ApiLocks, AuthInfo, AuthSecret, CachedAllowedIps, CachedNodeInfo, CachedRoleSecret,
NodeInfo,
@@ -9,7 +9,7 @@ use super::{
use crate::{
auth::backend::{jwt::AuthRule, ComputeUserInfo},
compute,
console::messages::{ColdStartInfo, EndpointJwksResponse, Reason},
control_plane::messages::{ColdStartInfo, EndpointJwksResponse, Reason},
http,
metrics::{CacheOutcome, Metrics},
rate_limiter::WakeComputeRateLimiter,
@@ -348,7 +348,7 @@ impl super::Api for Api {
let (cached, info) = cached.take_value();
let info = info.map_err(|c| {
info!(key = &*key, "found cached wake_compute error");
WakeComputeError::ApiError(ApiError::Console(*c))
WakeComputeError::ApiError(ApiError::ControlPlane(*c))
})?;
debug!(key = &*key, "found cached compute node info");
@@ -395,9 +395,9 @@ impl super::Api for Api {
Ok(cached.map(|()| node))
}
Err(err) => match err {
WakeComputeError::ApiError(ApiError::Console(err)) => {
WakeComputeError::ApiError(ApiError::ControlPlane(err)) => {
let Some(status) = &err.status else {
return Err(WakeComputeError::ApiError(ApiError::Console(err)));
return Err(WakeComputeError::ApiError(ApiError::ControlPlane(err)));
};
let reason = status
@@ -407,7 +407,7 @@ impl super::Api for Api {
// if we can retry this error, do not cache it.
if reason.can_retry() {
return Err(WakeComputeError::ApiError(ApiError::Console(err)));
return Err(WakeComputeError::ApiError(ApiError::ControlPlane(err)));
}
// at this point, we should only have quota errors.
@@ -422,7 +422,7 @@ impl super::Api for Api {
Duration::from_secs(30),
);
Err(WakeComputeError::ApiError(ApiError::Console(err)))
Err(WakeComputeError::ApiError(ApiError::ControlPlane(err)))
}
err => return Err(err),
},
@@ -448,7 +448,7 @@ async fn parse_body<T: for<'a> serde::Deserialize<'a>>(
// as the fact that the request itself has failed.
let mut body = serde_json::from_slice(&s).unwrap_or_else(|e| {
warn!("failed to parse error body: {e}");
ConsoleError {
ControlPlaneError {
error: "reason unclear (malformed error message)".into(),
http_status_code: status,
status: None,
@@ -457,7 +457,7 @@ async fn parse_body<T: for<'a> serde::Deserialize<'a>>(
body.http_status_code = status;
error!("console responded with an error ({status}): {body:?}");
Err(ApiError::Console(body))
Err(ApiError::ControlPlane(body))
}
fn parse_host_port(input: &str) -> Option<(&str, u16)> {

View File

@@ -97,8 +97,8 @@ pub mod cache;
pub mod cancellation;
pub mod compute;
pub mod config;
pub mod console;
pub mod context;
pub mod control_plane;
pub mod error;
pub mod http;
pub mod intern;

View File

@@ -11,7 +11,7 @@ use metrics::{CounterPairAssoc, CounterPairVec, HyperLogLog, HyperLogLogVec};
use tokio::time::{self, Instant};
use crate::console::messages::ColdStartInfo;
use crate::control_plane::messages::ColdStartInfo;
#[derive(MetricGroup)]
#[metric(new(thread_pool: Arc<ThreadPoolMetrics>))]

View File

@@ -3,8 +3,8 @@ use crate::{
compute::COULD_NOT_CONNECT,
compute::{self, PostgresConnection},
config::RetryConfig,
console::{self, errors::WakeComputeError, locks::ApiLocks, CachedNodeInfo, NodeInfo},
context::RequestMonitoring,
control_plane::{self, errors::WakeComputeError, locks::ApiLocks, CachedNodeInfo, NodeInfo},
error::ReportableError,
metrics::{ConnectOutcome, ConnectionFailureKind, Metrics, RetriesMetricGroup, RetryType},
proxy::{
@@ -26,7 +26,7 @@ const CONNECT_TIMEOUT: time::Duration = time::Duration::from_secs(2);
/// (e.g. the compute node's address might've changed at the wrong time).
/// Invalidate the cache entry (if any) to prevent subsequent errors.
#[tracing::instrument(name = "invalidate_cache", skip_all)]
pub(crate) fn invalidate_cache(node_info: console::CachedNodeInfo) -> NodeInfo {
pub(crate) fn invalidate_cache(node_info: control_plane::CachedNodeInfo) -> NodeInfo {
let is_cached = node_info.cached();
if is_cached {
warn!("invalidating stalled compute node info cache entry");
@@ -49,7 +49,7 @@ pub(crate) trait ConnectMechanism {
async fn connect_once(
&self,
ctx: &RequestMonitoring,
node_info: &console::CachedNodeInfo,
node_info: &control_plane::CachedNodeInfo,
timeout: time::Duration,
) -> Result<Self::Connection, Self::ConnectError>;
@@ -61,7 +61,7 @@ pub(crate) trait ComputeConnectBackend {
async fn wake_compute(
&self,
ctx: &RequestMonitoring,
) -> Result<CachedNodeInfo, console::errors::WakeComputeError>;
) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError>;
fn get_keys(&self) -> &ComputeCredentialKeys;
}
@@ -84,7 +84,7 @@ impl ConnectMechanism for TcpMechanism<'_> {
async fn connect_once(
&self,
ctx: &RequestMonitoring,
node_info: &console::CachedNodeInfo,
node_info: &control_plane::CachedNodeInfo,
timeout: time::Duration,
) -> Result<PostgresConnection, Self::Error> {
let host = node_info.config.get_host()?;

View File

@@ -1,7 +1,7 @@
use crate::{
cancellation,
compute::PostgresConnection,
console::messages::MetricsAuxInfo,
control_plane::messages::MetricsAuxInfo,
metrics::{Direction, Metrics, NumClientConnectionsGuard, NumConnectionRequestsGuard},
stream::Stream,
usage_metrics::{Ids, MetricCounterRecorder, USAGE_METRICS},

View File

@@ -11,9 +11,11 @@ use crate::auth::backend::{
ComputeCredentialKeys, ComputeCredentials, ComputeUserInfo, MaybeOwned, TestBackend,
};
use crate::config::{CertResolver, RetryConfig};
use crate::console::messages::{ConsoleError, Details, MetricsAuxInfo, Status};
use crate::console::provider::{CachedAllowedIps, CachedRoleSecret, ConsoleBackend, NodeInfoCache};
use crate::console::{self, CachedNodeInfo, NodeInfo};
use crate::control_plane::messages::{ControlPlaneError, Details, MetricsAuxInfo, Status};
use crate::control_plane::provider::{
CachedAllowedIps, CachedRoleSecret, ControlPlaneBackend, NodeInfoCache,
};
use crate::control_plane::{self, CachedNodeInfo, NodeInfo};
use crate::error::ErrorKind;
use crate::{sasl, scram, BranchId, EndpointId, ProjectId};
use anyhow::{bail, Context};
@@ -459,7 +461,7 @@ impl ConnectMechanism for TestConnectMechanism {
async fn connect_once(
&self,
_ctx: &RequestMonitoring,
_node_info: &console::CachedNodeInfo,
_node_info: &control_plane::CachedNodeInfo,
_timeout: std::time::Duration,
) -> Result<Self::Connection, Self::ConnectError> {
let mut counter = self.counter.lock().unwrap();
@@ -483,23 +485,23 @@ impl ConnectMechanism for TestConnectMechanism {
}
impl TestBackend for TestConnectMechanism {
fn wake_compute(&self) -> Result<CachedNodeInfo, console::errors::WakeComputeError> {
fn wake_compute(&self) -> Result<CachedNodeInfo, control_plane::errors::WakeComputeError> {
let mut counter = self.counter.lock().unwrap();
let action = self.sequence[*counter];
*counter += 1;
match action {
ConnectAction::Wake => Ok(helper_create_cached_node_info(self.cache)),
ConnectAction::WakeFail => {
let err = console::errors::ApiError::Console(ConsoleError {
let err = control_plane::errors::ApiError::ControlPlane(ControlPlaneError {
http_status_code: StatusCode::BAD_REQUEST,
error: "TEST".into(),
status: None,
});
assert!(!err.could_retry());
Err(console::errors::WakeComputeError::ApiError(err))
Err(control_plane::errors::WakeComputeError::ApiError(err))
}
ConnectAction::WakeRetry => {
let err = console::errors::ApiError::Console(ConsoleError {
let err = control_plane::errors::ApiError::ControlPlane(ControlPlaneError {
http_status_code: StatusCode::BAD_REQUEST,
error: "TEST".into(),
status: Some(Status {
@@ -507,13 +509,15 @@ impl TestBackend for TestConnectMechanism {
message: "error".into(),
details: Details {
error_info: None,
retry_info: Some(console::messages::RetryInfo { retry_delay_ms: 1 }),
retry_info: Some(control_plane::messages::RetryInfo {
retry_delay_ms: 1,
}),
user_facing_message: None,
},
}),
});
assert!(err.could_retry());
Err(console::errors::WakeComputeError::ApiError(err))
Err(control_plane::errors::WakeComputeError::ApiError(err))
}
x => panic!("expecting action {x:?}, wake_compute is called instead"),
}
@@ -521,7 +525,7 @@ impl TestBackend for TestConnectMechanism {
fn get_allowed_ips_and_secret(
&self,
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), console::errors::GetAuthInfoError>
) -> Result<(CachedAllowedIps, Option<CachedRoleSecret>), control_plane::errors::GetAuthInfoError>
{
unimplemented!("not used in tests")
}
@@ -538,7 +542,7 @@ fn helper_create_cached_node_info(cache: &'static NodeInfoCache) -> CachedNodeIn
endpoint_id: (&EndpointId::from("endpoint")).into(),
project_id: (&ProjectId::from("project")).into(),
branch_id: (&BranchId::from("branch")).into(),
cold_start_info: crate::console::messages::ColdStartInfo::Warm,
cold_start_info: crate::control_plane::messages::ColdStartInfo::Warm,
},
allow_self_signed_compute: false,
};
@@ -549,8 +553,8 @@ fn helper_create_cached_node_info(cache: &'static NodeInfoCache) -> CachedNodeIn
fn helper_create_connect_info(
mechanism: &TestConnectMechanism,
) -> auth::Backend<'static, ComputeCredentials, &()> {
let user_info = auth::Backend::Console(
MaybeOwned::Owned(ConsoleBackend::Test(Box::new(mechanism.clone()))),
let user_info = auth::Backend::ControlPlane(
MaybeOwned::Owned(ControlPlaneBackend::Test(Box::new(mechanism.clone()))),
ComputeCredentials {
info: ComputeUserInfo {
endpoint: "endpoint".into(),

View File

@@ -1,7 +1,7 @@
use crate::config::RetryConfig;
use crate::console::messages::{ConsoleError, Reason};
use crate::console::{errors::WakeComputeError, provider::CachedNodeInfo};
use crate::context::RequestMonitoring;
use crate::control_plane::messages::{ControlPlaneError, Reason};
use crate::control_plane::{errors::WakeComputeError, provider::CachedNodeInfo};
use crate::metrics::{
ConnectOutcome, ConnectionFailuresBreakdownGroup, Metrics, RetriesMetricGroup, RetryType,
WakeupFailureKind,
@@ -59,11 +59,11 @@ pub(crate) async fn wake_compute<B: ComputeConnectBackend>(
}
fn report_error(e: &WakeComputeError, retry: bool) {
use crate::console::errors::ApiError;
use crate::control_plane::errors::ApiError;
let kind = match e {
WakeComputeError::BadComputeAddress(_) => WakeupFailureKind::BadComputeAddress,
WakeComputeError::ApiError(ApiError::Transport(_)) => WakeupFailureKind::ApiTransportError,
WakeComputeError::ApiError(ApiError::Console(e)) => match e.get_reason() {
WakeComputeError::ApiError(ApiError::ControlPlane(e)) => match e.get_reason() {
Reason::RoleProtected => WakeupFailureKind::ApiConsoleBadRequest,
Reason::ResourceNotFound => WakeupFailureKind::ApiConsoleBadRequest,
Reason::ProjectNotFound => WakeupFailureKind::ApiConsoleBadRequest,
@@ -80,7 +80,7 @@ fn report_error(e: &WakeComputeError, retry: bool) {
Reason::LockAlreadyTaken => WakeupFailureKind::ApiConsoleLocked,
Reason::RunningOperations => WakeupFailureKind::ApiConsoleLocked,
Reason::Unknown => match e {
ConsoleError {
ControlPlaneError {
http_status_code: StatusCode::LOCKED,
ref error,
..
@@ -89,27 +89,27 @@ fn report_error(e: &WakeComputeError, retry: bool) {
{
WakeupFailureKind::QuotaExceeded
}
ConsoleError {
ControlPlaneError {
http_status_code: StatusCode::UNPROCESSABLE_ENTITY,
ref error,
..
} if error.contains("compute time quota of non-primary branches is exceeded") => {
WakeupFailureKind::QuotaExceeded
}
ConsoleError {
ControlPlaneError {
http_status_code: StatusCode::LOCKED,
..
} => WakeupFailureKind::ApiConsoleLocked,
ConsoleError {
ControlPlaneError {
http_status_code: StatusCode::BAD_REQUEST,
..
} => WakeupFailureKind::ApiConsoleBadRequest,
ConsoleError {
ControlPlaneError {
http_status_code, ..
} if http_status_code.is_server_error() => {
WakeupFailureKind::ApiConsoleOtherServerError
}
ConsoleError { .. } => WakeupFailureKind::ApiConsoleOtherError,
ControlPlaneError { .. } => WakeupFailureKind::ApiConsoleOtherError,
},
},
WakeComputeError::TooManyConnections => WakeupFailureKind::ApiConsoleLocked,

View File

@@ -12,13 +12,13 @@ use crate::{
},
compute,
config::{AuthenticationConfig, ProxyConfig},
console::{
context::RequestMonitoring,
control_plane::{
errors::{GetAuthInfoError, WakeComputeError},
locks::ApiLocks,
provider::ApiLockError,
CachedNodeInfo,
},
context::RequestMonitoring,
error::{ErrorKind, ReportableError, UserFacingError},
intern::EndpointIdInt,
proxy::{
@@ -114,7 +114,7 @@ impl PoolingBackend {
jwt: String,
) -> Result<(), AuthError> {
match &self.config.auth_backend {
crate::auth::Backend::Console(console, ()) => {
crate::auth::Backend::ControlPlane(console, ()) => {
config
.jwks_cache
.check_jwt(
@@ -129,7 +129,7 @@ impl PoolingBackend {
Ok(())
}
crate::auth::Backend::Web(_, ()) => Err(AuthError::auth_failed(
crate::auth::Backend::ConsoleRedirect(_, ()) => Err(AuthError::auth_failed(
"JWT login over web auth proxy is not supported",
)),
crate::auth::Backend::Local(_) => {

View File

@@ -17,7 +17,7 @@ use tokio_postgres::tls::NoTlsStream;
use tokio_postgres::{AsyncMessage, ReadyForQueryStatus, Socket};
use tokio_util::sync::CancellationToken;
use crate::console::messages::{ColdStartInfo, MetricsAuxInfo};
use crate::control_plane::messages::{ColdStartInfo, MetricsAuxInfo};
use crate::metrics::{HttpEndpointPoolsGuard, Metrics};
use crate::usage_metrics::{Ids, MetricCounter, USAGE_METRICS};
use crate::{
@@ -760,7 +760,7 @@ mod tests {
endpoint_id: (&EndpointId::from("endpoint")).into(),
project_id: (&ProjectId::from("project")).into(),
branch_id: (&BranchId::from("branch")).into(),
cold_start_info: crate::console::messages::ColdStartInfo::Warm,
cold_start_info: crate::control_plane::messages::ColdStartInfo::Warm,
},
conn_id: uuid::Uuid::new_v4(),
}

View File

@@ -8,7 +8,7 @@ use std::sync::atomic::{self, AtomicUsize};
use std::{sync::Arc, sync::Weak};
use tokio::net::TcpStream;
use crate::console::messages::{ColdStartInfo, MetricsAuxInfo};
use crate::control_plane::messages::{ColdStartInfo, MetricsAuxInfo};
use crate::metrics::{HttpEndpointPoolsGuard, Metrics};
use crate::usage_metrics::{Ids, MetricCounter, USAGE_METRICS};
use crate::{context::RequestMonitoring, EndpointCacheKey};