tests: Default to LOCAL_FS for pageserver remote storage (#5402)

Part of #5172. Builds upon #5243, #5298. Includes the test changes:
- no more RemoteStorageKind.NOOP
- no more testing of pageserver without remote storage
- benchmarks now use LOCAL_FS as well

Support for running without RemoteStorage is still kept but in practice,
there are no tests and should not be any tests.

Co-authored-by: Christian Schwarz <christian@neon.tech>
This commit is contained in:
Joonas Koivunen
2023-09-28 12:25:20 +03:00
committed by GitHub
parent 6b4bb91d0a
commit af28362a47
11 changed files with 109 additions and 199 deletions

View File

@@ -644,7 +644,7 @@ fn create_remote_storage_client(
let config = if let Some(config) = &conf.remote_storage_config {
config
} else {
// No remote storage configured.
tracing::warn!("no remote storage configured, this is a deprecated configuration");
return Ok(None);
};

View File

@@ -105,6 +105,8 @@ class NeonCompare(PgCompare):
self._pg_bin = pg_bin
self.pageserver_http_client = self.env.pageserver.http_client()
# note that neon_simple_env now uses LOCAL_FS remote storage
# Create tenant
tenant_conf: Dict[str, str] = {}
if False: # TODO add pytest setting for this

View File

@@ -460,9 +460,11 @@ class NeonEnvBuilder:
), "Unexpectedly instantiated from outside a test function"
self.test_name = test_name
def init_configs(self) -> NeonEnv:
def init_configs(self, default_remote_storage_if_missing: bool = True) -> NeonEnv:
# Cannot create more than one environment from one builder
assert self.env is None, "environment already initialized"
if default_remote_storage_if_missing and self.pageserver_remote_storage is None:
self.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
self.env = NeonEnv(self)
return self.env
@@ -470,8 +472,19 @@ class NeonEnvBuilder:
assert self.env is not None, "environment is not already initialized, call init() first"
self.env.start()
def init_start(self, initial_tenant_conf: Optional[Dict[str, str]] = None) -> NeonEnv:
env = self.init_configs()
def init_start(
self,
initial_tenant_conf: Optional[Dict[str, str]] = None,
default_remote_storage_if_missing: bool = True,
) -> NeonEnv:
"""
Default way to create and start NeonEnv. Also creates the initial_tenant with root initial_timeline.
To avoid creating initial_tenant, call init_configs to setup the environment.
Configuring pageserver with remote storage is now the default. There will be a warning if pageserver is created without one.
"""
env = self.init_configs(default_remote_storage_if_missing=default_remote_storage_if_missing)
self.start()
# Prepare the default branch to start the postgres on later.
@@ -546,7 +559,7 @@ class NeonEnvBuilder:
user: RemoteStorageUser,
bucket_name: Optional[str] = None,
bucket_region: Optional[str] = None,
) -> Optional[RemoteStorage]:
) -> RemoteStorage:
ret = kind.configure(
self.repo_dir,
self.mock_s3_server,
@@ -889,6 +902,8 @@ def _shared_simple_env(
"""
# Internal fixture backing the `neon_simple_env` fixture. If TEST_SHARED_FIXTURES
is set, this is shared by all tests using `neon_simple_env`.
This fixture will use RemoteStorageKind.LOCAL_FS with pageserver.
"""
if os.environ.get("TEST_SHARED_FIXTURES") is None:

View File

@@ -202,9 +202,6 @@ class RemoteStorageKind(str, enum.Enum):
LOCAL_FS = "local_fs"
MOCK_S3 = "mock_s3"
REAL_S3 = "real_s3"
# Pass to tests that are generic to remote storage
# to ensure the test pass with or without the remote storage
NOOP = "noop"
def configure(
self,
@@ -215,10 +212,7 @@ class RemoteStorageKind(str, enum.Enum):
user: RemoteStorageUser,
bucket_name: Optional[str] = None,
bucket_region: Optional[str] = None,
) -> Optional[RemoteStorage]:
if self == RemoteStorageKind.NOOP:
return None
) -> RemoteStorage:
if self == RemoteStorageKind.LOCAL_FS:
return LocalFsStorage(LocalFsStorage.component_path(repo_dir, user))

View File

@@ -4,7 +4,12 @@ from typing import List, Tuple
import pytest
from fixtures.log_helper import log
from fixtures.neon_fixtures import Endpoint, NeonEnv, NeonEnvBuilder
from fixtures.neon_fixtures import (
Endpoint,
NeonEnv,
NeonEnvBuilder,
wait_for_last_flush_lsn,
)
from fixtures.types import TenantId, TimelineId
@@ -26,17 +31,18 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder):
tenant_timelines: List[Tuple[TenantId, TimelineId, Endpoint]] = []
for _ in range(4):
for _ in range(3):
tenant_id, timeline_id = env.neon_cli.create_tenant()
endpoint = env.endpoints.create_start("main", tenant_id=tenant_id)
with endpoint.cursor() as cur:
cur.execute("CREATE TABLE t(key int primary key, value text)")
cur.execute("INSERT INTO t SELECT generate_series(1,100), 'payload'")
wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id)
endpoint.stop()
tenant_timelines.append((tenant_id, timeline_id, endpoint))
# Stop the pageserver
# Stop the pageserver -- this has to be not immediate or we need to wait for uploads
env.pageserver.stop()
# Leave the first timeline alone, but corrupt the others in different ways
@@ -45,30 +51,21 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder):
(tenant1, timeline1, pg1) = tenant_timelines[1]
metadata_path = f"{env.pageserver.workdir}/tenants/{tenant1}/timelines/{timeline1}/metadata"
f = open(metadata_path, "w")
f.write("overwritten with garbage!")
f.close()
with open(metadata_path, "w") as f:
f.write("overwritten with garbage!")
log.info(f"Timeline {tenant1}/{timeline1} got its metadata spoiled")
(tenant2, timeline2, pg2) = tenant_timelines[2]
timeline_path = f"{env.pageserver.workdir}/tenants/{tenant2}/timelines/{timeline2}/"
for filename in os.listdir(timeline_path):
if filename.startswith("00000"):
# Looks like a layer file. Remove it
os.remove(f"{timeline_path}/{filename}")
log.info(
f"Timeline {tenant2}/{timeline2} got its layer files removed (no remote storage enabled)"
)
(tenant3, timeline3, pg3) = tenant_timelines[3]
timeline_path = f"{env.pageserver.workdir}/tenants/{tenant3}/timelines/{timeline3}/"
for filename in os.listdir(timeline_path):
if filename.startswith("00000"):
# Looks like a layer file. Corrupt it
f = open(f"{timeline_path}/{filename}", "w")
f.write("overwritten with garbage!")
f.close()
log.info(f"Timeline {tenant3}/{timeline3} got its layer files spoiled")
p = f"{timeline_path}/{filename}"
size = os.path.getsize(p)
with open(p, "wb") as f:
f.truncate(0)
f.truncate(size)
log.info(f"Timeline {tenant2}/{timeline2} got its local layer files spoiled")
env.pageserver.start()
@@ -87,22 +84,13 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder):
f"As expected, compute startup failed eagerly for timeline with corrupt metadata: {err}"
)
# Second timeline has no ancestors, only the metadata file and no layer files locally,
# and we don't have the remote storage enabled. It is loaded into memory, but getting
# the basebackup from it will fail.
with pytest.raises(
Exception, match=f"Tenant {tenant2} will not become active. Current state: Broken"
) as err:
pg2.start()
log.info(f"As expected, compute startup failed for timeline with missing layers: {err}")
# Third timeline will also fail during basebackup, because the layer file is corrupt.
# Second timeline will fail during basebackup, because the local layer file is corrupt.
# It will fail when we try to read (and reconstruct) a page from it, ergo the error message.
# (We don't check layer file contents on startup, when loading the timeline)
with pytest.raises(Exception, match="Failed to load delta layer") as err:
pg3.start()
pg2.start()
log.info(
f"As expected, compute startup failed for timeline {tenant3}/{timeline3} with corrupt layers: {err}"
f"As expected, compute startup failed for timeline {tenant2}/{timeline2} with corrupt layers: {err}"
)

View File

@@ -5,7 +5,6 @@ from pathlib import Path
from queue import SimpleQueue
from typing import Any, Dict, Set
import pytest
from fixtures.log_helper import log
from fixtures.neon_fixtures import (
NeonEnvBuilder,
@@ -17,18 +16,13 @@ from pytest_httpserver import HTTPServer
from werkzeug.wrappers.request import Request
from werkzeug.wrappers.response import Response
# TODO: collect all of the env setup *AFTER* removal of RemoteStorageKind.NOOP
@pytest.mark.parametrize(
"remote_storage_kind", [RemoteStorageKind.NOOP, RemoteStorageKind.LOCAL_FS]
)
def test_metric_collection(
httpserver: HTTPServer,
neon_env_builder: NeonEnvBuilder,
httpserver_listen_address,
remote_storage_kind: RemoteStorageKind,
):
(host, port) = httpserver_listen_address
metric_collection_endpoint = f"http://{host}:{port}/billing/api/v1/usage_events"
@@ -58,7 +52,7 @@ def test_metric_collection(
synthetic_size_calculation_interval="3s"
"""
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
log.info(f"test_metric_collection endpoint is {metric_collection_endpoint}")
@@ -109,17 +103,14 @@ def test_metric_collection(
total += sample[2]
return int(total)
remote_uploaded = 0
# upload some data to remote storage
if remote_storage_kind == RemoteStorageKind.LOCAL_FS:
wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id)
pageserver_http = env.pageserver.http_client()
pageserver_http.timeline_checkpoint(tenant_id, timeline_id)
pageserver_http.timeline_gc(tenant_id, timeline_id, 10000)
wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id)
pageserver_http = env.pageserver.http_client()
pageserver_http.timeline_checkpoint(tenant_id, timeline_id)
pageserver_http.timeline_gc(tenant_id, timeline_id, 10000)
remote_uploaded = get_num_remote_ops("index", "upload")
assert remote_uploaded > 0
remote_uploaded = get_num_remote_ops("index", "upload")
assert remote_uploaded > 0
# we expect uploads at 1Hz, on busy runners this could be too optimistic,
# so give 5s we only want to get the following upload after "ready" value.

View File

@@ -30,9 +30,7 @@ from fixtures.types import TenantId
from fixtures.utils import run_pg_bench_small
@pytest.mark.parametrize(
"remote_storage_kind", [RemoteStorageKind.NOOP, *available_remote_storages()]
)
@pytest.mark.parametrize("remote_storage_kind", available_remote_storages())
def test_tenant_delete_smoke(
neon_env_builder: NeonEnvBuilder,
remote_storage_kind: RemoteStorageKind,
@@ -144,18 +142,12 @@ FAILPOINTS_BEFORE_BACKGROUND = [
def combinations():
result = []
remotes = [RemoteStorageKind.NOOP, RemoteStorageKind.MOCK_S3]
remotes = [RemoteStorageKind.MOCK_S3]
if os.getenv("ENABLE_REAL_S3_REMOTE_STORAGE"):
remotes.append(RemoteStorageKind.REAL_S3)
for remote_storage_kind in remotes:
for delete_failpoint in FAILPOINTS:
if remote_storage_kind is RemoteStorageKind.NOOP and delete_failpoint in (
"timeline-delete-before-index-delete",
):
# the above failpoint are not relevant for config without remote storage
continue
# Simulate failures for only one type of remote storage
# to avoid log pollution and make tests run faster
if remote_storage_kind is RemoteStorageKind.MOCK_S3:
@@ -215,21 +207,18 @@ def test_delete_tenant_exercise_crash_safety_failpoints(
with env.endpoints.create_start("delete", tenant_id=tenant_id) as endpoint:
# generate enough layers
run_pg_bench_small(pg_bin, endpoint.connstr())
if remote_storage_kind is RemoteStorageKind.NOOP:
wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id)
else:
last_flush_lsn_upload(env, endpoint, tenant_id, timeline_id)
last_flush_lsn_upload(env, endpoint, tenant_id, timeline_id)
if remote_storage_kind in available_s3_storages():
assert_prefix_not_empty(
neon_env_builder,
prefix="/".join(
(
"tenants",
str(tenant_id),
)
),
)
if remote_storage_kind in available_s3_storages():
assert_prefix_not_empty(
neon_env_builder,
prefix="/".join(
(
"tenants",
str(tenant_id),
)
),
)
ps_http.configure_failpoints((failpoint, "return"))
@@ -260,12 +249,7 @@ def test_delete_tenant_exercise_crash_safety_failpoints(
env.pageserver.stop()
env.pageserver.start()
if (
remote_storage_kind is RemoteStorageKind.NOOP
and failpoint == "tenant-delete-before-create-local-mark"
):
tenant_delete_wait_completed(ps_http, tenant_id, iterations=iterations)
elif failpoint in (
if failpoint in (
"tenant-delete-before-shutdown",
"tenant-delete-before-create-remote-mark",
):

View File

@@ -519,11 +519,8 @@ def test_detach_while_attaching(
# * restart the pageserver and verify that ignored tenant is still not loaded
# * `load` the same tenant
# * ensure that it's status is `Active` and it's present in pageserver's memory with all timelines
@pytest.mark.parametrize("remote_storage_kind", [RemoteStorageKind.NOOP, RemoteStorageKind.MOCK_S3])
def test_ignored_tenant_reattach(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: RemoteStorageKind
):
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
def test_ignored_tenant_reattach(neon_env_builder: NeonEnvBuilder):
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.MOCK_S3)
env = neon_env_builder.init_start()
pageserver_http = env.pageserver.http_client()

View File

@@ -12,7 +12,6 @@ from fixtures.log_helper import log
from fixtures.metrics import (
PAGESERVER_GLOBAL_METRICS,
PAGESERVER_PER_TENANT_METRICS,
PAGESERVER_PER_TENANT_REMOTE_TIMELINE_CLIENT_METRICS,
parse_metrics,
)
from fixtures.neon_fixtures import (
@@ -232,17 +231,10 @@ def test_metrics_normal_work(neon_env_builder: NeonEnvBuilder):
assert value
@pytest.mark.parametrize(
"remote_storage_kind",
# exercise both the code paths where remote_storage=None and remote_storage=Some(...)
[RemoteStorageKind.NOOP, RemoteStorageKind.MOCK_S3],
)
def test_pageserver_metrics_removed_after_detach(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: RemoteStorageKind
):
def test_pageserver_metrics_removed_after_detach(neon_env_builder: NeonEnvBuilder):
"""Tests that when a tenant is detached, the tenant specific metrics are not left behind"""
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.MOCK_S3)
neon_env_builder.num_safekeepers = 3
@@ -282,9 +274,6 @@ def test_pageserver_metrics_removed_after_detach(
for tenant in [tenant_1, tenant_2]:
pre_detach_samples = set([x.name for x in get_ps_metric_samples_for_tenant(tenant)])
expected = set(PAGESERVER_PER_TENANT_METRICS)
if remote_storage_kind == RemoteStorageKind.NOOP:
# if there's no remote storage configured, we don't expose the remote timeline client metrics
expected -= set(PAGESERVER_PER_TENANT_REMOTE_TIMELINE_CLIENT_METRICS)
assert pre_detach_samples == expected
env.pageserver.http_client().tenant_detach(tenant)
@@ -294,9 +283,7 @@ def test_pageserver_metrics_removed_after_detach(
# Check that empty tenants work with or without the remote storage
@pytest.mark.parametrize(
"remote_storage_kind", available_remote_storages() + [RemoteStorageKind.NOOP]
)
@pytest.mark.parametrize("remote_storage_kind", available_remote_storages())
def test_pageserver_with_empty_tenants(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: RemoteStorageKind
):

View File

@@ -12,7 +12,6 @@ from fixtures.neon_fixtures import (
NeonEnvBuilder,
PgBin,
last_flush_lsn_upload,
wait_for_last_flush_lsn,
)
from fixtures.pageserver.http import PageserverApiException
from fixtures.pageserver.utils import (
@@ -145,19 +144,12 @@ DELETE_FAILPOINTS = [
def combinations():
result = []
remotes = [RemoteStorageKind.NOOP, RemoteStorageKind.MOCK_S3]
remotes = [RemoteStorageKind.MOCK_S3]
if os.getenv("ENABLE_REAL_S3_REMOTE_STORAGE"):
remotes.append(RemoteStorageKind.REAL_S3)
for remote_storage_kind in remotes:
for delete_failpoint in DELETE_FAILPOINTS:
if remote_storage_kind == RemoteStorageKind.NOOP and delete_failpoint in (
"timeline-delete-before-index-delete",
"timeline-delete-after-index-delete",
):
# the above failpoints are not relevant for config without remote storage
continue
result.append((remote_storage_kind, delete_failpoint))
return result
@@ -205,23 +197,21 @@ def test_delete_timeline_exercise_crash_safety_failpoints(
with env.endpoints.create_start("delete") as endpoint:
# generate enough layers
run_pg_bench_small(pg_bin, endpoint.connstr())
if remote_storage_kind is RemoteStorageKind.NOOP:
wait_for_last_flush_lsn(env, endpoint, env.initial_tenant, timeline_id)
else:
last_flush_lsn_upload(env, endpoint, env.initial_tenant, timeline_id)
if remote_storage_kind in available_s3_storages():
assert_prefix_not_empty(
neon_env_builder,
prefix="/".join(
(
"tenants",
str(env.initial_tenant),
"timelines",
str(timeline_id),
)
),
)
last_flush_lsn_upload(env, endpoint, env.initial_tenant, timeline_id)
if remote_storage_kind in available_s3_storages():
assert_prefix_not_empty(
neon_env_builder,
prefix="/".join(
(
"tenants",
str(env.initial_tenant),
"timelines",
str(timeline_id),
)
),
)
env.pageserver.allowed_errors.append(f".*{timeline_id}.*failpoint: {failpoint}")
# It appears when we stopped flush loop during deletion and then pageserver is stopped

View File

@@ -301,12 +301,8 @@ def test_timeline_initial_logical_size_calculation_cancellation(
# message emitted by the code behind failpoint "timeline-calculate-logical-size-check-dir-exists"
@pytest.mark.parametrize("remote_storage_kind", [None, RemoteStorageKind.LOCAL_FS])
def test_timeline_physical_size_init(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: Optional[RemoteStorageKind]
):
if remote_storage_kind is not None:
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
def test_timeline_physical_size_init(neon_env_builder: NeonEnvBuilder):
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
env = neon_env_builder.init_start()
@@ -337,17 +333,12 @@ def test_timeline_physical_size_init(
)
assert_physical_size_invariants(
get_physical_size_values(env, env.initial_tenant, new_timeline_id, remote_storage_kind),
remote_storage_kind,
get_physical_size_values(env, env.initial_tenant, new_timeline_id),
)
@pytest.mark.parametrize("remote_storage_kind", [None, RemoteStorageKind.LOCAL_FS])
def test_timeline_physical_size_post_checkpoint(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: Optional[RemoteStorageKind]
):
if remote_storage_kind is not None:
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
def test_timeline_physical_size_post_checkpoint(neon_env_builder: NeonEnvBuilder):
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
env = neon_env_builder.init_start()
@@ -369,19 +360,14 @@ def test_timeline_physical_size_post_checkpoint(
def check():
assert_physical_size_invariants(
get_physical_size_values(env, env.initial_tenant, new_timeline_id, remote_storage_kind),
remote_storage_kind,
get_physical_size_values(env, env.initial_tenant, new_timeline_id),
)
wait_until(10, 1, check)
@pytest.mark.parametrize("remote_storage_kind", [None, RemoteStorageKind.LOCAL_FS])
def test_timeline_physical_size_post_compaction(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: Optional[RemoteStorageKind]
):
if remote_storage_kind is not None:
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
def test_timeline_physical_size_post_compaction(neon_env_builder: NeonEnvBuilder):
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
# Disable background compaction as we don't want it to happen after `get_physical_size` request
# and before checking the expected size on disk, which makes the assertion failed
@@ -420,21 +406,15 @@ def test_timeline_physical_size_post_compaction(
pageserver_http.timeline_checkpoint(env.initial_tenant, new_timeline_id)
pageserver_http.timeline_compact(env.initial_tenant, new_timeline_id)
if remote_storage_kind is not None:
wait_for_upload_queue_empty(pageserver_http, env.initial_tenant, new_timeline_id)
wait_for_upload_queue_empty(pageserver_http, env.initial_tenant, new_timeline_id)
assert_physical_size_invariants(
get_physical_size_values(env, env.initial_tenant, new_timeline_id, remote_storage_kind),
remote_storage_kind,
get_physical_size_values(env, env.initial_tenant, new_timeline_id),
)
@pytest.mark.parametrize("remote_storage_kind", [None, RemoteStorageKind.LOCAL_FS])
def test_timeline_physical_size_post_gc(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: Optional[RemoteStorageKind]
):
if remote_storage_kind is not None:
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
def test_timeline_physical_size_post_gc(neon_env_builder: NeonEnvBuilder):
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
# Disable background compaction and GC as we don't want it to happen after `get_physical_size` request
# and before checking the expected size on disk, which makes the assertion failed
@@ -471,12 +451,10 @@ def test_timeline_physical_size_post_gc(
pageserver_http.timeline_checkpoint(env.initial_tenant, new_timeline_id)
pageserver_http.timeline_gc(env.initial_tenant, new_timeline_id, gc_horizon=None)
if remote_storage_kind is not None:
wait_for_upload_queue_empty(pageserver_http, env.initial_tenant, new_timeline_id)
wait_for_upload_queue_empty(pageserver_http, env.initial_tenant, new_timeline_id)
assert_physical_size_invariants(
get_physical_size_values(env, env.initial_tenant, new_timeline_id, remote_storage_kind),
remote_storage_kind,
get_physical_size_values(env, env.initial_tenant, new_timeline_id),
)
@@ -560,14 +538,10 @@ def test_timeline_size_metrics(
assert math.isclose(dbsize_sum, tl_logical_size_metric, abs_tol=2 * 1024 * 1024)
@pytest.mark.parametrize("remote_storage_kind", [None, RemoteStorageKind.LOCAL_FS])
def test_tenant_physical_size(
neon_env_builder: NeonEnvBuilder, remote_storage_kind: Optional[RemoteStorageKind]
):
def test_tenant_physical_size(neon_env_builder: NeonEnvBuilder):
random.seed(100)
if remote_storage_kind is not None:
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
env = neon_env_builder.init_start()
@@ -575,12 +549,10 @@ def test_tenant_physical_size(
client = env.pageserver.http_client()
tenant, timeline = env.neon_cli.create_tenant()
if remote_storage_kind is not None:
wait_for_upload_queue_empty(pageserver_http, tenant, timeline)
def get_timeline_resident_physical_size(timeline: TimelineId):
sizes = get_physical_size_values(env, tenant, timeline, remote_storage_kind)
assert_physical_size_invariants(sizes, remote_storage_kind)
sizes = get_physical_size_values(env, tenant, timeline)
assert_physical_size_invariants(sizes)
return sizes.prometheus_resident_physical
timeline_total_resident_physical_size = get_timeline_resident_physical_size(timeline)
@@ -600,8 +572,7 @@ def test_tenant_physical_size(
wait_for_last_flush_lsn(env, endpoint, tenant, timeline)
pageserver_http.timeline_checkpoint(tenant, timeline)
if remote_storage_kind is not None:
wait_for_upload_queue_empty(pageserver_http, tenant, timeline)
wait_for_upload_queue_empty(pageserver_http, tenant, timeline)
timeline_total_resident_physical_size += get_timeline_resident_physical_size(timeline)
@@ -630,7 +601,6 @@ def get_physical_size_values(
env: NeonEnv,
tenant_id: TenantId,
timeline_id: TimelineId,
remote_storage_kind: Optional[RemoteStorageKind],
) -> TimelinePhysicalSizeValues:
res = TimelinePhysicalSizeValues()
@@ -646,12 +616,9 @@ def get_physical_size_values(
res.prometheus_resident_physical = metrics.query_one(
"pageserver_resident_physical_size", metrics_filter
).value
if remote_storage_kind is not None:
res.prometheus_remote_physical = metrics.query_one(
"pageserver_remote_physical_size", metrics_filter
).value
else:
res.prometheus_remote_physical = None
res.prometheus_remote_physical = metrics.query_one(
"pageserver_remote_physical_size", metrics_filter
).value
detail = client.timeline_detail(
tenant_id, timeline_id, include_timeline_dir_layer_file_size_sum=True
@@ -664,20 +631,15 @@ def get_physical_size_values(
return res
def assert_physical_size_invariants(
sizes: TimelinePhysicalSizeValues, remote_storage_kind: Optional[RemoteStorageKind]
):
def assert_physical_size_invariants(sizes: TimelinePhysicalSizeValues):
# resident phyiscal size is defined as
assert sizes.python_timelinedir_layerfiles_physical == sizes.prometheus_resident_physical
assert sizes.python_timelinedir_layerfiles_physical == sizes.layer_map_file_size_sum
# we don't do layer eviction, so, all layers are resident
assert sizes.api_current_physical == sizes.prometheus_resident_physical
if remote_storage_kind is not None:
assert sizes.prometheus_resident_physical == sizes.prometheus_remote_physical
# XXX would be nice to assert layer file physical storage utilization here as well, but we can only do that for LocalFS
else:
assert sizes.prometheus_remote_physical is None
assert sizes.prometheus_resident_physical == sizes.prometheus_remote_physical
# XXX would be nice to assert layer file physical storage utilization here as well, but we can only do that for LocalFS
# Timeline logical size initialization is an asynchronous background task that runs once,