mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-26 06:40:38 +00:00
Compare commits
7 Commits
release-73
...
arpad/blob
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
229157e323 | ||
|
|
b593e51eae | ||
|
|
4c4cb80186 | ||
|
|
92273b6d5e | ||
|
|
e74e7aac93 | ||
|
|
4cca5cdb12 | ||
|
|
9d425b54f7 |
2
.github/actionlint.yml
vendored
2
.github/actionlint.yml
vendored
@@ -21,3 +21,5 @@ config-variables:
|
||||
- SLACK_UPCOMING_RELEASE_CHANNEL_ID
|
||||
- DEV_AWS_OIDC_ROLE_ARN
|
||||
- BENCHMARK_INGEST_TARGET_PROJECTID
|
||||
- PGREGRESS_PG16_PROJECT_ID
|
||||
- PGREGRESS_PG17_PROJECT_ID
|
||||
|
||||
32
.github/workflows/cloud-regress.yml
vendored
32
.github/workflows/cloud-regress.yml
vendored
@@ -23,11 +23,14 @@ jobs:
|
||||
regress:
|
||||
env:
|
||||
POSTGRES_DISTRIB_DIR: /tmp/neon/pg_install
|
||||
DEFAULT_PG_VERSION: 16
|
||||
TEST_OUTPUT: /tmp/test_output
|
||||
BUILD_TYPE: remote
|
||||
AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_DEV }}
|
||||
AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_KEY_DEV }}
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
pg-version: [16, 17]
|
||||
|
||||
runs-on: us-east-2
|
||||
container:
|
||||
@@ -40,9 +43,11 @@ jobs:
|
||||
submodules: true
|
||||
|
||||
- name: Patch the test
|
||||
env:
|
||||
PG_VERSION: ${{matrix.pg-version}}
|
||||
run: |
|
||||
cd "vendor/postgres-v${DEFAULT_PG_VERSION}"
|
||||
patch -p1 < "../../compute/patches/cloud_regress_pg${DEFAULT_PG_VERSION}.patch"
|
||||
cd "vendor/postgres-v${PG_VERSION}"
|
||||
patch -p1 < "../../compute/patches/cloud_regress_pg${PG_VERSION}.patch"
|
||||
|
||||
- name: Generate a random password
|
||||
id: pwgen
|
||||
@@ -55,8 +60,9 @@ jobs:
|
||||
- name: Change tests according to the generated password
|
||||
env:
|
||||
DBPASS: ${{ steps.pwgen.outputs.DBPASS }}
|
||||
PG_VERSION: ${{matrix.pg-version}}
|
||||
run: |
|
||||
cd vendor/postgres-v"${DEFAULT_PG_VERSION}"/src/test/regress
|
||||
cd vendor/postgres-v"${PG_VERSION}"/src/test/regress
|
||||
for fname in sql/*.sql expected/*.out; do
|
||||
sed -i.bak s/NEON_PASSWORD_PLACEHOLDER/"'${DBPASS}'"/ "${fname}"
|
||||
done
|
||||
@@ -73,15 +79,29 @@ jobs:
|
||||
path: /tmp/neon/
|
||||
prefix: latest
|
||||
|
||||
- name: Create a new branch
|
||||
id: create-branch
|
||||
uses: ./.github/actions/neon-branch-create
|
||||
with:
|
||||
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
|
||||
project_id: ${{ vars[format('PGREGRESS_PG{0}_PROJECT_ID', matrix.pg-version)] }}
|
||||
|
||||
- name: Run the regression tests
|
||||
uses: ./.github/actions/run-python-test-set
|
||||
with:
|
||||
build_type: ${{ env.BUILD_TYPE }}
|
||||
test_selection: cloud_regress
|
||||
pg_version: ${{ env.DEFAULT_PG_VERSION }}
|
||||
pg_version: ${{matrix.pg-version}}
|
||||
extra_params: -m remote_cluster
|
||||
env:
|
||||
BENCHMARK_CONNSTR: ${{ secrets.PG_REGRESS_CONNSTR }}
|
||||
BENCHMARK_CONNSTR: ${{steps.create-branch.outputs.dsn}}
|
||||
|
||||
- name: Delete branch
|
||||
uses: ./.github/actions/neon-branch-delete
|
||||
with:
|
||||
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
|
||||
project_id: ${{ vars[format('PGREGRESS_PG{0}_PROJECT_ID', matrix.pg-version)] }}
|
||||
branch_id: ${{steps.create-branch.outputs.branch_id}}
|
||||
|
||||
- name: Create Allure report
|
||||
id: create-allure-report
|
||||
|
||||
515
Cargo.lock
generated
515
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
10
Cargo.toml
10
Cargo.toml
@@ -51,10 +51,6 @@ anyhow = { version = "1.0", features = ["backtrace"] }
|
||||
arc-swap = "1.6"
|
||||
async-compression = { version = "0.4.0", features = ["tokio", "gzip", "zstd"] }
|
||||
atomic-take = "1.1.0"
|
||||
azure_core = { version = "0.19", default-features = false, features = ["enable_reqwest_rustls", "hmac_rust"] }
|
||||
azure_identity = { version = "0.19", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
azure_storage = { version = "0.19", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
azure_storage_blobs = { version = "0.19", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
flate2 = "1.0.26"
|
||||
async-stream = "0.3"
|
||||
async-trait = "0.1"
|
||||
@@ -216,6 +212,12 @@ postgres-protocol = { git = "https://github.com/neondatabase/rust-postgres.git",
|
||||
postgres-types = { git = "https://github.com/neondatabase/rust-postgres.git", branch = "neon" }
|
||||
tokio-postgres = { git = "https://github.com/neondatabase/rust-postgres.git", branch = "neon" }
|
||||
|
||||
## Azure SDK crates
|
||||
azure_core = { git = "https://github.com/neondatabase/azure-sdk-for-rust.git", branch = "arpad/blob_batch", default-features = false, features = ["enable_reqwest_rustls", "hmac_rust"] }
|
||||
azure_identity = { git = "https://github.com/neondatabase/azure-sdk-for-rust.git", branch = "arpad/blob_batch", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
azure_storage = { git = "https://github.com/neondatabase/azure-sdk-for-rust.git", branch = "arpad/blob_batch", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
azure_storage_blobs = { git = "https://github.com/neondatabase/azure-sdk-for-rust.git", branch = "arpad/blob_batch", default-features = false, features = ["enable_reqwest_rustls"] }
|
||||
|
||||
## Local libraries
|
||||
compute_api = { version = "0.1", path = "./libs/compute_api/" }
|
||||
consumption_metrics = { version = "0.1", path = "./libs/consumption_metrics/" }
|
||||
|
||||
4047
compute/patches/cloud_regress_pg17.patch
Normal file
4047
compute/patches/cloud_regress_pg17.patch
Normal file
File diff suppressed because it is too large
Load Diff
@@ -42,6 +42,7 @@ allow = [
|
||||
"MPL-2.0",
|
||||
"OpenSSL",
|
||||
"Unicode-DFS-2016",
|
||||
"Unicode-3.0",
|
||||
]
|
||||
confidence-threshold = 0.8
|
||||
exceptions = [
|
||||
|
||||
@@ -8,15 +8,14 @@ use std::io;
|
||||
use std::num::NonZeroU32;
|
||||
use std::pin::Pin;
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use std::time::SystemTime;
|
||||
|
||||
use super::REMOTE_STORAGE_PREFIX_SEPARATOR;
|
||||
use anyhow::Context;
|
||||
use anyhow::Result;
|
||||
use azure_core::request_options::{IfMatchCondition, MaxResults, Metadata, Range};
|
||||
use azure_core::{Continuable, RetryOptions};
|
||||
use azure_identity::DefaultAzureCredential;
|
||||
use azure_storage::StorageCredentials;
|
||||
use azure_storage_blobs::blob::CopyStatus;
|
||||
use azure_storage_blobs::prelude::ClientBuilder;
|
||||
@@ -76,8 +75,9 @@ impl AzureBlobStorage {
|
||||
let credentials = if let Ok(access_key) = env::var("AZURE_STORAGE_ACCESS_KEY") {
|
||||
StorageCredentials::access_key(account.clone(), access_key)
|
||||
} else {
|
||||
let token_credential = DefaultAzureCredential::default();
|
||||
StorageCredentials::token_credential(Arc::new(token_credential))
|
||||
let token_credential = azure_identity::create_default_credential()
|
||||
.context("trying to obtain Azure default credentials")?;
|
||||
StorageCredentials::token_credential(token_credential)
|
||||
};
|
||||
|
||||
// we have an outer retry
|
||||
@@ -556,7 +556,7 @@ impl RemoteStorage for AzureBlobStorage {
|
||||
let op = async {
|
||||
// TODO batch requests are not supported by the SDK
|
||||
// https://github.com/Azure/azure-sdk-for-rust/issues/1068
|
||||
for path in paths {
|
||||
for path_chunk in paths.chunks(256) {
|
||||
#[derive(Debug)]
|
||||
enum AzureOrTimeout {
|
||||
AzureError(azure_core::Error),
|
||||
@@ -572,13 +572,20 @@ impl RemoteStorage for AzureBlobStorage {
|
||||
let max_retries = 5;
|
||||
backoff::retry(
|
||||
|| async {
|
||||
let blob_client = self.client.blob_client(self.relative_path_to_name(path));
|
||||
let mut batch_client = self.client.blob_batch();
|
||||
for path in path_chunk {
|
||||
batch_client = match batch_client.delete(self.relative_path_to_name(path)) {
|
||||
Ok(batch_client) => batch_client,
|
||||
Err(e) => return Err(AzureOrTimeout::AzureError(e)),
|
||||
};
|
||||
}
|
||||
|
||||
let request = blob_client.delete().into_future();
|
||||
let request = batch_client.into_future();
|
||||
|
||||
let res = tokio::time::timeout(self.timeout, request).await;
|
||||
|
||||
match res {
|
||||
// TODO: validate that all deletions were successful
|
||||
Ok(Ok(_v)) => Ok(()),
|
||||
Ok(Err(azure_err)) => {
|
||||
if let Some(http_err) = azure_err.as_http_error() {
|
||||
|
||||
@@ -2061,7 +2061,7 @@ async fn timeline_compact_handler(
|
||||
let tenant = state
|
||||
.tenant_manager
|
||||
.get_attached_tenant_shard(tenant_shard_id)?;
|
||||
let rx = tenant.schedule_compaction(timeline_id, options).await;
|
||||
let rx = tenant.schedule_compaction(timeline_id, options).await.map_err(ApiError::InternalServerError)?;
|
||||
if wait_until_scheduled_compaction_done {
|
||||
// It is possible that this will take a long time, dropping the HTTP request will not cancel the compaction.
|
||||
rx.await.ok();
|
||||
|
||||
@@ -3028,14 +3028,23 @@ impl Tenant {
|
||||
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
|
||||
let tline_pending_tasks = guard.entry(*timeline_id).or_default();
|
||||
for (idx, job) in jobs.into_iter().enumerate() {
|
||||
tline_pending_tasks.push_back(ScheduledCompactionTask {
|
||||
options: job,
|
||||
result_tx: if idx == jobs_len - 1 {
|
||||
// The last compaction job sends the completion signal
|
||||
next_scheduled_compaction_task.result_tx.take()
|
||||
} else {
|
||||
None
|
||||
},
|
||||
tline_pending_tasks.push_back(if idx == jobs_len - 1 {
|
||||
ScheduledCompactionTask {
|
||||
options: job,
|
||||
// The last job in the queue sends the signal and releases the gc guard
|
||||
result_tx: next_scheduled_compaction_task
|
||||
.result_tx
|
||||
.take(),
|
||||
gc_block: next_scheduled_compaction_task
|
||||
.gc_block
|
||||
.take(),
|
||||
}
|
||||
} else {
|
||||
ScheduledCompactionTask {
|
||||
options: job,
|
||||
result_tx: None,
|
||||
gc_block: None,
|
||||
}
|
||||
});
|
||||
}
|
||||
info!("scheduled enhanced gc bottom-most compaction with sub-compaction, split into {} jobs", jobs_len);
|
||||
@@ -3095,15 +3104,22 @@ impl Tenant {
|
||||
&self,
|
||||
timeline_id: TimelineId,
|
||||
options: CompactOptions,
|
||||
) -> tokio::sync::oneshot::Receiver<()> {
|
||||
) -> anyhow::Result<tokio::sync::oneshot::Receiver<()>> {
|
||||
let gc_guard = match self.gc_block.start().await {
|
||||
Ok(guard) => guard,
|
||||
Err(e) => {
|
||||
bail!("cannot run gc-compaction because gc is blocked: {}", e);
|
||||
}
|
||||
};
|
||||
let (tx, rx) = tokio::sync::oneshot::channel();
|
||||
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
|
||||
let tline_pending_tasks = guard.entry(timeline_id).or_default();
|
||||
tline_pending_tasks.push_back(ScheduledCompactionTask {
|
||||
options,
|
||||
result_tx: Some(tx),
|
||||
gc_block: Some(gc_guard),
|
||||
});
|
||||
rx
|
||||
Ok(rx)
|
||||
}
|
||||
|
||||
// Call through to all timelines to freeze ephemeral layers if needed. Usually
|
||||
@@ -8150,6 +8166,12 @@ mod tests {
|
||||
)
|
||||
.await?;
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x30))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
guard.cutoffs.time = Lsn(0x30);
|
||||
@@ -8252,6 +8274,12 @@ mod tests {
|
||||
|
||||
// increase GC horizon and compact again
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x40))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
guard.cutoffs.time = Lsn(0x40);
|
||||
@@ -8632,6 +8660,12 @@ mod tests {
|
||||
.await?
|
||||
};
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x30))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
@@ -8713,6 +8747,12 @@ mod tests {
|
||||
|
||||
// increase GC horizon and compact again
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x40))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
guard.cutoffs.time = Lsn(0x40);
|
||||
@@ -9160,6 +9200,12 @@ mod tests {
|
||||
)
|
||||
.await?;
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x30))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
@@ -9302,6 +9348,12 @@ mod tests {
|
||||
|
||||
// increase GC horizon and compact again
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x38))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
guard.cutoffs.time = Lsn(0x38);
|
||||
@@ -9397,6 +9449,12 @@ mod tests {
|
||||
)
|
||||
.await?;
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x30))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
@@ -9641,6 +9699,12 @@ mod tests {
|
||||
branch_tline.add_extra_test_dense_keyspace(KeySpace::single(get_key(0)..get_key(10)));
|
||||
|
||||
{
|
||||
parent_tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x10))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = parent_tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
@@ -9655,6 +9719,12 @@ mod tests {
|
||||
}
|
||||
|
||||
{
|
||||
branch_tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x50))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = branch_tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
@@ -9984,6 +10054,12 @@ mod tests {
|
||||
.await?;
|
||||
|
||||
{
|
||||
tline
|
||||
.latest_gc_cutoff_lsn
|
||||
.lock_for_write()
|
||||
.store_and_unlock(Lsn(0x30))
|
||||
.wait()
|
||||
.await;
|
||||
// Update GC info
|
||||
let mut guard = tline.gc_info.write().unwrap();
|
||||
*guard = GcInfo {
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
use std::collections::HashMap;
|
||||
use std::{collections::HashMap, sync::Arc};
|
||||
|
||||
use utils::id::TimelineId;
|
||||
|
||||
@@ -20,7 +20,7 @@ pub(crate) struct GcBlock {
|
||||
/// Do not add any more features taking and forbidding taking this lock. It should be
|
||||
/// `tokio::sync::Notify`, but that is rarely used. On the other side, [`GcBlock::insert`]
|
||||
/// synchronizes with gc attempts by locking and unlocking this mutex.
|
||||
blocking: tokio::sync::Mutex<()>,
|
||||
blocking: Arc<tokio::sync::Mutex<()>>,
|
||||
}
|
||||
|
||||
impl GcBlock {
|
||||
@@ -30,7 +30,7 @@ impl GcBlock {
|
||||
/// it's ending, or if not currently possible, a value describing the reasons why not.
|
||||
///
|
||||
/// Cancellation safe.
|
||||
pub(super) async fn start(&self) -> Result<Guard<'_>, BlockingReasons> {
|
||||
pub(super) async fn start(&self) -> Result<Guard, BlockingReasons> {
|
||||
let reasons = {
|
||||
let g = self.reasons.lock().unwrap();
|
||||
|
||||
@@ -44,7 +44,7 @@ impl GcBlock {
|
||||
Err(reasons)
|
||||
} else {
|
||||
Ok(Guard {
|
||||
_inner: self.blocking.lock().await,
|
||||
_inner: self.blocking.clone().lock_owned().await,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -170,8 +170,8 @@ impl GcBlock {
|
||||
}
|
||||
}
|
||||
|
||||
pub(super) struct Guard<'a> {
|
||||
_inner: tokio::sync::MutexGuard<'a, ()>,
|
||||
pub(crate) struct Guard {
|
||||
_inner: tokio::sync::OwnedMutexGuard<()>,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
|
||||
@@ -41,7 +41,7 @@ use crate::tenant::storage_layer::{
|
||||
use crate::tenant::timeline::ImageLayerCreationOutcome;
|
||||
use crate::tenant::timeline::{drop_rlock, DeltaLayerWriter, ImageLayerWriter};
|
||||
use crate::tenant::timeline::{Layer, ResidentLayer};
|
||||
use crate::tenant::{DeltaLayer, MaybeOffloaded};
|
||||
use crate::tenant::{gc_block, DeltaLayer, MaybeOffloaded};
|
||||
use crate::virtual_file::{MaybeFatalIo, VirtualFile};
|
||||
use pageserver_api::config::tenant_conf_defaults::{
|
||||
DEFAULT_CHECKPOINT_DISTANCE, DEFAULT_COMPACTION_THRESHOLD,
|
||||
@@ -63,9 +63,12 @@ use super::CompactionError;
|
||||
const COMPACTION_DELTA_THRESHOLD: usize = 5;
|
||||
|
||||
/// A scheduled compaction task.
|
||||
pub struct ScheduledCompactionTask {
|
||||
pub(crate) struct ScheduledCompactionTask {
|
||||
pub options: CompactOptions,
|
||||
/// The channel to send the compaction result. If this is a subcompaction, the last compaction job holds the sender.
|
||||
pub result_tx: Option<tokio::sync::oneshot::Sender<()>>,
|
||||
/// Hold the GC block. If this is a subcompaction, the last compaction job holds the gc block guard.
|
||||
pub gc_block: Option<gc_block::Guard>,
|
||||
}
|
||||
|
||||
pub struct GcCompactionJobDescription {
|
||||
@@ -1768,8 +1771,7 @@ impl Timeline {
|
||||
let compact_below_lsn = if let Some(compact_below_lsn) = options.compact_below_lsn {
|
||||
compact_below_lsn
|
||||
} else {
|
||||
let gc_info = self.gc_info.read().unwrap();
|
||||
gc_info.cutoffs.select_min() // use the real gc cutoff
|
||||
*self.get_latest_gc_cutoff_lsn() // use the real gc cutoff
|
||||
};
|
||||
let mut compact_jobs = Vec::new();
|
||||
// For now, we simply use the key partitioning information; we should do a more fine-grained partitioning
|
||||
@@ -1962,7 +1964,11 @@ impl Timeline {
|
||||
let gc_info = self.gc_info.read().unwrap();
|
||||
let mut retain_lsns_below_horizon = Vec::new();
|
||||
let gc_cutoff = {
|
||||
let real_gc_cutoff = gc_info.cutoffs.select_min();
|
||||
// Currently, gc-compaction only kicks in after the legacy gc has updated the gc_cutoff.
|
||||
// Therefore, it can only clean up data that cannot be cleaned up with legacy gc, instead of
|
||||
// cleaning everything that theoritically it could. In the future, it should use `self.gc_info`
|
||||
// to get the truth data.
|
||||
let real_gc_cutoff = *self.get_latest_gc_cutoff_lsn();
|
||||
// The compaction algorithm will keep all keys above the gc_cutoff while keeping only necessary keys below the gc_cutoff for
|
||||
// each of the retain_lsn. Therefore, if the user-provided `compact_below_lsn` is larger than the real gc cutoff, we will use
|
||||
// the real cutoff.
|
||||
|
||||
@@ -83,14 +83,20 @@ impl Env {
|
||||
node_id: NodeId,
|
||||
ttid: TenantTimelineId,
|
||||
) -> anyhow::Result<Arc<Timeline>> {
|
||||
let conf = self.make_conf(node_id);
|
||||
let conf = Arc::new(self.make_conf(node_id));
|
||||
let timeline_dir = get_timeline_dir(&conf, &ttid);
|
||||
let remote_path = remote_timeline_path(&ttid)?;
|
||||
|
||||
let safekeeper = self.make_safekeeper(node_id, ttid).await?;
|
||||
let shared_state = SharedState::new(StateSK::Loaded(safekeeper));
|
||||
|
||||
let timeline = Timeline::new(ttid, &timeline_dir, &remote_path, shared_state);
|
||||
let timeline = Timeline::new(
|
||||
ttid,
|
||||
&timeline_dir,
|
||||
&remote_path,
|
||||
shared_state,
|
||||
conf.clone(),
|
||||
);
|
||||
timeline.bootstrap(
|
||||
&mut timeline.write_shared_state().await,
|
||||
&conf,
|
||||
|
||||
@@ -338,7 +338,7 @@ async fn main() -> anyhow::Result<()> {
|
||||
}
|
||||
};
|
||||
|
||||
let conf = SafeKeeperConf {
|
||||
let conf = Arc::new(SafeKeeperConf {
|
||||
workdir,
|
||||
my_id: id,
|
||||
listen_pg_addr: args.listen_pg,
|
||||
@@ -368,7 +368,7 @@ async fn main() -> anyhow::Result<()> {
|
||||
control_file_save_interval: args.control_file_save_interval,
|
||||
partial_backup_concurrency: args.partial_backup_concurrency,
|
||||
eviction_min_resident: args.eviction_min_resident,
|
||||
};
|
||||
});
|
||||
|
||||
// initialize sentry if SENTRY_DSN is provided
|
||||
let _sentry_guard = init_sentry(
|
||||
@@ -382,7 +382,7 @@ async fn main() -> anyhow::Result<()> {
|
||||
/// complete, e.g. panicked, inner is error produced by task itself.
|
||||
type JoinTaskRes = Result<anyhow::Result<()>, JoinError>;
|
||||
|
||||
async fn start_safekeeper(conf: SafeKeeperConf) -> Result<()> {
|
||||
async fn start_safekeeper(conf: Arc<SafeKeeperConf>) -> Result<()> {
|
||||
// fsync the datadir to make sure we have a consistent state on disk.
|
||||
if !conf.no_sync {
|
||||
let dfd = File::open(&conf.workdir).context("open datadir for syncfs")?;
|
||||
@@ -428,9 +428,11 @@ async fn start_safekeeper(conf: SafeKeeperConf) -> Result<()> {
|
||||
e
|
||||
})?;
|
||||
|
||||
let global_timelines = Arc::new(GlobalTimelines::new(conf.clone()));
|
||||
|
||||
// Register metrics collector for active timelines. It's important to do this
|
||||
// after daemonizing, otherwise process collector will be upset.
|
||||
let timeline_collector = safekeeper::metrics::TimelineCollector::new();
|
||||
let timeline_collector = safekeeper::metrics::TimelineCollector::new(global_timelines.clone());
|
||||
metrics::register_internal(Box::new(timeline_collector))?;
|
||||
|
||||
wal_backup::init_remote_storage(&conf).await;
|
||||
@@ -447,9 +449,8 @@ async fn start_safekeeper(conf: SafeKeeperConf) -> Result<()> {
|
||||
.then(|| Handle::try_current().expect("no runtime in main"));
|
||||
|
||||
// Load all timelines from disk to memory.
|
||||
GlobalTimelines::init(conf.clone()).await?;
|
||||
global_timelines.init().await?;
|
||||
|
||||
let conf_ = conf.clone();
|
||||
// Run everything in current thread rt, if asked.
|
||||
if conf.current_thread_runtime {
|
||||
info!("running in current thread runtime");
|
||||
@@ -459,14 +460,16 @@ async fn start_safekeeper(conf: SafeKeeperConf) -> Result<()> {
|
||||
.as_ref()
|
||||
.unwrap_or_else(|| WAL_SERVICE_RUNTIME.handle())
|
||||
.spawn(wal_service::task_main(
|
||||
conf_,
|
||||
conf.clone(),
|
||||
pg_listener,
|
||||
Scope::SafekeeperData,
|
||||
global_timelines.clone(),
|
||||
))
|
||||
// wrap with task name for error reporting
|
||||
.map(|res| ("WAL service main".to_owned(), res));
|
||||
tasks_handles.push(Box::pin(wal_service_handle));
|
||||
|
||||
let global_timelines_ = global_timelines.clone();
|
||||
let timeline_housekeeping_handle = current_thread_rt
|
||||
.as_ref()
|
||||
.unwrap_or_else(|| WAL_SERVICE_RUNTIME.handle())
|
||||
@@ -474,40 +477,45 @@ async fn start_safekeeper(conf: SafeKeeperConf) -> Result<()> {
|
||||
const TOMBSTONE_TTL: Duration = Duration::from_secs(3600 * 24);
|
||||
loop {
|
||||
tokio::time::sleep(TOMBSTONE_TTL).await;
|
||||
GlobalTimelines::housekeeping(&TOMBSTONE_TTL);
|
||||
global_timelines_.housekeeping(&TOMBSTONE_TTL);
|
||||
}
|
||||
})
|
||||
.map(|res| ("Timeline map housekeeping".to_owned(), res));
|
||||
tasks_handles.push(Box::pin(timeline_housekeeping_handle));
|
||||
|
||||
if let Some(pg_listener_tenant_only) = pg_listener_tenant_only {
|
||||
let conf_ = conf.clone();
|
||||
let wal_service_handle = current_thread_rt
|
||||
.as_ref()
|
||||
.unwrap_or_else(|| WAL_SERVICE_RUNTIME.handle())
|
||||
.spawn(wal_service::task_main(
|
||||
conf_,
|
||||
conf.clone(),
|
||||
pg_listener_tenant_only,
|
||||
Scope::Tenant,
|
||||
global_timelines.clone(),
|
||||
))
|
||||
// wrap with task name for error reporting
|
||||
.map(|res| ("WAL service tenant only main".to_owned(), res));
|
||||
tasks_handles.push(Box::pin(wal_service_handle));
|
||||
}
|
||||
|
||||
let conf_ = conf.clone();
|
||||
let http_handle = current_thread_rt
|
||||
.as_ref()
|
||||
.unwrap_or_else(|| HTTP_RUNTIME.handle())
|
||||
.spawn(http::task_main(conf_, http_listener))
|
||||
.spawn(http::task_main(
|
||||
conf.clone(),
|
||||
http_listener,
|
||||
global_timelines.clone(),
|
||||
))
|
||||
.map(|res| ("HTTP service main".to_owned(), res));
|
||||
tasks_handles.push(Box::pin(http_handle));
|
||||
|
||||
let conf_ = conf.clone();
|
||||
let broker_task_handle = current_thread_rt
|
||||
.as_ref()
|
||||
.unwrap_or_else(|| BROKER_RUNTIME.handle())
|
||||
.spawn(broker::task_main(conf_).instrument(info_span!("broker")))
|
||||
.spawn(
|
||||
broker::task_main(conf.clone(), global_timelines.clone())
|
||||
.instrument(info_span!("broker")),
|
||||
)
|
||||
.map(|res| ("broker main".to_owned(), res));
|
||||
tasks_handles.push(Box::pin(broker_task_handle));
|
||||
|
||||
|
||||
@@ -39,14 +39,17 @@ const RETRY_INTERVAL_MSEC: u64 = 1000;
|
||||
const PUSH_INTERVAL_MSEC: u64 = 1000;
|
||||
|
||||
/// Push once in a while data about all active timelines to the broker.
|
||||
async fn push_loop(conf: SafeKeeperConf) -> anyhow::Result<()> {
|
||||
async fn push_loop(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> anyhow::Result<()> {
|
||||
if conf.disable_periodic_broker_push {
|
||||
info!("broker push_loop is disabled, doing nothing...");
|
||||
futures::future::pending::<()>().await; // sleep forever
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
let active_timelines_set = GlobalTimelines::get_global_broker_active_set();
|
||||
let active_timelines_set = global_timelines.get_global_broker_active_set();
|
||||
|
||||
let mut client =
|
||||
storage_broker::connect(conf.broker_endpoint.clone(), conf.broker_keepalive_interval)?;
|
||||
@@ -87,8 +90,13 @@ async fn push_loop(conf: SafeKeeperConf) -> anyhow::Result<()> {
|
||||
|
||||
/// Subscribe and fetch all the interesting data from the broker.
|
||||
#[instrument(name = "broker_pull", skip_all)]
|
||||
async fn pull_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<()> {
|
||||
let mut client = storage_broker::connect(conf.broker_endpoint, conf.broker_keepalive_interval)?;
|
||||
async fn pull_loop(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
stats: Arc<BrokerStats>,
|
||||
) -> Result<()> {
|
||||
let mut client =
|
||||
storage_broker::connect(conf.broker_endpoint.clone(), conf.broker_keepalive_interval)?;
|
||||
|
||||
// TODO: subscribe only to local timelines instead of all
|
||||
let request = SubscribeSafekeeperInfoRequest {
|
||||
@@ -113,7 +121,7 @@ async fn pull_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<()>
|
||||
.as_ref()
|
||||
.ok_or_else(|| anyhow!("missing tenant_timeline_id"))?;
|
||||
let ttid = parse_proto_ttid(proto_ttid)?;
|
||||
if let Ok(tli) = GlobalTimelines::get(ttid) {
|
||||
if let Ok(tli) = global_timelines.get(ttid) {
|
||||
// Note that we also receive *our own* info. That's
|
||||
// important, as it is used as an indication of live
|
||||
// connection to the broker.
|
||||
@@ -135,7 +143,11 @@ async fn pull_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<()>
|
||||
|
||||
/// Process incoming discover requests. This is done in a separate task to avoid
|
||||
/// interfering with the normal pull/push loops.
|
||||
async fn discover_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<()> {
|
||||
async fn discover_loop(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
stats: Arc<BrokerStats>,
|
||||
) -> Result<()> {
|
||||
let mut client =
|
||||
storage_broker::connect(conf.broker_endpoint.clone(), conf.broker_keepalive_interval)?;
|
||||
|
||||
@@ -171,7 +183,7 @@ async fn discover_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<
|
||||
.as_ref()
|
||||
.ok_or_else(|| anyhow!("missing tenant_timeline_id"))?;
|
||||
let ttid = parse_proto_ttid(proto_ttid)?;
|
||||
if let Ok(tli) = GlobalTimelines::get(ttid) {
|
||||
if let Ok(tli) = global_timelines.get(ttid) {
|
||||
// we received a discovery request for a timeline we know about
|
||||
discover_counter.inc();
|
||||
|
||||
@@ -210,7 +222,10 @@ async fn discover_loop(conf: SafeKeeperConf, stats: Arc<BrokerStats>) -> Result<
|
||||
bail!("end of stream");
|
||||
}
|
||||
|
||||
pub async fn task_main(conf: SafeKeeperConf) -> anyhow::Result<()> {
|
||||
pub async fn task_main(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> anyhow::Result<()> {
|
||||
info!("started, broker endpoint {:?}", conf.broker_endpoint);
|
||||
|
||||
let mut ticker = tokio::time::interval(Duration::from_millis(RETRY_INTERVAL_MSEC));
|
||||
@@ -261,13 +276,13 @@ pub async fn task_main(conf: SafeKeeperConf) -> anyhow::Result<()> {
|
||||
},
|
||||
_ = ticker.tick() => {
|
||||
if push_handle.is_none() {
|
||||
push_handle = Some(tokio::spawn(push_loop(conf.clone())));
|
||||
push_handle = Some(tokio::spawn(push_loop(conf.clone(), global_timelines.clone())));
|
||||
}
|
||||
if pull_handle.is_none() {
|
||||
pull_handle = Some(tokio::spawn(pull_loop(conf.clone(), stats.clone())));
|
||||
pull_handle = Some(tokio::spawn(pull_loop(conf.clone(), global_timelines.clone(), stats.clone())));
|
||||
}
|
||||
if discover_handle.is_none() {
|
||||
discover_handle = Some(tokio::spawn(discover_loop(conf.clone(), stats.clone())));
|
||||
discover_handle = Some(tokio::spawn(discover_loop(conf.clone(), global_timelines.clone(), stats.clone())));
|
||||
}
|
||||
},
|
||||
_ = &mut stats_task => {}
|
||||
|
||||
@@ -1,9 +1,7 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use anyhow::{bail, Result};
|
||||
use camino::Utf8PathBuf;
|
||||
|
||||
use postgres_ffi::{MAX_SEND_SIZE, WAL_SEGMENT_SIZE};
|
||||
use std::sync::Arc;
|
||||
use tokio::{
|
||||
fs::OpenOptions,
|
||||
io::{AsyncSeekExt, AsyncWriteExt},
|
||||
@@ -14,7 +12,7 @@ use utils::{id::TenantTimelineId, lsn::Lsn};
|
||||
use crate::{
|
||||
control_file::FileStorage,
|
||||
state::TimelinePersistentState,
|
||||
timeline::{Timeline, TimelineError, WalResidentTimeline},
|
||||
timeline::{TimelineError, WalResidentTimeline},
|
||||
timelines_global_map::{create_temp_timeline_dir, validate_temp_timeline},
|
||||
wal_backup::copy_s3_segments,
|
||||
wal_storage::{wal_file_paths, WalReader},
|
||||
@@ -25,16 +23,19 @@ use crate::{
|
||||
const MAX_BACKUP_LAG: u64 = 10 * WAL_SEGMENT_SIZE as u64;
|
||||
|
||||
pub struct Request {
|
||||
pub source: Arc<Timeline>,
|
||||
pub source_ttid: TenantTimelineId,
|
||||
pub until_lsn: Lsn,
|
||||
pub destination_ttid: TenantTimelineId,
|
||||
}
|
||||
|
||||
pub async fn handle_request(request: Request) -> Result<()> {
|
||||
pub async fn handle_request(
|
||||
request: Request,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> Result<()> {
|
||||
// TODO: request.until_lsn MUST be a valid LSN, and we cannot check it :(
|
||||
// if LSN will point to the middle of a WAL record, timeline will be in "broken" state
|
||||
|
||||
match GlobalTimelines::get(request.destination_ttid) {
|
||||
match global_timelines.get(request.destination_ttid) {
|
||||
// timeline already exists. would be good to check that this timeline is the copy
|
||||
// of the source timeline, but it isn't obvious how to do that
|
||||
Ok(_) => return Ok(()),
|
||||
@@ -46,9 +47,10 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
}
|
||||
}
|
||||
|
||||
let source_tli = request.source.wal_residence_guard().await?;
|
||||
let source = global_timelines.get(request.source_ttid)?;
|
||||
let source_tli = source.wal_residence_guard().await?;
|
||||
|
||||
let conf = &GlobalTimelines::get_global_config();
|
||||
let conf = &global_timelines.get_global_config();
|
||||
let ttid = request.destination_ttid;
|
||||
|
||||
let (_tmp_dir, tli_dir_path) = create_temp_timeline_dir(conf, ttid).await?;
|
||||
@@ -127,7 +129,7 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
|
||||
copy_s3_segments(
|
||||
wal_seg_size,
|
||||
&request.source.ttid,
|
||||
&request.source_ttid,
|
||||
&request.destination_ttid,
|
||||
first_segment,
|
||||
first_ondisk_segment,
|
||||
@@ -158,7 +160,9 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
|
||||
// now we have a ready timeline in a temp directory
|
||||
validate_temp_timeline(conf, request.destination_ttid, &tli_dir_path).await?;
|
||||
GlobalTimelines::load_temp_timeline(request.destination_ttid, &tli_dir_path, true).await?;
|
||||
global_timelines
|
||||
.load_temp_timeline(request.destination_ttid, &tli_dir_path, true)
|
||||
.await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -207,23 +207,23 @@ pub struct FileInfo {
|
||||
}
|
||||
|
||||
/// Build debug dump response, using the provided [`Args`] filters.
|
||||
pub async fn build(args: Args) -> Result<Response> {
|
||||
pub async fn build(args: Args, global_timelines: Arc<GlobalTimelines>) -> Result<Response> {
|
||||
let start_time = Utc::now();
|
||||
let timelines_count = GlobalTimelines::timelines_count();
|
||||
let config = GlobalTimelines::get_global_config();
|
||||
let timelines_count = global_timelines.timelines_count();
|
||||
let config = global_timelines.get_global_config();
|
||||
|
||||
let ptrs_snapshot = if args.tenant_id.is_some() && args.timeline_id.is_some() {
|
||||
// If both tenant_id and timeline_id are specified, we can just get the
|
||||
// timeline directly, without taking a snapshot of the whole list.
|
||||
let ttid = TenantTimelineId::new(args.tenant_id.unwrap(), args.timeline_id.unwrap());
|
||||
if let Ok(tli) = GlobalTimelines::get(ttid) {
|
||||
if let Ok(tli) = global_timelines.get(ttid) {
|
||||
vec![tli]
|
||||
} else {
|
||||
vec![]
|
||||
}
|
||||
} else {
|
||||
// Otherwise, take a snapshot of the whole list.
|
||||
GlobalTimelines::get_all()
|
||||
global_timelines.get_all()
|
||||
};
|
||||
|
||||
let mut timelines = Vec::new();
|
||||
@@ -344,12 +344,12 @@ fn get_wal_last_modified(path: &Utf8Path) -> Result<Option<DateTime<Utc>>> {
|
||||
|
||||
/// Converts SafeKeeperConf to Config, filtering out the fields that are not
|
||||
/// supposed to be exposed.
|
||||
fn build_config(config: SafeKeeperConf) -> Config {
|
||||
fn build_config(config: Arc<SafeKeeperConf>) -> Config {
|
||||
Config {
|
||||
id: config.my_id,
|
||||
workdir: config.workdir.into(),
|
||||
listen_pg_addr: config.listen_pg_addr,
|
||||
listen_http_addr: config.listen_http_addr,
|
||||
workdir: config.workdir.clone().into(),
|
||||
listen_pg_addr: config.listen_pg_addr.clone(),
|
||||
listen_http_addr: config.listen_http_addr.clone(),
|
||||
no_sync: config.no_sync,
|
||||
max_offloader_lag_bytes: config.max_offloader_lag_bytes,
|
||||
wal_backup_enabled: config.wal_backup_enabled,
|
||||
|
||||
@@ -33,7 +33,7 @@ use utils::{
|
||||
|
||||
/// Safekeeper handler of postgres commands
|
||||
pub struct SafekeeperPostgresHandler {
|
||||
pub conf: SafeKeeperConf,
|
||||
pub conf: Arc<SafeKeeperConf>,
|
||||
/// assigned application name
|
||||
pub appname: Option<String>,
|
||||
pub tenant_id: Option<TenantId>,
|
||||
@@ -43,6 +43,7 @@ pub struct SafekeeperPostgresHandler {
|
||||
pub protocol: Option<PostgresClientProtocol>,
|
||||
/// Unique connection id is logged in spans for observability.
|
||||
pub conn_id: ConnectionId,
|
||||
pub global_timelines: Arc<GlobalTimelines>,
|
||||
/// Auth scope allowed on the connections and public key used to check auth tokens. None if auth is not configured.
|
||||
auth: Option<(Scope, Arc<JwtAuth>)>,
|
||||
claims: Option<Claims>,
|
||||
@@ -314,10 +315,11 @@ impl<IO: AsyncRead + AsyncWrite + Unpin + Send> postgres_backend::Handler<IO>
|
||||
|
||||
impl SafekeeperPostgresHandler {
|
||||
pub fn new(
|
||||
conf: SafeKeeperConf,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
conn_id: u32,
|
||||
io_metrics: Option<TrafficMetrics>,
|
||||
auth: Option<(Scope, Arc<JwtAuth>)>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> Self {
|
||||
SafekeeperPostgresHandler {
|
||||
conf,
|
||||
@@ -331,6 +333,7 @@ impl SafekeeperPostgresHandler {
|
||||
claims: None,
|
||||
auth,
|
||||
io_metrics,
|
||||
global_timelines,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -360,7 +363,7 @@ impl SafekeeperPostgresHandler {
|
||||
pgb: &mut PostgresBackend<IO>,
|
||||
) -> Result<(), QueryError> {
|
||||
// Get timeline, handling "not found" error
|
||||
let tli = match GlobalTimelines::get(self.ttid) {
|
||||
let tli = match self.global_timelines.get(self.ttid) {
|
||||
Ok(tli) => Ok(Some(tli)),
|
||||
Err(TimelineError::NotFound(_)) => Ok(None),
|
||||
Err(e) => Err(QueryError::Other(e.into())),
|
||||
@@ -394,7 +397,10 @@ impl SafekeeperPostgresHandler {
|
||||
&mut self,
|
||||
pgb: &mut PostgresBackend<IO>,
|
||||
) -> Result<(), QueryError> {
|
||||
let tli = GlobalTimelines::get(self.ttid).map_err(|e| QueryError::Other(e.into()))?;
|
||||
let tli = self
|
||||
.global_timelines
|
||||
.get(self.ttid)
|
||||
.map_err(|e| QueryError::Other(e.into()))?;
|
||||
|
||||
let lsn = if self.is_walproposer_recovery() {
|
||||
// walproposer should get all local WAL until flush_lsn
|
||||
|
||||
@@ -3,14 +3,16 @@ pub mod routes;
|
||||
pub use routes::make_router;
|
||||
|
||||
pub use safekeeper_api::models;
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::SafeKeeperConf;
|
||||
use crate::{GlobalTimelines, SafeKeeperConf};
|
||||
|
||||
pub async fn task_main(
|
||||
conf: SafeKeeperConf,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
http_listener: std::net::TcpListener,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> anyhow::Result<()> {
|
||||
let router = make_router(conf)
|
||||
let router = make_router(conf, global_timelines)
|
||||
.build()
|
||||
.map_err(|err| anyhow::anyhow!(err))?;
|
||||
let service = utils::http::RouterService::new(router).unwrap();
|
||||
|
||||
@@ -66,6 +66,13 @@ fn get_conf(request: &Request<Body>) -> &SafeKeeperConf {
|
||||
.as_ref()
|
||||
}
|
||||
|
||||
fn get_global_timelines(request: &Request<Body>) -> Arc<GlobalTimelines> {
|
||||
request
|
||||
.data::<Arc<GlobalTimelines>>()
|
||||
.expect("unknown state type")
|
||||
.clone()
|
||||
}
|
||||
|
||||
/// Same as TermLsn, but serializes LSN using display serializer
|
||||
/// in Postgres format, i.e. 0/FFFFFFFF. Used only for the API response.
|
||||
#[derive(Debug, Clone, Copy, Serialize, Deserialize)]
|
||||
@@ -123,9 +130,11 @@ async fn tenant_delete_handler(mut request: Request<Body>) -> Result<Response<Bo
|
||||
let only_local = parse_query_param(&request, "only_local")?.unwrap_or(false);
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
ensure_no_body(&mut request).await?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
// FIXME: `delete_force_all_for_tenant` can return an error for multiple different reasons;
|
||||
// Using an `InternalServerError` should be fixed when the types support it
|
||||
let delete_info = GlobalTimelines::delete_force_all_for_tenant(&tenant_id, only_local)
|
||||
let delete_info = global_timelines
|
||||
.delete_force_all_for_tenant(&tenant_id, only_local)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
json_response(
|
||||
@@ -156,7 +165,9 @@ async fn timeline_create_handler(mut request: Request<Body>) -> Result<Response<
|
||||
.commit_lsn
|
||||
.segment_lsn(server_info.wal_seg_size as usize)
|
||||
});
|
||||
GlobalTimelines::create(ttid, server_info, request_data.commit_lsn, local_start_lsn)
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
global_timelines
|
||||
.create(ttid, server_info, request_data.commit_lsn, local_start_lsn)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -167,7 +178,9 @@ async fn timeline_create_handler(mut request: Request<Body>) -> Result<Response<
|
||||
/// Note: it is possible to do the same with debug_dump.
|
||||
async fn timeline_list_handler(request: Request<Body>) -> Result<Response<Body>, ApiError> {
|
||||
check_permission(&request, None)?;
|
||||
let res: Vec<TenantTimelineId> = GlobalTimelines::get_all()
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let res: Vec<TenantTimelineId> = global_timelines
|
||||
.get_all()
|
||||
.iter()
|
||||
.map(|tli| tli.ttid)
|
||||
.collect();
|
||||
@@ -182,7 +195,8 @@ async fn timeline_status_handler(request: Request<Body>) -> Result<Response<Body
|
||||
);
|
||||
check_permission(&request, Some(ttid.tenant_id))?;
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
let (inmem, state) = tli.get_state().await;
|
||||
let flush_lsn = tli.get_flush_lsn().await;
|
||||
|
||||
@@ -233,9 +247,11 @@ async fn timeline_delete_handler(mut request: Request<Body>) -> Result<Response<
|
||||
let only_local = parse_query_param(&request, "only_local")?.unwrap_or(false);
|
||||
check_permission(&request, Some(ttid.tenant_id))?;
|
||||
ensure_no_body(&mut request).await?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
// FIXME: `delete_force` can fail from both internal errors and bad requests. Add better
|
||||
// error handling here when we're able to.
|
||||
let resp = GlobalTimelines::delete(&ttid, only_local)
|
||||
let resp = global_timelines
|
||||
.delete(&ttid, only_local)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
json_response(StatusCode::OK, resp)
|
||||
@@ -247,8 +263,9 @@ async fn timeline_pull_handler(mut request: Request<Body>) -> Result<Response<Bo
|
||||
|
||||
let data: pull_timeline::Request = json_request(&mut request).await?;
|
||||
let conf = get_conf(&request);
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
|
||||
let resp = pull_timeline::handle_request(data, conf.sk_auth_token.clone())
|
||||
let resp = pull_timeline::handle_request(data, conf.sk_auth_token.clone(), global_timelines)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
json_response(StatusCode::OK, resp)
|
||||
@@ -263,7 +280,8 @@ async fn timeline_snapshot_handler(request: Request<Body>) -> Result<Response<Bo
|
||||
);
|
||||
check_permission(&request, Some(ttid.tenant_id))?;
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
|
||||
// To stream the body use wrap_stream which wants Stream of Result<Bytes>,
|
||||
// so create the chan and write to it in another task.
|
||||
@@ -293,19 +311,19 @@ async fn timeline_copy_handler(mut request: Request<Body>) -> Result<Response<Bo
|
||||
check_permission(&request, None)?;
|
||||
|
||||
let request_data: TimelineCopyRequest = json_request(&mut request).await?;
|
||||
let ttid = TenantTimelineId::new(
|
||||
let source_ttid = TenantTimelineId::new(
|
||||
parse_request_param(&request, "tenant_id")?,
|
||||
parse_request_param(&request, "source_timeline_id")?,
|
||||
);
|
||||
|
||||
let source = GlobalTimelines::get(ttid)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
|
||||
copy_timeline::handle_request(copy_timeline::Request{
|
||||
source,
|
||||
source_ttid,
|
||||
until_lsn: request_data.until_lsn,
|
||||
destination_ttid: TenantTimelineId::new(ttid.tenant_id, request_data.target_timeline_id),
|
||||
})
|
||||
.instrument(info_span!("copy_timeline", from=%ttid, to=%request_data.target_timeline_id, until_lsn=%request_data.until_lsn))
|
||||
destination_ttid: TenantTimelineId::new(source_ttid.tenant_id, request_data.target_timeline_id),
|
||||
}, global_timelines)
|
||||
.instrument(info_span!("copy_timeline", from=%source_ttid, to=%request_data.target_timeline_id, until_lsn=%request_data.until_lsn))
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -322,7 +340,8 @@ async fn patch_control_file_handler(
|
||||
parse_request_param(&request, "timeline_id")?,
|
||||
);
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
|
||||
let patch_request: patch_control_file::Request = json_request(&mut request).await?;
|
||||
let response = patch_control_file::handle_request(tli, patch_request)
|
||||
@@ -341,7 +360,8 @@ async fn timeline_checkpoint_handler(request: Request<Body>) -> Result<Response<
|
||||
parse_request_param(&request, "timeline_id")?,
|
||||
);
|
||||
|
||||
let tli = GlobalTimelines::get(ttid)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid)?;
|
||||
tli.write_shared_state()
|
||||
.await
|
||||
.sk
|
||||
@@ -359,6 +379,7 @@ async fn timeline_digest_handler(request: Request<Body>) -> Result<Response<Body
|
||||
);
|
||||
check_permission(&request, Some(ttid.tenant_id))?;
|
||||
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let from_lsn: Option<Lsn> = parse_query_param(&request, "from_lsn")?;
|
||||
let until_lsn: Option<Lsn> = parse_query_param(&request, "until_lsn")?;
|
||||
|
||||
@@ -371,7 +392,7 @@ async fn timeline_digest_handler(request: Request<Body>) -> Result<Response<Body
|
||||
)))?,
|
||||
};
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
let tli = tli
|
||||
.wal_residence_guard()
|
||||
.await
|
||||
@@ -393,7 +414,8 @@ async fn timeline_backup_partial_reset(request: Request<Body>) -> Result<Respons
|
||||
);
|
||||
check_permission(&request, Some(ttid.tenant_id))?;
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
|
||||
let response = tli
|
||||
.backup_partial_reset()
|
||||
@@ -415,7 +437,8 @@ async fn timeline_term_bump_handler(
|
||||
|
||||
let request_data: TimelineTermBumpRequest = json_request(&mut request).await?;
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
let response = tli
|
||||
.term_bump(request_data.term)
|
||||
.await
|
||||
@@ -452,7 +475,8 @@ async fn record_safekeeper_info(mut request: Request<Body>) -> Result<Response<B
|
||||
standby_horizon: sk_info.standby_horizon.0,
|
||||
};
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
let tli = global_timelines.get(ttid).map_err(ApiError::from)?;
|
||||
tli.record_safekeeper_info(proto_sk_info)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
@@ -506,6 +530,8 @@ async fn dump_debug_handler(mut request: Request<Body>) -> Result<Response<Body>
|
||||
let dump_term_history = dump_term_history.unwrap_or(true);
|
||||
let dump_wal_last_modified = dump_wal_last_modified.unwrap_or(dump_all);
|
||||
|
||||
let global_timelines = get_global_timelines(&request);
|
||||
|
||||
let args = debug_dump::Args {
|
||||
dump_all,
|
||||
dump_control_file,
|
||||
@@ -517,7 +543,7 @@ async fn dump_debug_handler(mut request: Request<Body>) -> Result<Response<Body>
|
||||
timeline_id,
|
||||
};
|
||||
|
||||
let resp = debug_dump::build(args)
|
||||
let resp = debug_dump::build(args, global_timelines)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -570,7 +596,10 @@ async fn dump_debug_handler(mut request: Request<Body>) -> Result<Response<Body>
|
||||
}
|
||||
|
||||
/// Safekeeper http router.
|
||||
pub fn make_router(conf: SafeKeeperConf) -> RouterBuilder<hyper::Body, ApiError> {
|
||||
pub fn make_router(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> RouterBuilder<hyper::Body, ApiError> {
|
||||
let mut router = endpoint::make_router();
|
||||
if conf.http_auth.is_some() {
|
||||
router = router.middleware(auth_middleware(|request| {
|
||||
@@ -592,7 +621,8 @@ pub fn make_router(conf: SafeKeeperConf) -> RouterBuilder<hyper::Body, ApiError>
|
||||
// located nearby (/safekeeper/src/http/openapi_spec.yaml).
|
||||
let auth = conf.http_auth.clone();
|
||||
router
|
||||
.data(Arc::new(conf))
|
||||
.data(conf)
|
||||
.data(global_timelines)
|
||||
.data(auth)
|
||||
.get("/metrics", |r| request_span(r, prometheus_metrics_handler))
|
||||
.get("/profile/cpu", |r| request_span(r, profile_cpu_handler))
|
||||
|
||||
@@ -11,7 +11,6 @@ use postgres_backend::QueryError;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tokio::io::{AsyncRead, AsyncWrite};
|
||||
use tracing::*;
|
||||
use utils::id::TenantTimelineId;
|
||||
|
||||
use crate::handler::SafekeeperPostgresHandler;
|
||||
use crate::safekeeper::{AcceptorProposerMessage, AppendResponse, ServerInfo};
|
||||
@@ -21,7 +20,6 @@ use crate::safekeeper::{
|
||||
use crate::safekeeper::{Term, TermHistory, TermLsn};
|
||||
use crate::state::TimelinePersistentState;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::GlobalTimelines;
|
||||
use postgres_backend::PostgresBackend;
|
||||
use postgres_ffi::encode_logical_message;
|
||||
use postgres_ffi::WAL_SEGMENT_SIZE;
|
||||
@@ -70,7 +68,7 @@ pub async fn handle_json_ctrl<IO: AsyncRead + AsyncWrite + Unpin>(
|
||||
info!("JSON_CTRL request: {append_request:?}");
|
||||
|
||||
// need to init safekeeper state before AppendRequest
|
||||
let tli = prepare_safekeeper(spg.ttid, append_request.pg_version).await?;
|
||||
let tli = prepare_safekeeper(spg, append_request.pg_version).await?;
|
||||
|
||||
// if send_proposer_elected is true, we need to update local history
|
||||
if append_request.send_proposer_elected {
|
||||
@@ -99,20 +97,22 @@ pub async fn handle_json_ctrl<IO: AsyncRead + AsyncWrite + Unpin>(
|
||||
/// Prepare safekeeper to process append requests without crashes,
|
||||
/// by sending ProposerGreeting with default server.wal_seg_size.
|
||||
async fn prepare_safekeeper(
|
||||
ttid: TenantTimelineId,
|
||||
spg: &SafekeeperPostgresHandler,
|
||||
pg_version: u32,
|
||||
) -> anyhow::Result<WalResidentTimeline> {
|
||||
let tli = GlobalTimelines::create(
|
||||
ttid,
|
||||
ServerInfo {
|
||||
pg_version,
|
||||
wal_seg_size: WAL_SEGMENT_SIZE as u32,
|
||||
system_id: 0,
|
||||
},
|
||||
Lsn::INVALID,
|
||||
Lsn::INVALID,
|
||||
)
|
||||
.await?;
|
||||
let tli = spg
|
||||
.global_timelines
|
||||
.create(
|
||||
spg.ttid,
|
||||
ServerInfo {
|
||||
pg_version,
|
||||
wal_seg_size: WAL_SEGMENT_SIZE as u32,
|
||||
system_id: 0,
|
||||
},
|
||||
Lsn::INVALID,
|
||||
Lsn::INVALID,
|
||||
)
|
||||
.await?;
|
||||
|
||||
tli.wal_residence_guard().await
|
||||
}
|
||||
|
||||
@@ -455,6 +455,7 @@ pub struct FullTimelineInfo {
|
||||
|
||||
/// Collects metrics for all active timelines.
|
||||
pub struct TimelineCollector {
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
descs: Vec<Desc>,
|
||||
commit_lsn: GenericGaugeVec<AtomicU64>,
|
||||
backup_lsn: GenericGaugeVec<AtomicU64>,
|
||||
@@ -478,14 +479,8 @@ pub struct TimelineCollector {
|
||||
active_timelines_count: IntGauge,
|
||||
}
|
||||
|
||||
impl Default for TimelineCollector {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl TimelineCollector {
|
||||
pub fn new() -> TimelineCollector {
|
||||
pub fn new(global_timelines: Arc<GlobalTimelines>) -> TimelineCollector {
|
||||
let mut descs = Vec::new();
|
||||
|
||||
let commit_lsn = GenericGaugeVec::new(
|
||||
@@ -676,6 +671,7 @@ impl TimelineCollector {
|
||||
descs.extend(active_timelines_count.desc().into_iter().cloned());
|
||||
|
||||
TimelineCollector {
|
||||
global_timelines,
|
||||
descs,
|
||||
commit_lsn,
|
||||
backup_lsn,
|
||||
@@ -728,17 +724,18 @@ impl Collector for TimelineCollector {
|
||||
self.written_wal_seconds.reset();
|
||||
self.flushed_wal_seconds.reset();
|
||||
|
||||
let timelines_count = GlobalTimelines::get_all().len();
|
||||
let timelines_count = self.global_timelines.get_all().len();
|
||||
let mut active_timelines_count = 0;
|
||||
|
||||
// Prometheus Collector is sync, and data is stored under async lock. To
|
||||
// bridge the gap with a crutch, collect data in spawned thread with
|
||||
// local tokio runtime.
|
||||
let global_timelines = self.global_timelines.clone();
|
||||
let infos = std::thread::spawn(|| {
|
||||
let rt = tokio::runtime::Builder::new_current_thread()
|
||||
.build()
|
||||
.expect("failed to create rt");
|
||||
rt.block_on(collect_timeline_metrics())
|
||||
rt.block_on(collect_timeline_metrics(global_timelines))
|
||||
})
|
||||
.join()
|
||||
.expect("collect_timeline_metrics thread panicked");
|
||||
@@ -857,9 +854,9 @@ impl Collector for TimelineCollector {
|
||||
}
|
||||
}
|
||||
|
||||
async fn collect_timeline_metrics() -> Vec<FullTimelineInfo> {
|
||||
async fn collect_timeline_metrics(global_timelines: Arc<GlobalTimelines>) -> Vec<FullTimelineInfo> {
|
||||
let mut res = vec![];
|
||||
let active_timelines = GlobalTimelines::get_global_broker_active_set().get_all();
|
||||
let active_timelines = global_timelines.get_global_broker_active_set().get_all();
|
||||
|
||||
for tli in active_timelines {
|
||||
if let Some(info) = tli.info_for_metrics().await {
|
||||
|
||||
@@ -409,8 +409,9 @@ pub struct DebugDumpResponse {
|
||||
pub async fn handle_request(
|
||||
request: Request,
|
||||
sk_auth_token: Option<SecretString>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> Result<Response> {
|
||||
let existing_tli = GlobalTimelines::get(TenantTimelineId::new(
|
||||
let existing_tli = global_timelines.get(TenantTimelineId::new(
|
||||
request.tenant_id,
|
||||
request.timeline_id,
|
||||
));
|
||||
@@ -453,13 +454,14 @@ pub async fn handle_request(
|
||||
assert!(status.tenant_id == request.tenant_id);
|
||||
assert!(status.timeline_id == request.timeline_id);
|
||||
|
||||
pull_timeline(status, safekeeper_host, sk_auth_token).await
|
||||
pull_timeline(status, safekeeper_host, sk_auth_token, global_timelines).await
|
||||
}
|
||||
|
||||
async fn pull_timeline(
|
||||
status: TimelineStatus,
|
||||
host: String,
|
||||
sk_auth_token: Option<SecretString>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> Result<Response> {
|
||||
let ttid = TenantTimelineId::new(status.tenant_id, status.timeline_id);
|
||||
info!(
|
||||
@@ -472,7 +474,7 @@ async fn pull_timeline(
|
||||
status.acceptor_state.epoch
|
||||
);
|
||||
|
||||
let conf = &GlobalTimelines::get_global_config();
|
||||
let conf = &global_timelines.get_global_config();
|
||||
|
||||
let (_tmp_dir, tli_dir_path) = create_temp_timeline_dir(conf, ttid).await?;
|
||||
|
||||
@@ -531,7 +533,9 @@ async fn pull_timeline(
|
||||
assert!(status.commit_lsn <= status.flush_lsn);
|
||||
|
||||
// Finally, load the timeline.
|
||||
let _tli = GlobalTimelines::load_temp_timeline(ttid, &tli_dir_path, false).await?;
|
||||
let _tli = global_timelines
|
||||
.load_temp_timeline(ttid, &tli_dir_path, false)
|
||||
.await?;
|
||||
|
||||
Ok(Response {
|
||||
safekeeper_host: host,
|
||||
|
||||
@@ -267,6 +267,7 @@ impl SafekeeperPostgresHandler {
|
||||
pgb_reader: &mut pgb_reader,
|
||||
peer_addr,
|
||||
acceptor_handle: &mut acceptor_handle,
|
||||
global_timelines: self.global_timelines.clone(),
|
||||
};
|
||||
|
||||
// Read first message and create timeline if needed.
|
||||
@@ -331,6 +332,7 @@ struct NetworkReader<'a, IO> {
|
||||
// WalAcceptor is spawned when we learn server info from walproposer and
|
||||
// create timeline; handle is put here.
|
||||
acceptor_handle: &'a mut Option<JoinHandle<anyhow::Result<()>>>,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
}
|
||||
|
||||
impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
@@ -350,10 +352,11 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
system_id: greeting.system_id,
|
||||
wal_seg_size: greeting.wal_seg_size,
|
||||
};
|
||||
let tli =
|
||||
GlobalTimelines::create(self.ttid, server_info, Lsn::INVALID, Lsn::INVALID)
|
||||
.await
|
||||
.context("create timeline")?;
|
||||
let tli = self
|
||||
.global_timelines
|
||||
.create(self.ttid, server_info, Lsn::INVALID, Lsn::INVALID)
|
||||
.await
|
||||
.context("create timeline")?;
|
||||
tli.wal_residence_guard().await?
|
||||
}
|
||||
_ => {
|
||||
|
||||
@@ -10,7 +10,6 @@ use crate::timeline::WalResidentTimeline;
|
||||
use crate::wal_reader_stream::WalReaderStreamBuilder;
|
||||
use crate::wal_service::ConnectionId;
|
||||
use crate::wal_storage::WalReader;
|
||||
use crate::GlobalTimelines;
|
||||
use anyhow::{bail, Context as AnyhowContext};
|
||||
use bytes::Bytes;
|
||||
use futures::future::Either;
|
||||
@@ -400,7 +399,10 @@ impl SafekeeperPostgresHandler {
|
||||
start_pos: Lsn,
|
||||
term: Option<Term>,
|
||||
) -> Result<(), QueryError> {
|
||||
let tli = GlobalTimelines::get(self.ttid).map_err(|e| QueryError::Other(e.into()))?;
|
||||
let tli = self
|
||||
.global_timelines
|
||||
.get(self.ttid)
|
||||
.map_err(|e| QueryError::Other(e.into()))?;
|
||||
let residence_guard = tli.wal_residence_guard().await?;
|
||||
|
||||
if let Err(end) = self
|
||||
|
||||
@@ -44,8 +44,8 @@ use crate::wal_backup_partial::PartialRemoteSegment;
|
||||
|
||||
use crate::metrics::{FullTimelineInfo, WalStorageMetrics, MISC_OPERATION_SECONDS};
|
||||
use crate::wal_storage::{Storage as wal_storage_iface, WalReader};
|
||||
use crate::SafeKeeperConf;
|
||||
use crate::{debug_dump, timeline_manager, wal_storage};
|
||||
use crate::{GlobalTimelines, SafeKeeperConf};
|
||||
|
||||
/// Things safekeeper should know about timeline state on peers.
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
@@ -467,6 +467,7 @@ pub struct Timeline {
|
||||
walreceivers: Arc<WalReceivers>,
|
||||
timeline_dir: Utf8PathBuf,
|
||||
manager_ctl: ManagerCtl,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
|
||||
/// Hold this gate from code that depends on the Timeline's non-shut-down state. While holding
|
||||
/// this gate, you must respect [`Timeline::cancel`]
|
||||
@@ -489,6 +490,7 @@ impl Timeline {
|
||||
timeline_dir: &Utf8Path,
|
||||
remote_path: &RemotePath,
|
||||
shared_state: SharedState,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
) -> Arc<Self> {
|
||||
let (commit_lsn_watch_tx, commit_lsn_watch_rx) =
|
||||
watch::channel(shared_state.sk.state().commit_lsn);
|
||||
@@ -516,6 +518,7 @@ impl Timeline {
|
||||
gate: Default::default(),
|
||||
cancel: CancellationToken::default(),
|
||||
manager_ctl: ManagerCtl::new(),
|
||||
conf,
|
||||
broker_active: AtomicBool::new(false),
|
||||
wal_backup_active: AtomicBool::new(false),
|
||||
last_removed_segno: AtomicU64::new(0),
|
||||
@@ -524,11 +527,14 @@ impl Timeline {
|
||||
}
|
||||
|
||||
/// Load existing timeline from disk.
|
||||
pub fn load_timeline(conf: &SafeKeeperConf, ttid: TenantTimelineId) -> Result<Arc<Timeline>> {
|
||||
pub fn load_timeline(
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
ttid: TenantTimelineId,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let _enter = info_span!("load_timeline", timeline = %ttid.timeline_id).entered();
|
||||
|
||||
let shared_state = SharedState::restore(conf, &ttid)?;
|
||||
let timeline_dir = get_timeline_dir(conf, &ttid);
|
||||
let shared_state = SharedState::restore(conf.as_ref(), &ttid)?;
|
||||
let timeline_dir = get_timeline_dir(conf.as_ref(), &ttid);
|
||||
let remote_path = remote_timeline_path(&ttid)?;
|
||||
|
||||
Ok(Timeline::new(
|
||||
@@ -536,6 +542,7 @@ impl Timeline {
|
||||
&timeline_dir,
|
||||
&remote_path,
|
||||
shared_state,
|
||||
conf,
|
||||
))
|
||||
}
|
||||
|
||||
@@ -604,8 +611,7 @@ impl Timeline {
|
||||
// it is cancelled, so WAL storage won't be opened again.
|
||||
shared_state.sk.close_wal_store();
|
||||
|
||||
let conf = GlobalTimelines::get_global_config();
|
||||
if !only_local && conf.is_wal_backup_enabled() {
|
||||
if !only_local && self.conf.is_wal_backup_enabled() {
|
||||
// Note: we concurrently delete remote storage data from multiple
|
||||
// safekeepers. That's ok, s3 replies 200 if object doesn't exist and we
|
||||
// do some retries anyway.
|
||||
@@ -951,7 +957,7 @@ impl WalResidentTimeline {
|
||||
|
||||
pub async fn get_walreader(&self, start_lsn: Lsn) -> Result<WalReader> {
|
||||
let (_, persisted_state) = self.get_state().await;
|
||||
let enable_remote_read = GlobalTimelines::get_global_config().is_wal_backup_enabled();
|
||||
let enable_remote_read = self.conf.is_wal_backup_enabled();
|
||||
|
||||
WalReader::new(
|
||||
&self.ttid,
|
||||
@@ -1061,7 +1067,6 @@ impl ManagerTimeline {
|
||||
|
||||
/// Try to switch state Offloaded->Present.
|
||||
pub(crate) async fn switch_to_present(&self) -> anyhow::Result<()> {
|
||||
let conf = GlobalTimelines::get_global_config();
|
||||
let mut shared = self.write_shared_state().await;
|
||||
|
||||
// trying to restore WAL storage
|
||||
@@ -1069,7 +1074,7 @@ impl ManagerTimeline {
|
||||
&self.ttid,
|
||||
&self.timeline_dir,
|
||||
shared.sk.state(),
|
||||
conf.no_sync,
|
||||
self.conf.no_sync,
|
||||
)?;
|
||||
|
||||
// updating control file
|
||||
@@ -1096,7 +1101,7 @@ impl ManagerTimeline {
|
||||
// now we can switch shared.sk to Present, shouldn't fail
|
||||
let prev_sk = std::mem::replace(&mut shared.sk, StateSK::Empty);
|
||||
let cfile_state = prev_sk.take_state();
|
||||
shared.sk = StateSK::Loaded(SafeKeeper::new(cfile_state, wal_store, conf.my_id)?);
|
||||
shared.sk = StateSK::Loaded(SafeKeeper::new(cfile_state, wal_store, self.conf.my_id)?);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -13,7 +13,6 @@ use crate::{control_file, wal_storage, SafeKeeperConf};
|
||||
use anyhow::{bail, Context, Result};
|
||||
use camino::Utf8PathBuf;
|
||||
use camino_tempfile::Utf8TempDir;
|
||||
use once_cell::sync::Lazy;
|
||||
use serde::Serialize;
|
||||
use std::collections::HashMap;
|
||||
use std::str::FromStr;
|
||||
@@ -42,23 +41,16 @@ struct GlobalTimelinesState {
|
||||
// this map is dropped on restart.
|
||||
tombstones: HashMap<TenantTimelineId, Instant>,
|
||||
|
||||
conf: Option<SafeKeeperConf>,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
broker_active_set: Arc<TimelinesSet>,
|
||||
global_rate_limiter: RateLimiter,
|
||||
}
|
||||
|
||||
impl GlobalTimelinesState {
|
||||
/// Get configuration, which must be set once during init.
|
||||
fn get_conf(&self) -> &SafeKeeperConf {
|
||||
self.conf
|
||||
.as_ref()
|
||||
.expect("GlobalTimelinesState conf is not initialized")
|
||||
}
|
||||
|
||||
/// Get dependencies for a timeline constructor.
|
||||
fn get_dependencies(&self) -> (SafeKeeperConf, Arc<TimelinesSet>, RateLimiter) {
|
||||
fn get_dependencies(&self) -> (Arc<SafeKeeperConf>, Arc<TimelinesSet>, RateLimiter) {
|
||||
(
|
||||
self.get_conf().clone(),
|
||||
self.conf.clone(),
|
||||
self.broker_active_set.clone(),
|
||||
self.global_rate_limiter.clone(),
|
||||
)
|
||||
@@ -82,35 +74,39 @@ impl GlobalTimelinesState {
|
||||
}
|
||||
}
|
||||
|
||||
static TIMELINES_STATE: Lazy<Mutex<GlobalTimelinesState>> = Lazy::new(|| {
|
||||
Mutex::new(GlobalTimelinesState {
|
||||
timelines: HashMap::new(),
|
||||
tombstones: HashMap::new(),
|
||||
conf: None,
|
||||
broker_active_set: Arc::new(TimelinesSet::default()),
|
||||
global_rate_limiter: RateLimiter::new(1, 1),
|
||||
})
|
||||
});
|
||||
|
||||
/// A zero-sized struct used to manage access to the global timelines map.
|
||||
pub struct GlobalTimelines;
|
||||
/// A struct used to manage access to the global timelines map.
|
||||
pub struct GlobalTimelines {
|
||||
state: Mutex<GlobalTimelinesState>,
|
||||
}
|
||||
|
||||
impl GlobalTimelines {
|
||||
/// Create a new instance of the global timelines map.
|
||||
pub fn new(conf: Arc<SafeKeeperConf>) -> Self {
|
||||
Self {
|
||||
state: Mutex::new(GlobalTimelinesState {
|
||||
timelines: HashMap::new(),
|
||||
tombstones: HashMap::new(),
|
||||
conf,
|
||||
broker_active_set: Arc::new(TimelinesSet::default()),
|
||||
global_rate_limiter: RateLimiter::new(1, 1),
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
/// Inject dependencies needed for the timeline constructors and load all timelines to memory.
|
||||
pub async fn init(conf: SafeKeeperConf) -> Result<()> {
|
||||
pub async fn init(&self) -> Result<()> {
|
||||
// clippy isn't smart enough to understand that drop(state) releases the
|
||||
// lock, so use explicit block
|
||||
let tenants_dir = {
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
let mut state = self.state.lock().unwrap();
|
||||
state.global_rate_limiter = RateLimiter::new(
|
||||
conf.partial_backup_concurrency,
|
||||
state.conf.partial_backup_concurrency,
|
||||
DEFAULT_EVICTION_CONCURRENCY,
|
||||
);
|
||||
state.conf = Some(conf);
|
||||
|
||||
// Iterate through all directories and load tenants for all directories
|
||||
// named as a valid tenant_id.
|
||||
state.get_conf().workdir.clone()
|
||||
state.conf.workdir.clone()
|
||||
};
|
||||
let mut tenant_count = 0;
|
||||
for tenants_dir_entry in std::fs::read_dir(&tenants_dir)
|
||||
@@ -122,7 +118,7 @@ impl GlobalTimelines {
|
||||
TenantId::from_str(tenants_dir_entry.file_name().to_str().unwrap_or(""))
|
||||
{
|
||||
tenant_count += 1;
|
||||
GlobalTimelines::load_tenant_timelines(tenant_id).await?;
|
||||
self.load_tenant_timelines(tenant_id).await?;
|
||||
}
|
||||
}
|
||||
Err(e) => error!(
|
||||
@@ -135,7 +131,7 @@ impl GlobalTimelines {
|
||||
info!(
|
||||
"found {} tenants directories, successfully loaded {} timelines",
|
||||
tenant_count,
|
||||
TIMELINES_STATE.lock().unwrap().timelines.len()
|
||||
self.state.lock().unwrap().timelines.len()
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
@@ -143,13 +139,13 @@ impl GlobalTimelines {
|
||||
/// Loads all timelines for the given tenant to memory. Returns fs::read_dir
|
||||
/// errors if any.
|
||||
///
|
||||
/// It is async, but TIMELINES_STATE lock is sync and there is no important
|
||||
/// It is async, but self.state lock is sync and there is no important
|
||||
/// reason to make it async (it is always held for a short while), so we
|
||||
/// just lock and unlock it for each timeline -- this function is called
|
||||
/// during init when nothing else is running, so this is fine.
|
||||
async fn load_tenant_timelines(tenant_id: TenantId) -> Result<()> {
|
||||
async fn load_tenant_timelines(&self, tenant_id: TenantId) -> Result<()> {
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
let state = self.state.lock().unwrap();
|
||||
state.get_dependencies()
|
||||
};
|
||||
|
||||
@@ -163,10 +159,10 @@ impl GlobalTimelines {
|
||||
TimelineId::from_str(timeline_dir_entry.file_name().to_str().unwrap_or(""))
|
||||
{
|
||||
let ttid = TenantTimelineId::new(tenant_id, timeline_id);
|
||||
match Timeline::load_timeline(&conf, ttid) {
|
||||
match Timeline::load_timeline(conf.clone(), ttid) {
|
||||
Ok(tli) => {
|
||||
let mut shared_state = tli.write_shared_state().await;
|
||||
TIMELINES_STATE
|
||||
self.state
|
||||
.lock()
|
||||
.unwrap()
|
||||
.timelines
|
||||
@@ -200,29 +196,30 @@ impl GlobalTimelines {
|
||||
}
|
||||
|
||||
/// Get the number of timelines in the map.
|
||||
pub fn timelines_count() -> usize {
|
||||
TIMELINES_STATE.lock().unwrap().timelines.len()
|
||||
pub fn timelines_count(&self) -> usize {
|
||||
self.state.lock().unwrap().timelines.len()
|
||||
}
|
||||
|
||||
/// Get the global safekeeper config.
|
||||
pub fn get_global_config() -> SafeKeeperConf {
|
||||
TIMELINES_STATE.lock().unwrap().get_conf().clone()
|
||||
pub fn get_global_config(&self) -> Arc<SafeKeeperConf> {
|
||||
self.state.lock().unwrap().conf.clone()
|
||||
}
|
||||
|
||||
pub fn get_global_broker_active_set() -> Arc<TimelinesSet> {
|
||||
TIMELINES_STATE.lock().unwrap().broker_active_set.clone()
|
||||
pub fn get_global_broker_active_set(&self) -> Arc<TimelinesSet> {
|
||||
self.state.lock().unwrap().broker_active_set.clone()
|
||||
}
|
||||
|
||||
/// Create a new timeline with the given id. If the timeline already exists, returns
|
||||
/// an existing timeline.
|
||||
pub(crate) async fn create(
|
||||
&self,
|
||||
ttid: TenantTimelineId,
|
||||
server_info: ServerInfo,
|
||||
commit_lsn: Lsn,
|
||||
local_start_lsn: Lsn,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let (conf, _, _) = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
let state = self.state.lock().unwrap();
|
||||
if let Ok(timeline) = state.get(&ttid) {
|
||||
// Timeline already exists, return it.
|
||||
return Ok(timeline);
|
||||
@@ -245,7 +242,7 @@ impl GlobalTimelines {
|
||||
let state =
|
||||
TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn)?;
|
||||
control_file::FileStorage::create_new(&tmp_dir_path, state, conf.no_sync).await?;
|
||||
let timeline = GlobalTimelines::load_temp_timeline(ttid, &tmp_dir_path, true).await?;
|
||||
let timeline = self.load_temp_timeline(ttid, &tmp_dir_path, true).await?;
|
||||
Ok(timeline)
|
||||
}
|
||||
|
||||
@@ -261,13 +258,14 @@ impl GlobalTimelines {
|
||||
/// 2) move the directory and load the timeline
|
||||
/// 3) take lock again and insert the timeline into the global map.
|
||||
pub async fn load_temp_timeline(
|
||||
&self,
|
||||
ttid: TenantTimelineId,
|
||||
tmp_path: &Utf8PathBuf,
|
||||
check_tombstone: bool,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
// Check for existence and mark that we're creating it.
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) = {
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
let mut state = self.state.lock().unwrap();
|
||||
match state.timelines.get(&ttid) {
|
||||
Some(GlobalMapTimeline::CreationInProgress) => {
|
||||
bail!(TimelineError::CreationInProgress(ttid));
|
||||
@@ -295,10 +293,10 @@ impl GlobalTimelines {
|
||||
};
|
||||
|
||||
// Do the actual move and reflect the result in the map.
|
||||
match GlobalTimelines::install_temp_timeline(ttid, tmp_path, &conf).await {
|
||||
match GlobalTimelines::install_temp_timeline(ttid, tmp_path, conf.clone()).await {
|
||||
Ok(timeline) => {
|
||||
let mut timeline_shared_state = timeline.write_shared_state().await;
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
let mut state = self.state.lock().unwrap();
|
||||
assert!(matches!(
|
||||
state.timelines.get(&ttid),
|
||||
Some(GlobalMapTimeline::CreationInProgress)
|
||||
@@ -319,7 +317,7 @@ impl GlobalTimelines {
|
||||
}
|
||||
Err(e) => {
|
||||
// Init failed, remove the marker from the map
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
let mut state = self.state.lock().unwrap();
|
||||
assert!(matches!(
|
||||
state.timelines.get(&ttid),
|
||||
Some(GlobalMapTimeline::CreationInProgress)
|
||||
@@ -334,10 +332,10 @@ impl GlobalTimelines {
|
||||
async fn install_temp_timeline(
|
||||
ttid: TenantTimelineId,
|
||||
tmp_path: &Utf8PathBuf,
|
||||
conf: &SafeKeeperConf,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let tenant_path = get_tenant_dir(conf, &ttid.tenant_id);
|
||||
let timeline_path = get_timeline_dir(conf, &ttid);
|
||||
let tenant_path = get_tenant_dir(conf.as_ref(), &ttid.tenant_id);
|
||||
let timeline_path = get_timeline_dir(conf.as_ref(), &ttid);
|
||||
|
||||
// We must have already checked that timeline doesn't exist in the map,
|
||||
// but there might be existing datadir: if timeline is corrupted it is
|
||||
@@ -382,9 +380,9 @@ impl GlobalTimelines {
|
||||
/// Get a timeline from the global map. If it's not present, it doesn't exist on disk,
|
||||
/// or was corrupted and couldn't be loaded on startup. Returned timeline is always valid,
|
||||
/// i.e. loaded in memory and not cancelled.
|
||||
pub(crate) fn get(ttid: TenantTimelineId) -> Result<Arc<Timeline>, TimelineError> {
|
||||
pub(crate) fn get(&self, ttid: TenantTimelineId) -> Result<Arc<Timeline>, TimelineError> {
|
||||
let tli_res = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
let state = self.state.lock().unwrap();
|
||||
state.get(&ttid)
|
||||
};
|
||||
match tli_res {
|
||||
@@ -399,8 +397,8 @@ impl GlobalTimelines {
|
||||
}
|
||||
|
||||
/// Returns all timelines. This is used for background timeline processes.
|
||||
pub fn get_all() -> Vec<Arc<Timeline>> {
|
||||
let global_lock = TIMELINES_STATE.lock().unwrap();
|
||||
pub fn get_all(&self) -> Vec<Arc<Timeline>> {
|
||||
let global_lock = self.state.lock().unwrap();
|
||||
global_lock
|
||||
.timelines
|
||||
.values()
|
||||
@@ -419,8 +417,8 @@ impl GlobalTimelines {
|
||||
|
||||
/// Returns all timelines belonging to a given tenant. Used for deleting all timelines of a tenant,
|
||||
/// and that's why it can return cancelled timelines, to retry deleting them.
|
||||
fn get_all_for_tenant(tenant_id: TenantId) -> Vec<Arc<Timeline>> {
|
||||
let global_lock = TIMELINES_STATE.lock().unwrap();
|
||||
fn get_all_for_tenant(&self, tenant_id: TenantId) -> Vec<Arc<Timeline>> {
|
||||
let global_lock = self.state.lock().unwrap();
|
||||
global_lock
|
||||
.timelines
|
||||
.values()
|
||||
@@ -435,11 +433,12 @@ impl GlobalTimelines {
|
||||
/// Cancels timeline, then deletes the corresponding data directory.
|
||||
/// If only_local, doesn't remove WAL segments in remote storage.
|
||||
pub(crate) async fn delete(
|
||||
&self,
|
||||
ttid: &TenantTimelineId,
|
||||
only_local: bool,
|
||||
) -> Result<TimelineDeleteForceResult> {
|
||||
let tli_res = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
let state = self.state.lock().unwrap();
|
||||
|
||||
if state.tombstones.contains_key(ttid) {
|
||||
// Presence of a tombstone guarantees that a previous deletion has completed and there is no work to do.
|
||||
@@ -472,7 +471,7 @@ impl GlobalTimelines {
|
||||
}
|
||||
Err(_) => {
|
||||
// Timeline is not memory, but it may still exist on disk in broken state.
|
||||
let dir_path = get_timeline_dir(TIMELINES_STATE.lock().unwrap().get_conf(), ttid);
|
||||
let dir_path = get_timeline_dir(self.state.lock().unwrap().conf.as_ref(), ttid);
|
||||
let dir_existed = delete_dir(dir_path)?;
|
||||
|
||||
Ok(TimelineDeleteForceResult {
|
||||
@@ -485,7 +484,7 @@ impl GlobalTimelines {
|
||||
// Finalize deletion, by dropping Timeline objects and storing smaller tombstones. The tombstones
|
||||
// are used to prevent still-running computes from re-creating the same timeline when they send data,
|
||||
// and to speed up repeated deletion calls by avoiding re-listing objects.
|
||||
TIMELINES_STATE.lock().unwrap().delete(*ttid);
|
||||
self.state.lock().unwrap().delete(*ttid);
|
||||
|
||||
result
|
||||
}
|
||||
@@ -497,17 +496,18 @@ impl GlobalTimelines {
|
||||
///
|
||||
/// If only_local, doesn't remove WAL segments in remote storage.
|
||||
pub async fn delete_force_all_for_tenant(
|
||||
&self,
|
||||
tenant_id: &TenantId,
|
||||
only_local: bool,
|
||||
) -> Result<HashMap<TenantTimelineId, TimelineDeleteForceResult>> {
|
||||
info!("deleting all timelines for tenant {}", tenant_id);
|
||||
let to_delete = Self::get_all_for_tenant(*tenant_id);
|
||||
let to_delete = self.get_all_for_tenant(*tenant_id);
|
||||
|
||||
let mut err = None;
|
||||
|
||||
let mut deleted = HashMap::new();
|
||||
for tli in &to_delete {
|
||||
match Self::delete(&tli.ttid, only_local).await {
|
||||
match self.delete(&tli.ttid, only_local).await {
|
||||
Ok(result) => {
|
||||
deleted.insert(tli.ttid, result);
|
||||
}
|
||||
@@ -529,15 +529,15 @@ impl GlobalTimelines {
|
||||
// so the directory may be not empty. In this case timelines will have bad state
|
||||
// and timeline background jobs can panic.
|
||||
delete_dir(get_tenant_dir(
|
||||
TIMELINES_STATE.lock().unwrap().get_conf(),
|
||||
self.state.lock().unwrap().conf.as_ref(),
|
||||
tenant_id,
|
||||
))?;
|
||||
|
||||
Ok(deleted)
|
||||
}
|
||||
|
||||
pub fn housekeeping(tombstone_ttl: &Duration) {
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
pub fn housekeeping(&self, tombstone_ttl: &Duration) {
|
||||
let mut state = self.state.lock().unwrap();
|
||||
|
||||
// We keep tombstones long enough to have a good chance of preventing rogue computes from re-creating deleted
|
||||
// timelines. If a compute kept running for longer than this TTL (or across a safekeeper restart) then they
|
||||
|
||||
@@ -4,6 +4,7 @@
|
||||
//!
|
||||
use anyhow::{Context, Result};
|
||||
use postgres_backend::QueryError;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use tokio::net::TcpStream;
|
||||
use tokio_io_timeout::TimeoutReader;
|
||||
@@ -11,9 +12,9 @@ use tokio_util::sync::CancellationToken;
|
||||
use tracing::*;
|
||||
use utils::{auth::Scope, measured_stream::MeasuredStream};
|
||||
|
||||
use crate::handler::SafekeeperPostgresHandler;
|
||||
use crate::metrics::TrafficMetrics;
|
||||
use crate::SafeKeeperConf;
|
||||
use crate::{handler::SafekeeperPostgresHandler, GlobalTimelines};
|
||||
use postgres_backend::{AuthType, PostgresBackend};
|
||||
|
||||
/// Accept incoming TCP connections and spawn them into a background thread.
|
||||
@@ -22,9 +23,10 @@ use postgres_backend::{AuthType, PostgresBackend};
|
||||
/// to any tenant are allowed) or Tenant (only tokens giving access to specific
|
||||
/// tenant are allowed). Doesn't matter if auth is disabled in conf.
|
||||
pub async fn task_main(
|
||||
conf: SafeKeeperConf,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
pg_listener: std::net::TcpListener,
|
||||
allowed_auth_scope: Scope,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> anyhow::Result<()> {
|
||||
// Tokio's from_std won't do this for us, per its comment.
|
||||
pg_listener.set_nonblocking(true)?;
|
||||
@@ -37,10 +39,10 @@ pub async fn task_main(
|
||||
debug!("accepted connection from {}", peer_addr);
|
||||
let conf = conf.clone();
|
||||
let conn_id = issue_connection_id(&mut connection_count);
|
||||
|
||||
let global_timelines = global_timelines.clone();
|
||||
tokio::spawn(
|
||||
async move {
|
||||
if let Err(err) = handle_socket(socket, conf, conn_id, allowed_auth_scope).await {
|
||||
if let Err(err) = handle_socket(socket, conf, conn_id, allowed_auth_scope, global_timelines).await {
|
||||
error!("connection handler exited: {}", err);
|
||||
}
|
||||
}
|
||||
@@ -53,9 +55,10 @@ pub async fn task_main(
|
||||
///
|
||||
async fn handle_socket(
|
||||
socket: TcpStream,
|
||||
conf: SafeKeeperConf,
|
||||
conf: Arc<SafeKeeperConf>,
|
||||
conn_id: ConnectionId,
|
||||
allowed_auth_scope: Scope,
|
||||
global_timelines: Arc<GlobalTimelines>,
|
||||
) -> Result<(), QueryError> {
|
||||
socket.set_nodelay(true)?;
|
||||
let peer_addr = socket.peer_addr()?;
|
||||
@@ -96,8 +99,13 @@ async fn handle_socket(
|
||||
Some(_) => AuthType::NeonJWT,
|
||||
};
|
||||
let auth_pair = auth_key.map(|key| (allowed_auth_scope, key));
|
||||
let mut conn_handler =
|
||||
SafekeeperPostgresHandler::new(conf, conn_id, Some(traffic_metrics.clone()), auth_pair);
|
||||
let mut conn_handler = SafekeeperPostgresHandler::new(
|
||||
conf,
|
||||
conn_id,
|
||||
Some(traffic_metrics.clone()),
|
||||
auth_pair,
|
||||
global_timelines,
|
||||
);
|
||||
let pgbackend = PostgresBackend::new_from_io(socket, peer_addr, auth_type, None)?;
|
||||
// libpq protocol between safekeeper and walproposer / pageserver
|
||||
// We don't use shutdown.
|
||||
|
||||
21
test_runner/cloud_regress/README.md
Normal file
21
test_runner/cloud_regress/README.md
Normal file
@@ -0,0 +1,21 @@
|
||||
# How to run the `pg_regress` tests on a cloud Neon instance.
|
||||
|
||||
* Create a Neon project on staging.
|
||||
* Grant the superuser privileges to the DB user.
|
||||
* (Optional) create a branch for testing
|
||||
* Configure the endpoint by updating the control-plane database with the following settings:
|
||||
* `Timeone`: `America/Los_Angeles`
|
||||
* `DateStyle`: `Postgres,MDY`
|
||||
* `compute_query_id`: `off`
|
||||
* Checkout the actual `Neon` sources
|
||||
* Patch the sql and expected files for the specific PostgreSQL version, e.g. for v17:
|
||||
```bash
|
||||
$ cd vendor/postgres-v17
|
||||
$ patch -p1 <../../compute/patches/cloud_regress_pg17.patch
|
||||
```
|
||||
* Set the environment variable `BENCHMARK_CONNSTR` to the connection URI of your project.
|
||||
* Set the environment variable `PG_VERSION` to the version of your project.
|
||||
* Run
|
||||
```bash
|
||||
$ pytest -m remote_cluster -k cloud_regress
|
||||
```
|
||||
@@ -5,68 +5,15 @@ Run the regression tests on the cloud instance of Neon
|
||||
from __future__ import annotations
|
||||
|
||||
from pathlib import Path
|
||||
from typing import Any
|
||||
|
||||
import psycopg2
|
||||
import pytest
|
||||
from fixtures.log_helper import log
|
||||
from fixtures.neon_fixtures import RemotePostgres
|
||||
from fixtures.pg_version import PgVersion
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def setup(remote_pg: RemotePostgres):
|
||||
"""
|
||||
Setup and teardown of the tests
|
||||
"""
|
||||
with psycopg2.connect(remote_pg.connstr()) as conn:
|
||||
with conn.cursor() as cur:
|
||||
log.info("Creating the extension")
|
||||
cur.execute("CREATE EXTENSION IF NOT EXISTS regress_so")
|
||||
conn.commit()
|
||||
# TODO: Migrate to branches and remove this code
|
||||
log.info("Looking for subscriptions in the regress database")
|
||||
cur.execute(
|
||||
"SELECT subname FROM pg_catalog.pg_subscription WHERE "
|
||||
"subdbid = (SELECT oid FROM pg_catalog.pg_database WHERE datname='regression');"
|
||||
)
|
||||
if cur.rowcount > 0:
|
||||
with psycopg2.connect(
|
||||
dbname="regression",
|
||||
host=remote_pg.default_options["host"],
|
||||
user=remote_pg.default_options["user"],
|
||||
password=remote_pg.default_options["password"],
|
||||
) as regress_conn:
|
||||
with regress_conn.cursor() as regress_cur:
|
||||
for sub in cur:
|
||||
regress_cur.execute(f"ALTER SUBSCRIPTION {sub[0]} DISABLE")
|
||||
regress_cur.execute(
|
||||
f"ALTER SUBSCRIPTION {sub[0]} SET (slot_name = NONE)"
|
||||
)
|
||||
regress_cur.execute(f"DROP SUBSCRIPTION {sub[0]}")
|
||||
regress_conn.commit()
|
||||
|
||||
yield
|
||||
# TODO: Migrate to branches and remove this code
|
||||
log.info("Looking for extra roles...")
|
||||
with psycopg2.connect(remote_pg.connstr()) as conn:
|
||||
with conn.cursor() as cur:
|
||||
cur.execute(
|
||||
"SELECT rolname FROM pg_catalog.pg_roles WHERE oid > 16384 AND rolname <> 'neondb_owner'"
|
||||
)
|
||||
roles: list[Any] = []
|
||||
for role in cur:
|
||||
log.info("Role found: %s", role[0])
|
||||
roles.append(role[0])
|
||||
for role in roles:
|
||||
cur.execute(f"DROP ROLE {role}")
|
||||
conn.commit()
|
||||
|
||||
|
||||
@pytest.mark.timeout(7200)
|
||||
@pytest.mark.remote_cluster
|
||||
def test_cloud_regress(
|
||||
setup,
|
||||
remote_pg: RemotePostgres,
|
||||
pg_version: PgVersion,
|
||||
pg_distrib_dir: Path,
|
||||
|
||||
@@ -121,9 +121,6 @@ page_cache_size=10
|
||||
assert vectored_average < 8
|
||||
|
||||
|
||||
@pytest.mark.skip(
|
||||
"This is being fixed and tracked in https://github.com/neondatabase/neon/issues/9114"
|
||||
)
|
||||
@skip_in_debug_build("only run with release build")
|
||||
def test_pageserver_gc_compaction_smoke(neon_env_builder: NeonEnvBuilder):
|
||||
SMOKE_CONF = {
|
||||
@@ -156,20 +153,20 @@ def test_pageserver_gc_compaction_smoke(neon_env_builder: NeonEnvBuilder):
|
||||
if i % 10 == 0:
|
||||
log.info(f"Running churn round {i}/{churn_rounds} ...")
|
||||
|
||||
ps_http.timeline_compact(
|
||||
tenant_id,
|
||||
timeline_id,
|
||||
enhanced_gc_bottom_most_compaction=True,
|
||||
body={
|
||||
"scheduled": True,
|
||||
"sub_compaction": True,
|
||||
"compact_range": {
|
||||
"start": "000000000000000000000000000000000000",
|
||||
# skip the SLRU range for now -- it races with get-lsn-by-timestamp, TODO: fix this
|
||||
"end": "010000000000000000000000000000000000",
|
||||
# Run gc-compaction every 10 rounds to ensure the test doesn't take too long time.
|
||||
ps_http.timeline_compact(
|
||||
tenant_id,
|
||||
timeline_id,
|
||||
enhanced_gc_bottom_most_compaction=True,
|
||||
body={
|
||||
"scheduled": True,
|
||||
"sub_compaction": True,
|
||||
"compact_range": {
|
||||
"start": "000000000000000000000000000000000000",
|
||||
"end": "030000000000000000000000000000000000",
|
||||
},
|
||||
},
|
||||
},
|
||||
)
|
||||
)
|
||||
|
||||
workload.churn_rows(row_count, env.pageserver.id)
|
||||
|
||||
@@ -181,6 +178,10 @@ def test_pageserver_gc_compaction_smoke(neon_env_builder: NeonEnvBuilder):
|
||||
log.info("Validating at workload end ...")
|
||||
workload.validate(env.pageserver.id)
|
||||
|
||||
# Run a legacy compaction+gc to ensure gc-compaction can coexist with legacy compaction.
|
||||
ps_http.timeline_checkpoint(tenant_id, timeline_id, wait_until_uploaded=True)
|
||||
ps_http.timeline_gc(tenant_id, timeline_id, None)
|
||||
|
||||
|
||||
# Stripe sizes in number of pages.
|
||||
TINY_STRIPES = 16
|
||||
|
||||
@@ -33,6 +33,7 @@ deranged = { version = "0.3", default-features = false, features = ["powerfmt",
|
||||
digest = { version = "0.10", features = ["mac", "oid", "std"] }
|
||||
either = { version = "1" }
|
||||
fail = { version = "0.5", default-features = false, features = ["failpoints"] }
|
||||
form_urlencoded = { version = "1" }
|
||||
futures-channel = { version = "0.3", features = ["sink"] }
|
||||
futures-executor = { version = "0.3" }
|
||||
futures-io = { version = "0.3" }
|
||||
@@ -78,6 +79,7 @@ sha2 = { version = "0.10", features = ["asm", "oid"] }
|
||||
signature = { version = "2", default-features = false, features = ["digest", "rand_core", "std"] }
|
||||
smallvec = { version = "1", default-features = false, features = ["const_new", "write"] }
|
||||
spki = { version = "0.7", default-features = false, features = ["pem", "std"] }
|
||||
stable_deref_trait = { version = "1" }
|
||||
subtle = { version = "2" }
|
||||
sync_wrapper = { version = "0.1", default-features = false, features = ["futures"] }
|
||||
tikv-jemalloc-ctl = { version = "0.6", features = ["stats", "use_std"] }
|
||||
@@ -105,6 +107,7 @@ anyhow = { version = "1", features = ["backtrace"] }
|
||||
bytes = { version = "1", features = ["serde"] }
|
||||
cc = { version = "1", default-features = false, features = ["parallel"] }
|
||||
chrono = { version = "0.4", default-features = false, features = ["clock", "serde", "wasmbind"] }
|
||||
displaydoc = { version = "0.2" }
|
||||
either = { version = "1" }
|
||||
getrandom = { version = "0.2", default-features = false, features = ["std"] }
|
||||
half = { version = "2", default-features = false, features = ["num-traits"] }
|
||||
|
||||
Reference in New Issue
Block a user