mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-10 23:12:54 +00:00
feat(pageserver): integrate PostHog with gc-compaction rollout (#11917)
## Problem part of https://github.com/neondatabase/neon/issues/11813 ## Summary of changes * Integrate feature store with tenant structure. * gc-compaction picks up the current strategy from the feature store. * We only log them for now for testing purpose. They will not be used until we have more patches to support different strategies defined in PostHog. * We don't support property-based evaulation for now; it will be implemented later. * Evaluating result of the feature flag is not cached -- it's not efficient and cannot be used on hot path right now. * We don't report the evaluation result back to PostHog right now. I plan to enable it in staging once we get the patch merged. --------- Signed-off-by: Alex Chi Z <chi@neon.tech>
This commit is contained in:
9
Cargo.lock
generated
9
Cargo.lock
generated
@@ -4330,6 +4330,7 @@ dependencies = [
|
||||
"postgres_connection",
|
||||
"postgres_ffi",
|
||||
"postgres_initdb",
|
||||
"posthog_client_lite",
|
||||
"pprof",
|
||||
"pq_proto",
|
||||
"procfs",
|
||||
@@ -4907,11 +4908,16 @@ name = "posthog_client_lite"
|
||||
version = "0.1.0"
|
||||
dependencies = [
|
||||
"anyhow",
|
||||
"arc-swap",
|
||||
"reqwest",
|
||||
"serde",
|
||||
"serde_json",
|
||||
"sha2",
|
||||
"thiserror 1.0.69",
|
||||
"tokio",
|
||||
"tokio-util",
|
||||
"tracing",
|
||||
"tracing-utils",
|
||||
"workspace_hack",
|
||||
]
|
||||
|
||||
@@ -8575,10 +8581,8 @@ dependencies = [
|
||||
"fail",
|
||||
"form_urlencoded",
|
||||
"futures-channel",
|
||||
"futures-core",
|
||||
"futures-executor",
|
||||
"futures-io",
|
||||
"futures-task",
|
||||
"futures-util",
|
||||
"generic-array",
|
||||
"getrandom 0.2.11",
|
||||
@@ -8608,7 +8612,6 @@ dependencies = [
|
||||
"once_cell",
|
||||
"p256 0.13.2",
|
||||
"parquet",
|
||||
"percent-encoding",
|
||||
"prettyplease",
|
||||
"proc-macro2",
|
||||
"prost 0.13.5",
|
||||
|
||||
@@ -247,6 +247,7 @@ azure_storage_blobs = { git = "https://github.com/neondatabase/azure-sdk-for-rus
|
||||
## Local libraries
|
||||
compute_api = { version = "0.1", path = "./libs/compute_api/" }
|
||||
consumption_metrics = { version = "0.1", path = "./libs/consumption_metrics/" }
|
||||
desim = { version = "0.1", path = "./libs/desim" }
|
||||
endpoint_storage = { version = "0.0.1", path = "./endpoint_storage/" }
|
||||
http-utils = { version = "0.1", path = "./libs/http-utils/" }
|
||||
metrics = { version = "0.1", path = "./libs/metrics/" }
|
||||
@@ -259,19 +260,19 @@ postgres_backend = { version = "0.1", path = "./libs/postgres_backend/" }
|
||||
postgres_connection = { version = "0.1", path = "./libs/postgres_connection/" }
|
||||
postgres_ffi = { version = "0.1", path = "./libs/postgres_ffi/" }
|
||||
postgres_initdb = { path = "./libs/postgres_initdb" }
|
||||
posthog_client_lite = { version = "0.1", path = "./libs/posthog_client_lite" }
|
||||
pq_proto = { version = "0.1", path = "./libs/pq_proto/" }
|
||||
remote_storage = { version = "0.1", path = "./libs/remote_storage/" }
|
||||
safekeeper_api = { version = "0.1", path = "./libs/safekeeper_api" }
|
||||
safekeeper_client = { path = "./safekeeper/client" }
|
||||
desim = { version = "0.1", path = "./libs/desim" }
|
||||
storage_broker = { version = "0.1", path = "./storage_broker/" } # Note: main broker code is inside the binary crate, so linking with the library shouldn't be heavy.
|
||||
storage_controller_client = { path = "./storage_controller/client" }
|
||||
tenant_size_model = { version = "0.1", path = "./libs/tenant_size_model/" }
|
||||
tracing-utils = { version = "0.1", path = "./libs/tracing-utils/" }
|
||||
utils = { version = "0.1", path = "./libs/utils/" }
|
||||
vm_monitor = { version = "0.1", path = "./libs/vm_monitor/" }
|
||||
walproposer = { version = "0.1", path = "./libs/walproposer/" }
|
||||
wal_decoder = { version = "0.1", path = "./libs/wal_decoder" }
|
||||
walproposer = { version = "0.1", path = "./libs/walproposer/" }
|
||||
|
||||
## Common library dependency
|
||||
workspace_hack = { version = "0.1", path = "./workspace_hack/" }
|
||||
|
||||
@@ -45,6 +45,21 @@ pub struct NodeMetadata {
|
||||
pub other: HashMap<String, serde_json::Value>,
|
||||
}
|
||||
|
||||
/// PostHog integration config.
|
||||
#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
|
||||
pub struct PostHogConfig {
|
||||
/// PostHog project ID
|
||||
pub project_id: String,
|
||||
/// Server-side (private) API key
|
||||
pub server_api_key: String,
|
||||
/// Client-side (public) API key
|
||||
pub client_api_key: String,
|
||||
/// Private API URL
|
||||
pub private_api_url: String,
|
||||
/// Public API URL
|
||||
pub public_api_url: String,
|
||||
}
|
||||
|
||||
/// `pageserver.toml`
|
||||
///
|
||||
/// We use serde derive with `#[serde(default)]` to generate a deserializer
|
||||
@@ -186,6 +201,8 @@ pub struct ConfigToml {
|
||||
pub tracing: Option<Tracing>,
|
||||
pub enable_tls_page_service_api: bool,
|
||||
pub dev_mode: bool,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
pub posthog_config: Option<PostHogConfig>,
|
||||
pub timeline_import_config: TimelineImportConfig,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
pub basebackup_cache_config: Option<BasebackupCacheConfig>,
|
||||
@@ -701,6 +718,7 @@ impl Default for ConfigToml {
|
||||
import_job_checkpoint_threshold: NonZeroUsize::new(128).unwrap(),
|
||||
},
|
||||
basebackup_cache_config: None,
|
||||
posthog_config: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,9 +6,14 @@ license.workspace = true
|
||||
|
||||
[dependencies]
|
||||
anyhow.workspace = true
|
||||
arc-swap.workspace = true
|
||||
reqwest.workspace = true
|
||||
serde.workspace = true
|
||||
serde_json.workspace = true
|
||||
serde.workspace = true
|
||||
sha2.workspace = true
|
||||
workspace_hack.workspace = true
|
||||
thiserror.workspace = true
|
||||
tokio = { workspace = true, features = ["process", "sync", "fs", "rt", "io-util", "time"] }
|
||||
tokio-util.workspace = true
|
||||
tracing-utils.workspace = true
|
||||
tracing.workspace = true
|
||||
workspace_hack.workspace = true
|
||||
|
||||
59
libs/posthog_client_lite/src/background_loop.rs
Normal file
59
libs/posthog_client_lite/src/background_loop.rs
Normal file
@@ -0,0 +1,59 @@
|
||||
//! A background loop that fetches feature flags from PostHog and updates the feature store.
|
||||
|
||||
use std::{sync::Arc, time::Duration};
|
||||
|
||||
use arc_swap::ArcSwap;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
|
||||
use crate::{FeatureStore, PostHogClient, PostHogClientConfig};
|
||||
|
||||
/// A background loop that fetches feature flags from PostHog and updates the feature store.
|
||||
pub struct FeatureResolverBackgroundLoop {
|
||||
posthog_client: PostHogClient,
|
||||
feature_store: ArcSwap<FeatureStore>,
|
||||
cancel: CancellationToken,
|
||||
}
|
||||
|
||||
impl FeatureResolverBackgroundLoop {
|
||||
pub fn new(config: PostHogClientConfig, shutdown_pageserver: CancellationToken) -> Self {
|
||||
Self {
|
||||
posthog_client: PostHogClient::new(config),
|
||||
feature_store: ArcSwap::new(Arc::new(FeatureStore::new())),
|
||||
cancel: shutdown_pageserver,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn spawn(self: Arc<Self>, handle: &tokio::runtime::Handle, refresh_period: Duration) {
|
||||
let this = self.clone();
|
||||
let cancel = self.cancel.clone();
|
||||
handle.spawn(async move {
|
||||
tracing::info!("Starting PostHog feature resolver");
|
||||
let mut ticker = tokio::time::interval(refresh_period);
|
||||
ticker.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Skip);
|
||||
loop {
|
||||
tokio::select! {
|
||||
_ = ticker.tick() => {}
|
||||
_ = cancel.cancelled() => break
|
||||
}
|
||||
let resp = match this
|
||||
.posthog_client
|
||||
.get_feature_flags_local_evaluation()
|
||||
.await
|
||||
{
|
||||
Ok(resp) => resp,
|
||||
Err(e) => {
|
||||
tracing::warn!("Cannot get feature flags: {}", e);
|
||||
continue;
|
||||
}
|
||||
};
|
||||
let feature_store = FeatureStore::new_with_flags(resp.flags);
|
||||
this.feature_store.store(Arc::new(feature_store));
|
||||
}
|
||||
tracing::info!("PostHog feature resolver stopped");
|
||||
});
|
||||
}
|
||||
|
||||
pub fn feature_store(&self) -> Arc<FeatureStore> {
|
||||
self.feature_store.load_full()
|
||||
}
|
||||
}
|
||||
@@ -1,5 +1,9 @@
|
||||
//! A lite version of the PostHog client that only supports local evaluation of feature flags.
|
||||
|
||||
mod background_loop;
|
||||
|
||||
pub use background_loop::FeatureResolverBackgroundLoop;
|
||||
|
||||
use std::collections::HashMap;
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
@@ -20,8 +24,7 @@ pub enum PostHogEvaluationError {
|
||||
|
||||
#[derive(Deserialize)]
|
||||
pub struct LocalEvaluationResponse {
|
||||
#[allow(dead_code)]
|
||||
flags: Vec<LocalEvaluationFlag>,
|
||||
pub flags: Vec<LocalEvaluationFlag>,
|
||||
}
|
||||
|
||||
#[derive(Deserialize)]
|
||||
@@ -94,6 +97,12 @@ impl FeatureStore {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn new_with_flags(flags: Vec<LocalEvaluationFlag>) -> Self {
|
||||
let mut store = Self::new();
|
||||
store.set_flags(flags);
|
||||
store
|
||||
}
|
||||
|
||||
pub fn set_flags(&mut self, flags: Vec<LocalEvaluationFlag>) {
|
||||
self.flags.clear();
|
||||
for flag in flags {
|
||||
@@ -267,6 +276,7 @@ impl FeatureStore {
|
||||
&self,
|
||||
flag_key: &str,
|
||||
user_id: &str,
|
||||
properties: &HashMap<String, PostHogFlagFilterPropertyValue>,
|
||||
) -> Result<String, PostHogEvaluationError> {
|
||||
let hash_on_global_rollout_percentage =
|
||||
Self::consistent_hash(user_id, flag_key, "multivariate");
|
||||
@@ -276,7 +286,7 @@ impl FeatureStore {
|
||||
flag_key,
|
||||
hash_on_global_rollout_percentage,
|
||||
hash_on_group_rollout_percentage,
|
||||
&HashMap::new(),
|
||||
properties,
|
||||
)
|
||||
}
|
||||
|
||||
@@ -344,6 +354,19 @@ impl FeatureStore {
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PostHogClientConfig {
|
||||
/// The server API key.
|
||||
pub server_api_key: String,
|
||||
/// The client API key.
|
||||
pub client_api_key: String,
|
||||
/// The project ID.
|
||||
pub project_id: String,
|
||||
/// The private API URL.
|
||||
pub private_api_url: String,
|
||||
/// The public API URL.
|
||||
pub public_api_url: String,
|
||||
}
|
||||
|
||||
/// A lite PostHog client.
|
||||
///
|
||||
/// At the point of writing this code, PostHog does not have a functional Rust client with feature flag support.
|
||||
@@ -360,37 +383,16 @@ impl FeatureStore {
|
||||
/// want to report the feature flag usage back to PostHog. The current plan is to use PostHog only as an UI to
|
||||
/// configure feature flags so it is very likely that the client API will not be used.
|
||||
pub struct PostHogClient {
|
||||
/// The server API key.
|
||||
server_api_key: String,
|
||||
/// The client API key.
|
||||
client_api_key: String,
|
||||
/// The project ID.
|
||||
project_id: String,
|
||||
/// The private API URL.
|
||||
private_api_url: String,
|
||||
/// The public API URL.
|
||||
public_api_url: String,
|
||||
/// The config.
|
||||
config: PostHogClientConfig,
|
||||
/// The HTTP client.
|
||||
client: reqwest::Client,
|
||||
}
|
||||
|
||||
impl PostHogClient {
|
||||
pub fn new(
|
||||
server_api_key: String,
|
||||
client_api_key: String,
|
||||
project_id: String,
|
||||
private_api_url: String,
|
||||
public_api_url: String,
|
||||
) -> Self {
|
||||
pub fn new(config: PostHogClientConfig) -> Self {
|
||||
let client = reqwest::Client::new();
|
||||
Self {
|
||||
server_api_key,
|
||||
client_api_key,
|
||||
project_id,
|
||||
private_api_url,
|
||||
public_api_url,
|
||||
client,
|
||||
}
|
||||
Self { config, client }
|
||||
}
|
||||
|
||||
pub fn new_with_us_region(
|
||||
@@ -398,13 +400,13 @@ impl PostHogClient {
|
||||
client_api_key: String,
|
||||
project_id: String,
|
||||
) -> Self {
|
||||
Self::new(
|
||||
Self::new(PostHogClientConfig {
|
||||
server_api_key,
|
||||
client_api_key,
|
||||
project_id,
|
||||
"https://us.posthog.com".to_string(),
|
||||
"https://us.i.posthog.com".to_string(),
|
||||
)
|
||||
private_api_url: "https://us.posthog.com".to_string(),
|
||||
public_api_url: "https://us.i.posthog.com".to_string(),
|
||||
})
|
||||
}
|
||||
|
||||
/// Fetch the feature flag specs from the server.
|
||||
@@ -422,12 +424,12 @@ impl PostHogClient {
|
||||
// with bearer token of self.server_api_key
|
||||
let url = format!(
|
||||
"{}/api/projects/{}/feature_flags/local_evaluation",
|
||||
self.private_api_url, self.project_id
|
||||
self.config.private_api_url, self.config.project_id
|
||||
);
|
||||
let response = self
|
||||
.client
|
||||
.get(url)
|
||||
.bearer_auth(&self.server_api_key)
|
||||
.bearer_auth(&self.config.server_api_key)
|
||||
.send()
|
||||
.await?;
|
||||
let body = response.text().await?;
|
||||
@@ -446,11 +448,11 @@ impl PostHogClient {
|
||||
) -> anyhow::Result<()> {
|
||||
// PUBLIC_URL/capture/
|
||||
// with bearer token of self.client_api_key
|
||||
let url = format!("{}/capture/", self.public_api_url);
|
||||
let url = format!("{}/capture/", self.config.public_api_url);
|
||||
self.client
|
||||
.post(url)
|
||||
.body(serde_json::to_string(&json!({
|
||||
"api_key": self.client_api_key,
|
||||
"api_key": self.config.client_api_key,
|
||||
"distinct_id": distinct_id,
|
||||
"event": event,
|
||||
"properties": properties,
|
||||
|
||||
@@ -17,51 +17,69 @@ anyhow.workspace = true
|
||||
arc-swap.workspace = true
|
||||
async-compression.workspace = true
|
||||
async-stream.workspace = true
|
||||
bit_field.workspace = true
|
||||
bincode.workspace = true
|
||||
bit_field.workspace = true
|
||||
byteorder.workspace = true
|
||||
bytes.workspace = true
|
||||
camino.workspace = true
|
||||
camino-tempfile.workspace = true
|
||||
camino.workspace = true
|
||||
chrono = { workspace = true, features = ["serde"] }
|
||||
clap = { workspace = true, features = ["string"] }
|
||||
consumption_metrics.workspace = true
|
||||
crc32c.workspace = true
|
||||
either.workspace = true
|
||||
enum-map.workspace = true
|
||||
enumset = { workspace = true, features = ["serde"]}
|
||||
fail.workspace = true
|
||||
futures.workspace = true
|
||||
hashlink.workspace = true
|
||||
hex.workspace = true
|
||||
humantime.workspace = true
|
||||
http-utils.workspace = true
|
||||
humantime-serde.workspace = true
|
||||
humantime.workspace = true
|
||||
hyper0.workspace = true
|
||||
itertools.workspace = true
|
||||
jsonwebtoken.workspace = true
|
||||
md5.workspace = true
|
||||
metrics.workspace = true
|
||||
nix.workspace = true
|
||||
# hack to get the number of worker threads tokio uses
|
||||
num_cpus.workspace = true
|
||||
num_cpus.workspace = true # hack to get the number of worker threads tokio uses
|
||||
num-traits.workspace = true
|
||||
once_cell.workspace = true
|
||||
pageserver_api.workspace = true
|
||||
pageserver_client.workspace = true # for ResponseErrorMessageExt TOOD refactor that
|
||||
pageserver_compaction.workspace = true
|
||||
pageserver_page_api.workspace = true
|
||||
pem.workspace = true
|
||||
pin-project-lite.workspace = true
|
||||
postgres_backend.workspace = true
|
||||
postgres_connection.workspace = true
|
||||
postgres_ffi.workspace = true
|
||||
postgres_initdb.workspace = true
|
||||
postgres-protocol.workspace = true
|
||||
postgres-types.workspace = true
|
||||
postgres_initdb.workspace = true
|
||||
posthog_client_lite.workspace = true
|
||||
pprof.workspace = true
|
||||
pq_proto.workspace = true
|
||||
rand.workspace = true
|
||||
range-set-blaze = { version = "0.1.16", features = ["alloc"] }
|
||||
regex.workspace = true
|
||||
remote_storage.workspace = true
|
||||
reqwest.workspace = true
|
||||
rpds.workspace = true
|
||||
rustls.workspace = true
|
||||
scopeguard.workspace = true
|
||||
send-future.workspace = true
|
||||
serde.workspace = true
|
||||
serde_json = { workspace = true, features = ["raw_value"] }
|
||||
serde_path_to_error.workspace = true
|
||||
serde_with.workspace = true
|
||||
serde.workspace = true
|
||||
smallvec.workspace = true
|
||||
storage_broker.workspace = true
|
||||
strum_macros.workspace = true
|
||||
strum.workspace = true
|
||||
sysinfo.workspace = true
|
||||
tokio-tar.workspace = true
|
||||
tenant_size_model.workspace = true
|
||||
thiserror.workspace = true
|
||||
tikv-jemallocator.workspace = true
|
||||
tokio = { workspace = true, features = ["process", "sync", "fs", "rt", "io-util", "time"] }
|
||||
@@ -70,6 +88,7 @@ tokio-io-timeout.workspace = true
|
||||
tokio-postgres.workspace = true
|
||||
tokio-rustls.workspace = true
|
||||
tokio-stream.workspace = true
|
||||
tokio-tar.workspace = true
|
||||
tokio-util.workspace = true
|
||||
toml_edit = { workspace = true, features = [ "serde" ] }
|
||||
tonic.workspace = true
|
||||
@@ -77,29 +96,10 @@ tonic-reflection.workspace = true
|
||||
tracing.workspace = true
|
||||
tracing-utils.workspace = true
|
||||
url.workspace = true
|
||||
walkdir.workspace = true
|
||||
metrics.workspace = true
|
||||
pageserver_api.workspace = true
|
||||
pageserver_client.workspace = true # for ResponseErrorMessageExt TOOD refactor that
|
||||
pageserver_compaction.workspace = true
|
||||
pem.workspace = true
|
||||
postgres_connection.workspace = true
|
||||
postgres_ffi.workspace = true
|
||||
pq_proto.workspace = true
|
||||
remote_storage.workspace = true
|
||||
storage_broker.workspace = true
|
||||
tenant_size_model.workspace = true
|
||||
http-utils.workspace = true
|
||||
utils.workspace = true
|
||||
workspace_hack.workspace = true
|
||||
reqwest.workspace = true
|
||||
rpds.workspace = true
|
||||
enum-map.workspace = true
|
||||
enumset = { workspace = true, features = ["serde"]}
|
||||
strum.workspace = true
|
||||
strum_macros.workspace = true
|
||||
wal_decoder.workspace = true
|
||||
smallvec.workspace = true
|
||||
walkdir.workspace = true
|
||||
workspace_hack.workspace = true
|
||||
twox-hash.workspace = true
|
||||
|
||||
[target.'cfg(target_os = "linux")'.dependencies]
|
||||
|
||||
@@ -21,6 +21,7 @@ use pageserver::config::{PageServerConf, PageserverIdentity, ignored_fields};
|
||||
use pageserver::controller_upcall_client::StorageControllerUpcallClient;
|
||||
use pageserver::deletion_queue::DeletionQueue;
|
||||
use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task};
|
||||
use pageserver::feature_resolver::FeatureResolver;
|
||||
use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING};
|
||||
use pageserver::task_mgr::{
|
||||
BACKGROUND_RUNTIME, COMPUTE_REQUEST_RUNTIME, MGMT_REQUEST_RUNTIME, WALRECEIVER_RUNTIME,
|
||||
@@ -522,6 +523,12 @@ fn start_pageserver(
|
||||
// Set up remote storage client
|
||||
let remote_storage = BACKGROUND_RUNTIME.block_on(create_remote_storage_client(conf))?;
|
||||
|
||||
let feature_resolver = create_feature_resolver(
|
||||
conf,
|
||||
shutdown_pageserver.clone(),
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
)?;
|
||||
|
||||
// Set up deletion queue
|
||||
let (deletion_queue, deletion_workers) = DeletionQueue::new(
|
||||
remote_storage.clone(),
|
||||
@@ -575,6 +582,7 @@ fn start_pageserver(
|
||||
deletion_queue_client,
|
||||
l0_flush_global_state,
|
||||
basebackup_prepare_sender,
|
||||
feature_resolver,
|
||||
},
|
||||
order,
|
||||
shutdown_pageserver.clone(),
|
||||
@@ -849,6 +857,14 @@ fn start_pageserver(
|
||||
})
|
||||
}
|
||||
|
||||
fn create_feature_resolver(
|
||||
conf: &'static PageServerConf,
|
||||
shutdown_pageserver: CancellationToken,
|
||||
handle: &tokio::runtime::Handle,
|
||||
) -> anyhow::Result<FeatureResolver> {
|
||||
FeatureResolver::spawn(conf, shutdown_pageserver, handle)
|
||||
}
|
||||
|
||||
async fn create_remote_storage_client(
|
||||
conf: &'static PageServerConf,
|
||||
) -> anyhow::Result<GenericRemoteStorage> {
|
||||
|
||||
@@ -14,7 +14,7 @@ use std::time::Duration;
|
||||
use anyhow::{Context, bail, ensure};
|
||||
use camino::{Utf8Path, Utf8PathBuf};
|
||||
use once_cell::sync::OnceCell;
|
||||
use pageserver_api::config::{DiskUsageEvictionTaskConfig, MaxVectoredReadBytes};
|
||||
use pageserver_api::config::{DiskUsageEvictionTaskConfig, MaxVectoredReadBytes, PostHogConfig};
|
||||
use pageserver_api::models::ImageCompressionAlgorithm;
|
||||
use pageserver_api::shard::TenantShardId;
|
||||
use pem::Pem;
|
||||
@@ -238,6 +238,9 @@ pub struct PageServerConf {
|
||||
/// This is insecure and should only be used in development environments.
|
||||
pub dev_mode: bool,
|
||||
|
||||
/// PostHog integration config.
|
||||
pub posthog_config: Option<PostHogConfig>,
|
||||
|
||||
pub timeline_import_config: pageserver_api::config::TimelineImportConfig,
|
||||
|
||||
pub basebackup_cache_config: Option<pageserver_api::config::BasebackupCacheConfig>,
|
||||
@@ -421,6 +424,7 @@ impl PageServerConf {
|
||||
tracing,
|
||||
enable_tls_page_service_api,
|
||||
dev_mode,
|
||||
posthog_config,
|
||||
timeline_import_config,
|
||||
basebackup_cache_config,
|
||||
} = config_toml;
|
||||
@@ -536,6 +540,7 @@ impl PageServerConf {
|
||||
}
|
||||
None => Vec::new(),
|
||||
},
|
||||
posthog_config,
|
||||
};
|
||||
|
||||
// ------------------------------------------------------------
|
||||
|
||||
65
pageserver/src/feature_resolver.rs
Normal file
65
pageserver/src/feature_resolver.rs
Normal file
@@ -0,0 +1,65 @@
|
||||
use std::{collections::HashMap, sync::Arc, time::Duration};
|
||||
|
||||
use posthog_client_lite::{
|
||||
FeatureResolverBackgroundLoop, PostHogClientConfig, PostHogEvaluationError,
|
||||
};
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use utils::id::TenantId;
|
||||
|
||||
use crate::config::PageServerConf;
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct FeatureResolver {
|
||||
inner: Option<Arc<FeatureResolverBackgroundLoop>>,
|
||||
}
|
||||
|
||||
impl FeatureResolver {
|
||||
pub fn new_disabled() -> Self {
|
||||
Self { inner: None }
|
||||
}
|
||||
|
||||
pub fn spawn(
|
||||
conf: &PageServerConf,
|
||||
shutdown_pageserver: CancellationToken,
|
||||
handle: &tokio::runtime::Handle,
|
||||
) -> anyhow::Result<Self> {
|
||||
// DO NOT block in this function: make it return as fast as possible to avoid startup delays.
|
||||
if let Some(posthog_config) = &conf.posthog_config {
|
||||
let inner = FeatureResolverBackgroundLoop::new(
|
||||
PostHogClientConfig {
|
||||
server_api_key: posthog_config.server_api_key.clone(),
|
||||
client_api_key: posthog_config.client_api_key.clone(),
|
||||
project_id: posthog_config.project_id.clone(),
|
||||
private_api_url: posthog_config.private_api_url.clone(),
|
||||
public_api_url: posthog_config.public_api_url.clone(),
|
||||
},
|
||||
shutdown_pageserver,
|
||||
);
|
||||
let inner = Arc::new(inner);
|
||||
// TODO: make this configurable
|
||||
inner.clone().spawn(handle, Duration::from_secs(60));
|
||||
Ok(FeatureResolver { inner: Some(inner) })
|
||||
} else {
|
||||
Ok(FeatureResolver { inner: None })
|
||||
}
|
||||
}
|
||||
|
||||
/// Evaluate a multivariate feature flag. Currently, we do not support any properties.
|
||||
pub fn evaluate_multivariate(
|
||||
&self,
|
||||
flag_key: &str,
|
||||
tenant_id: TenantId,
|
||||
) -> Result<String, PostHogEvaluationError> {
|
||||
if let Some(inner) = &self.inner {
|
||||
inner.feature_store().evaluate_multivariate(
|
||||
flag_key,
|
||||
&tenant_id.to_string(),
|
||||
&HashMap::new(),
|
||||
)
|
||||
} else {
|
||||
Err(PostHogEvaluationError::NotAvailable(
|
||||
"PostHog integration is not enabled".to_string(),
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -10,6 +10,7 @@ pub mod context;
|
||||
pub mod controller_upcall_client;
|
||||
pub mod deletion_queue;
|
||||
pub mod disk_usage_eviction_task;
|
||||
pub mod feature_resolver;
|
||||
pub mod http;
|
||||
pub mod import_datadir;
|
||||
pub mod l0_flush;
|
||||
|
||||
@@ -84,6 +84,7 @@ use crate::context;
|
||||
use crate::context::RequestContextBuilder;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::deletion_queue::{DeletionQueueClient, DeletionQueueError};
|
||||
use crate::feature_resolver::FeatureResolver;
|
||||
use crate::l0_flush::L0FlushGlobalState;
|
||||
use crate::metrics::{
|
||||
BROKEN_TENANTS_SET, CIRCUIT_BREAKERS_BROKEN, CIRCUIT_BREAKERS_UNBROKEN, CONCURRENT_INITDBS,
|
||||
@@ -159,6 +160,7 @@ pub struct TenantSharedResources {
|
||||
pub deletion_queue_client: DeletionQueueClient,
|
||||
pub l0_flush_global_state: L0FlushGlobalState,
|
||||
pub basebackup_prepare_sender: BasebackupPrepareSender,
|
||||
pub feature_resolver: FeatureResolver,
|
||||
}
|
||||
|
||||
/// A [`TenantShard`] is really an _attached_ tenant. The configuration
|
||||
@@ -380,6 +382,8 @@ pub struct TenantShard {
|
||||
pub(crate) gc_block: gc_block::GcBlock,
|
||||
|
||||
l0_flush_global_state: L0FlushGlobalState,
|
||||
|
||||
feature_resolver: FeatureResolver,
|
||||
}
|
||||
impl std::fmt::Debug for TenantShard {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
@@ -1292,6 +1296,7 @@ impl TenantShard {
|
||||
deletion_queue_client,
|
||||
l0_flush_global_state,
|
||||
basebackup_prepare_sender,
|
||||
feature_resolver,
|
||||
} = resources;
|
||||
|
||||
let attach_mode = attached_conf.location.attach_mode;
|
||||
@@ -1308,6 +1313,7 @@ impl TenantShard {
|
||||
deletion_queue_client,
|
||||
l0_flush_global_state,
|
||||
basebackup_prepare_sender,
|
||||
feature_resolver,
|
||||
));
|
||||
|
||||
// The attach task will carry a GateGuard, so that shutdown() reliably waits for it to drop out if
|
||||
@@ -3135,11 +3141,18 @@ impl TenantShard {
|
||||
.or_insert_with(|| Arc::new(GcCompactionQueue::new()))
|
||||
.clone()
|
||||
};
|
||||
let gc_compaction_strategy = self
|
||||
.feature_resolver
|
||||
.evaluate_multivariate("gc-comapction-strategy", self.tenant_shard_id.tenant_id)
|
||||
.ok();
|
||||
let span = if let Some(gc_compaction_strategy) = gc_compaction_strategy {
|
||||
info_span!("gc_compact_timeline", timeline_id = %timeline.timeline_id, strategy = %gc_compaction_strategy)
|
||||
} else {
|
||||
info_span!("gc_compact_timeline", timeline_id = %timeline.timeline_id)
|
||||
};
|
||||
outcome = queue
|
||||
.iteration(cancel, ctx, &self.gc_block, &timeline)
|
||||
.instrument(
|
||||
info_span!("gc_compact_timeline", timeline_id = %timeline.timeline_id),
|
||||
)
|
||||
.instrument(span)
|
||||
.await?;
|
||||
}
|
||||
|
||||
@@ -4247,6 +4260,7 @@ impl TenantShard {
|
||||
deletion_queue_client: DeletionQueueClient,
|
||||
l0_flush_global_state: L0FlushGlobalState,
|
||||
basebackup_prepare_sender: BasebackupPrepareSender,
|
||||
feature_resolver: FeatureResolver,
|
||||
) -> TenantShard {
|
||||
assert!(!attached_conf.location.generation.is_none());
|
||||
|
||||
@@ -4351,6 +4365,7 @@ impl TenantShard {
|
||||
gc_block: Default::default(),
|
||||
l0_flush_global_state,
|
||||
basebackup_prepare_sender,
|
||||
feature_resolver,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -5873,6 +5888,7 @@ pub(crate) mod harness {
|
||||
// TODO: ideally we should run all unit tests with both configs
|
||||
L0FlushGlobalState::new(L0FlushConfig::default()),
|
||||
basebackup_requst_sender,
|
||||
FeatureResolver::new_disabled(),
|
||||
));
|
||||
|
||||
let preload = tenant
|
||||
|
||||
@@ -41,10 +41,8 @@ env_logger = { version = "0.11" }
|
||||
fail = { version = "0.5", default-features = false, features = ["failpoints"] }
|
||||
form_urlencoded = { version = "1" }
|
||||
futures-channel = { version = "0.3", features = ["sink"] }
|
||||
futures-core = { version = "0.3" }
|
||||
futures-executor = { version = "0.3" }
|
||||
futures-io = { version = "0.3" }
|
||||
futures-task = { version = "0.3", default-features = false, features = ["std"] }
|
||||
futures-util = { version = "0.3", features = ["channel", "io", "sink"] }
|
||||
generic-array = { version = "0.14", default-features = false, features = ["more_lengths", "zeroize"] }
|
||||
getrandom = { version = "0.2", default-features = false, features = ["std"] }
|
||||
@@ -74,7 +72,6 @@ num-traits = { version = "0.2", features = ["i128", "libm"] }
|
||||
once_cell = { version = "1" }
|
||||
p256 = { version = "0.13", features = ["jwk"] }
|
||||
parquet = { version = "53", default-features = false, features = ["zstd"] }
|
||||
percent-encoding = { version = "2" }
|
||||
prost = { version = "0.13", features = ["no-recursion-limit", "prost-derive"] }
|
||||
rand = { version = "0.8", features = ["small_rng"] }
|
||||
regex = { version = "1" }
|
||||
|
||||
Reference in New Issue
Block a user