Files
neon/test_runner/regress/test_pageserver_layer_rolling.py
Christian Schwarz 9627747d35 bypass PageCache for InMemoryLayer + avoid Value::deser on L0 flush (#8537)
Part of [Epic: Bypass PageCache for user data
blocks](https://github.com/neondatabase/neon/issues/7386).

# Problem

`InMemoryLayer` still uses the `PageCache` for all data stored in the
`VirtualFile` that underlies the `EphemeralFile`.

# Background

Before this PR, `EphemeralFile` is a fancy and (code-bloated) buffered
writer around a `VirtualFile` that supports `blob_io`.

The `InMemoryLayerInner::index` stores offsets into the `EphemeralFile`.
At those offset, we find a varint length followed by the serialized
`Value`.

Vectored reads (`get_values_reconstruct_data`) are not in fact vectored
- each `Value` that needs to be read is read sequentially.

The `will_init` bit of information which we use to early-exit the
`get_values_reconstruct_data` for a given key is stored in the
serialized `Value`, meaning we have to read & deserialize the `Value`
from the `EphemeralFile`.

The L0 flushing **also** needs to re-determine the `will_init` bit of
information, by deserializing each value during L0 flush.

# Changes

1. Store the value length and `will_init` information in the
`InMemoryLayer::index`. The `EphemeralFile` thus only needs to store the
values.
2. For `get_values_reconstruct_data`:
- Use the in-memory `index` figures out which values need to be read.
Having the `will_init` stored in the index enables us to do that.
- View the EphemeralFile as a byte array of "DIO chunks", each 512 bytes
in size (adjustable constant). A "DIO chunk" is the minimal unit that we
can read under direct IO.
- Figure out which chunks need to be read to retrieve the serialized
bytes for thes values we need to read.
- Coalesce chunk reads such that each DIO chunk is only read once to
serve all value reads that need data from that chunk.
- Merge adjacent chunk reads into larger
`EphemeralFile::read_exact_at_eof_ok` of up to 128k (adjustable
constant).
3. The new `EphemeralFile::read_exact_at_eof_ok` fills the IO buffer
from the underlying VirtualFile and/or its in-memory buffer.
4. The L0 flush code is changed to use the `index` directly, `blob_io` 
5. We can remove the `ephemeral_file::page_caching` construct now.

The `get_values_reconstruct_data` changes seem like a bit overkill but
they are necessary so we issue the equivalent amount of read system
calls compared to before this PR where it was highly likely that even if
the first PageCache access was a miss, remaining reads within the same
`get_values_reconstruct_data` call from the same `EphemeralFile` page
were a hit.

The "DIO chunk" stuff is truly unnecessary for page cache bypass, but,
since we're working on [direct
IO](https://github.com/neondatabase/neon/issues/8130) and
https://github.com/neondatabase/neon/issues/8719 specifically, we need
to do _something_ like this anyways in the near future.

# Alternative Design

The original plan was to use the `vectored_blob_io` code it relies on
the invariant of Delta&Image layers that `index order == values order`.

Further, `vectored_blob_io` code's strategy for merging IOs is limited
to adjacent reads. However, with direct IO, there is another level of
merging that should be done, specifically, if multiple reads map to the
same "DIO chunk" (=alignment-requirement-sized and -aligned region of
the file), then it's "free" to read the chunk into an IO buffer and
serve the two reads from that buffer.
=> https://github.com/neondatabase/neon/issues/8719

# Testing / Performance

Correctness of the IO merging code is ensured by unit tests.

Additionally, minimal tests are added for the `EphemeralFile`
implementation and the bit-packed `InMemoryLayerIndexValue`.

Performance testing results are presented below.
All pref testing done on my M2 MacBook Pro, running a Linux VM.
It's a release build without `--features testing`.

We see definitive improvement in ingest performance microbenchmark and
an ad-hoc microbenchmark for getpage against InMemoryLayer.

```
baseline: commit 7c74112b2a origin/main
HEAD: ef1c55c52e
```

<details>

```
cargo bench --bench bench_ingest -- 'ingest 128MB/100b seq, no delta'

baseline

ingest-small-values/ingest 128MB/100b seq, no delta
                        time:   [483.50 ms 498.73 ms 522.53 ms]
                        thrpt:  [244.96 MiB/s 256.65 MiB/s 264.73 MiB/s]

HEAD

ingest-small-values/ingest 128MB/100b seq, no delta
                        time:   [479.22 ms 482.92 ms 487.35 ms]
                        thrpt:  [262.64 MiB/s 265.06 MiB/s 267.10 MiB/s]
```

</details>

We don't have a micro-benchmark for InMemoryLayer and it's quite
cumbersome to add one. So, I did manual testing in `neon_local`.

<details>

```

  ./target/release/neon_local stop
  rm -rf .neon
  ./target/release/neon_local init
  ./target/release/neon_local start
  ./target/release/neon_local tenant create --set-default
  ./target/release/neon_local endpoint create foo
  ./target/release/neon_local endpoint start foo
  psql 'postgresql://cloud_admin@127.0.0.1:55432/postgres'
psql (13.16 (Debian 13.16-0+deb11u1), server 15.7)

CREATE TABLE wal_test (
    id SERIAL PRIMARY KEY,
    data TEXT
);

DO $$
DECLARE
    i INTEGER := 1;
BEGIN
    WHILE i <= 500000 LOOP
        INSERT INTO wal_test (data) VALUES ('data');
        i := i + 1;
    END LOOP;
END $$;

-- => result is one L0 from initdb and one 137M-sized ephemeral-2

DO $$
DECLARE
    i INTEGER := 1;
    random_id INTEGER;
    random_record wal_test%ROWTYPE;
    start_time TIMESTAMP := clock_timestamp();
    selects_completed INTEGER := 0;
    min_id INTEGER := 1;  -- Minimum ID value
    max_id INTEGER := 100000;  -- Maximum ID value, based on your insert range
    iters INTEGER := 100000000;  -- Number of iterations to run
BEGIN
    WHILE i <= iters LOOP
        -- Generate a random ID within the known range
        random_id := min_id + floor(random() * (max_id - min_id + 1))::int;

        -- Select the row with the generated random ID
        SELECT * INTO random_record
        FROM wal_test
        WHERE id = random_id;

        -- Increment the select counter
        selects_completed := selects_completed + 1;

        -- Check if a second has passed
        IF EXTRACT(EPOCH FROM clock_timestamp() - start_time) >= 1 THEN
            -- Print the number of selects completed in the last second
            RAISE NOTICE 'Selects completed in last second: %', selects_completed;

            -- Reset counters for the next second
            selects_completed := 0;
            start_time := clock_timestamp();
        END IF;

        -- Increment the loop counter
        i := i + 1;
    END LOOP;
END $$;

./target/release/neon_local stop

baseline: commit 7c74112b2a origin/main

NOTICE:  Selects completed in last second: 1864
NOTICE:  Selects completed in last second: 1850
NOTICE:  Selects completed in last second: 1851
NOTICE:  Selects completed in last second: 1918
NOTICE:  Selects completed in last second: 1911
NOTICE:  Selects completed in last second: 1879
NOTICE:  Selects completed in last second: 1858
NOTICE:  Selects completed in last second: 1827
NOTICE:  Selects completed in last second: 1933

ours

NOTICE:  Selects completed in last second: 1915
NOTICE:  Selects completed in last second: 1928
NOTICE:  Selects completed in last second: 1913
NOTICE:  Selects completed in last second: 1932
NOTICE:  Selects completed in last second: 1846
NOTICE:  Selects completed in last second: 1955
NOTICE:  Selects completed in last second: 1991
NOTICE:  Selects completed in last second: 1973
```

NB: the ephemeral file sizes differ by ca 1MiB, ours being 1MiB smaller.

</details>

# Rollout

This PR changes the code in-place and  is not gated by a feature flag.
2024-08-28 18:31:41 +00:00

319 lines
12 KiB
Python

import asyncio
import os
import time
from typing import Optional, Tuple
import psutil
import pytest
from fixtures.common_types import Lsn, TenantId, TimelineId
from fixtures.log_helper import log
from fixtures.neon_fixtures import (
NeonEnv,
NeonEnvBuilder,
tenant_get_shards,
)
from fixtures.pageserver.http import PageserverHttpClient
from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload
from fixtures.utils import wait_until
TIMELINE_COUNT = 10
ENTRIES_PER_TIMELINE = 10_000
CHECKPOINT_TIMEOUT_SECONDS = 60
async def run_worker_for_tenant(
env: NeonEnv, entries: int, tenant: TenantId, offset: Optional[int] = None
) -> Lsn:
if offset is None:
offset = 0
with env.endpoints.create_start("main", tenant_id=tenant) as ep:
conn = await ep.connect_async()
try:
await conn.execute("CREATE TABLE IF NOT EXISTS t(key serial primary key, value text)")
await conn.execute(
f"INSERT INTO t SELECT i, CONCAT('payload_', i) FROM generate_series({offset},{entries}) as i"
)
finally:
await conn.close(timeout=10)
last_flush_lsn = Lsn(ep.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0])
return last_flush_lsn
async def run_worker(env: NeonEnv, tenant_conf, entries: int) -> Tuple[TenantId, TimelineId, Lsn]:
tenant, timeline = env.neon_cli.create_tenant(conf=tenant_conf)
last_flush_lsn = await run_worker_for_tenant(env, entries, tenant)
return tenant, timeline, last_flush_lsn
async def workload(
env: NeonEnv, tenant_conf, timelines: int, entries: int
) -> list[Tuple[TenantId, TimelineId, Lsn]]:
workers = [asyncio.create_task(run_worker(env, tenant_conf, entries)) for _ in range(timelines)]
return await asyncio.gather(*workers)
def wait_until_pageserver_is_caught_up(
env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]]
):
for tenant, timeline, last_flush_lsn in last_flush_lsns:
shards = tenant_get_shards(env, tenant)
for tenant_shard_id, pageserver in shards:
waited = wait_for_last_record_lsn(
pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn
)
assert waited >= last_flush_lsn
def wait_until_pageserver_has_uploaded(
env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]]
):
for tenant, timeline, last_flush_lsn in last_flush_lsns:
shards = tenant_get_shards(env, tenant)
for tenant_shard_id, pageserver in shards:
wait_for_upload(pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn)
def wait_for_wal_ingest_metric(pageserver_http: PageserverHttpClient) -> float:
def query():
value = pageserver_http.get_metric_value("pageserver_wal_ingest_records_received_total")
assert value is not None
return value
# The metric gets initialised on the first update.
# Retry a few times, but return 0 if it's stable.
try:
return float(wait_until(3, 0.5, query))
except Exception:
return 0
def get_dirty_bytes(env):
v = env.pageserver.http_client().get_metric_value("pageserver_timeline_ephemeral_bytes") or 0
log.info(f"dirty_bytes: {v}")
return v
def assert_dirty_bytes(env, v):
assert get_dirty_bytes(env) == v
def assert_dirty_bytes_nonzero(env):
dirty_bytes = get_dirty_bytes(env)
assert dirty_bytes > 0
return dirty_bytes
@pytest.mark.parametrize("immediate_shutdown", [True, False])
def test_pageserver_small_inmemory_layers(
neon_env_builder: NeonEnvBuilder, immediate_shutdown: bool
):
"""
Test that open layers get flushed after the `checkpoint_timeout` config
and do not require WAL reingest upon restart.
The workload creates a number of timelines and writes some data to each,
but not enough to trigger flushes via the `checkpoint_distance` config.
"""
tenant_conf = {
# Large `checkpoint_distance` effectively disables size
# based checkpointing.
"checkpoint_distance": f"{2 * 1024 ** 3}",
"checkpoint_timeout": f"{CHECKPOINT_TIMEOUT_SECONDS}s",
"compaction_period": "1s",
}
env = neon_env_builder.init_configs()
env.start()
last_flush_lsns = asyncio.run(workload(env, tenant_conf, TIMELINE_COUNT, ENTRIES_PER_TIMELINE))
wait_until_pageserver_is_caught_up(env, last_flush_lsns)
# We didn't write enough data to trigger a size-based checkpoint: we should see dirty data.
wait_until(10, 1, lambda: assert_dirty_bytes_nonzero(env)) # type: ignore
ps_http_client = env.pageserver.http_client()
total_wal_ingested_before_restart = wait_for_wal_ingest_metric(ps_http_client)
# Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed,
# such that there are zero bytes of ephemeral layer left on the pageserver
log.info("Waiting for background checkpoints...")
wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(env, 0)) # type: ignore
# Zero ephemeral layer bytes does not imply that all the frozen layers were uploaded: they
# must be uploaded to remain visible to the pageserver after restart.
wait_until_pageserver_has_uploaded(env, last_flush_lsns)
env.pageserver.restart(immediate=immediate_shutdown)
wait_until_pageserver_is_caught_up(env, last_flush_lsns)
# Catching up with WAL ingest should have resulted in zero bytes of ephemeral layers, since
# we froze, flushed and uploaded everything before restarting. There can be no more WAL writes
# because we shut down compute endpoints before flushing.
assert get_dirty_bytes(env) == 0
total_wal_ingested_after_restart = wait_for_wal_ingest_metric(ps_http_client)
log.info(f"WAL ingested before restart: {total_wal_ingested_before_restart}")
log.info(f"WAL ingested after restart: {total_wal_ingested_after_restart}")
assert total_wal_ingested_after_restart == 0
def test_idle_checkpoints(neon_env_builder: NeonEnvBuilder):
"""
Test that `checkpoint_timeout` is enforced even if there is no safekeeper input.
"""
tenant_conf = {
# Large `checkpoint_distance` effectively disables size
# based checkpointing.
"checkpoint_distance": f"{2 * 1024 ** 3}",
"checkpoint_timeout": f"{CHECKPOINT_TIMEOUT_SECONDS}s",
"compaction_period": "1s",
}
env = neon_env_builder.init_configs()
env.start()
last_flush_lsns = asyncio.run(workload(env, tenant_conf, TIMELINE_COUNT, ENTRIES_PER_TIMELINE))
wait_until_pageserver_is_caught_up(env, last_flush_lsns)
# We didn't write enough data to trigger a size-based checkpoint: we should see dirty data.
wait_until(10, 1, lambda: assert_dirty_bytes_nonzero(env)) # type: ignore
# Stop the safekeepers, so that we cannot have any more WAL receiver connections
for sk in env.safekeepers:
sk.stop()
# We should have got here fast enough that we didn't hit the background interval yet,
# and the teardown of SK connections shouldn't prompt any layer freezing.
assert get_dirty_bytes(env) > 0
# Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed,
# such that there are zero bytes of ephemeral layer left on the pageserver
log.info("Waiting for background checkpoints...")
wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(env, 0)) # type: ignore
# The code below verifies that we do not flush on the first write
# after an idle period longer than the checkpoint timeout.
# Sit quietly for longer than the checkpoint timeout
time.sleep(CHECKPOINT_TIMEOUT_SECONDS + CHECKPOINT_TIMEOUT_SECONDS / 2)
# Restart the safekeepers and write a bit of extra data into one tenant
for sk in env.safekeepers:
sk.start()
tenant_with_extra_writes = last_flush_lsns[0][0]
asyncio.run(
run_worker_for_tenant(env, 5, tenant_with_extra_writes, offset=ENTRIES_PER_TIMELINE)
)
dirty_after_write = wait_until(10, 1, lambda: assert_dirty_bytes_nonzero(env)) # type: ignore
# We shouldn't flush since we've just opened a new layer
waited_for = 0
while waited_for < CHECKPOINT_TIMEOUT_SECONDS // 4:
time.sleep(5)
waited_for += 5
assert get_dirty_bytes(env) >= dirty_after_write
@pytest.mark.skipif(
# We have to use at least ~100MB of data to hit the lowest limit we can configure, which is
# prohibitively slow in debug mode
os.getenv("BUILD_TYPE") == "debug",
reason="Avoid running bulkier ingest tests in debug mode",
)
def test_total_size_limit(neon_env_builder: NeonEnvBuilder):
"""
Test that checkpoints are done based on total ephemeral layer size, even if no one timeline is
individually exceeding checkpoint thresholds.
"""
system_memory = psutil.virtual_memory().total
# The smallest total size limit we can configure is 1/1024th of the system memory (e.g. 128MB on
# a system with 128GB of RAM). We will then write enough data to violate this limit.
max_dirty_data = 128 * 1024 * 1024
ephemeral_bytes_per_memory_kb = (max_dirty_data * 1024) // system_memory
assert ephemeral_bytes_per_memory_kb > 0
neon_env_builder.pageserver_config_override = f"""
ephemeral_bytes_per_memory_kb={ephemeral_bytes_per_memory_kb}
"""
compaction_period_s = 10
checkpoint_distance = 1024**3
tenant_conf = {
# Large space + time thresholds: effectively disable these limits
"checkpoint_distance": f"{checkpoint_distance}",
"checkpoint_timeout": "3600s",
"compaction_period": f"{compaction_period_s}s",
}
env = neon_env_builder.init_configs()
env.start()
timeline_count = 10
# This is about 2MiB of data per timeline
entries_per_timeline = 100_000
last_flush_lsns = asyncio.run(workload(env, tenant_conf, timeline_count, entries_per_timeline))
wait_until_pageserver_is_caught_up(env, last_flush_lsns)
total_bytes_ingested = 0
for tenant, timeline, last_flush_lsn in last_flush_lsns:
http_client = env.pageserver.http_client()
initdb_lsn = Lsn(http_client.timeline_detail(tenant, timeline)["initdb_lsn"])
this_timeline_ingested = last_flush_lsn - initdb_lsn
assert (
this_timeline_ingested < checkpoint_distance * 0.8
), "this test is supposed to fill InMemoryLayer"
total_bytes_ingested += this_timeline_ingested
log.info(f"Ingested {total_bytes_ingested} bytes since initdb (vs max dirty {max_dirty_data})")
assert total_bytes_ingested > max_dirty_data
# Expected end state: the total physical size of all the tenants is in excess of the max dirty
# data, but the total amount of dirty data is less than the limit: this demonstrates that we
# have exceeded the threshold but then rolled layers in response
def get_total_historic_layers():
total_ephemeral_layers = 0
total_historic_bytes = 0
for tenant, timeline, _last_flush_lsn in last_flush_lsns:
http_client = env.pageserver.http_client()
initdb_lsn = Lsn(http_client.timeline_detail(tenant, timeline)["initdb_lsn"])
layer_map = http_client.layer_map_info(tenant, timeline)
total_historic_bytes += sum(
layer.layer_file_size
for layer in layer_map.historic_layers
if Lsn(layer.lsn_start) > initdb_lsn
)
total_ephemeral_layers += len(layer_map.in_memory_layers)
log.info(
f"Total historic layer bytes: {total_historic_bytes} ({total_ephemeral_layers} ephemeral layers)"
)
return total_historic_bytes
def assert_bytes_rolled():
assert total_bytes_ingested - get_total_historic_layers() <= max_dirty_data
# Wait until enough layers have rolled that the amount of dirty data is under the threshold.
# We do this indirectly via layer maps, rather than the dirty bytes metric, to avoid false-passing
# if that metric isn't updated quickly enough to reflect the dirty bytes exceeding the limit.
wait_until(compaction_period_s * 2, 1, assert_bytes_rolled)
# The end state should also have the reported metric under the limit
def assert_dirty_data_limited():
dirty_bytes = get_dirty_bytes(env)
assert dirty_bytes < max_dirty_data
wait_until(compaction_period_s * 2, 1, lambda: assert_dirty_data_limited()) # type: ignore