diff --git a/.circleci/ansible/.gitignore b/.circleci/ansible/.gitignore new file mode 100644 index 0000000000..14a1c155ae --- /dev/null +++ b/.circleci/ansible/.gitignore @@ -0,0 +1,2 @@ +zenith_install.tar.gz +.zenith_current_version diff --git a/.circleci/ansible/deploy.yaml b/.circleci/ansible/deploy.yaml index 1f43adf950..020a852a00 100644 --- a/.circleci/ansible/deploy.yaml +++ b/.circleci/ansible/deploy.yaml @@ -1,14 +1,11 @@ - name: Upload Zenith binaries - hosts: pageservers:safekeepers + hosts: storage gather_facts: False remote_user: admin - vars: - force_deploy: false tasks: - name: get latest version of Zenith binaries - ignore_errors: true register: current_version_file set_fact: current_version: "{{ lookup('file', '.zenith_current_version') | trim }}" @@ -16,48 +13,13 @@ - pageserver - safekeeper - - name: set zero value for current_version - when: current_version_file is failed - set_fact: - current_version: "0" - tags: - - pageserver - - safekeeper - - - name: get deployed version from content of remote file - ignore_errors: true - ansible.builtin.slurp: - src: /usr/local/.zenith_current_version - register: remote_version_file - tags: - - pageserver - - safekeeper - - - name: decode remote file content - when: remote_version_file is succeeded - set_fact: - remote_version: "{{ remote_version_file['content'] | b64decode | trim }}" - tags: - - pageserver - - safekeeper - - - name: set zero value for remote_version - when: remote_version_file is failed - set_fact: - remote_version: "0" - tags: - - pageserver - - safekeeper - - name: inform about versions - debug: msg="Version to deploy - {{ current_version }}, version on storage node - {{ remote_version }}" + debug: msg="Version to deploy - {{ current_version }}" tags: - pageserver - safekeeper - - name: upload and extract Zenith binaries to /usr/local - when: current_version > remote_version or force_deploy ansible.builtin.unarchive: owner: root group: root @@ -74,14 +36,24 @@ hosts: pageservers gather_facts: False remote_user: admin - vars: - force_deploy: false tasks: + + - name: upload init script + when: console_mgmt_base_url is defined + ansible.builtin.template: + src: scripts/init_pageserver.sh + dest: /tmp/init_pageserver.sh + owner: root + group: root + mode: '0755' + become: true + tags: + - pageserver + - name: init pageserver - when: current_version > remote_version or force_deploy shell: - cmd: sudo -u pageserver /usr/local/bin/pageserver -c "pg_distrib_dir='/usr/local'" --init -D /storage/pageserver/data + cmd: /tmp/init_pageserver.sh args: creates: "/storage/pageserver/data/tenants" environment: @@ -107,7 +79,6 @@ # - pageserver - name: upload systemd service definition - when: current_version > remote_version or force_deploy ansible.builtin.template: src: systemd/pageserver.service dest: /etc/systemd/system/pageserver.service @@ -119,7 +90,6 @@ - pageserver - name: start systemd service - when: current_version > remote_version or force_deploy ansible.builtin.systemd: daemon_reload: yes name: pageserver @@ -130,7 +100,7 @@ - pageserver - name: post version to console - when: (current_version > remote_version or force_deploy) and console_mgmt_base_url is defined + when: console_mgmt_base_url is defined shell: cmd: | INSTANCE_ID=$(curl -s http://169.254.169.254/latest/meta-data/instance-id) @@ -142,22 +112,18 @@ hosts: safekeepers gather_facts: False remote_user: admin - vars: - force_deploy: false tasks: # in the future safekeepers should discover pageservers byself # but currently use first pageserver that was discovered - name: set first pageserver var for safekeepers - when: current_version > remote_version or force_deploy set_fact: first_pageserver: "{{ hostvars[groups['pageservers'][0]]['inventory_hostname'] }}" tags: - safekeeper - name: upload systemd service definition - when: current_version > remote_version or force_deploy ansible.builtin.template: src: systemd/safekeeper.service dest: /etc/systemd/system/safekeeper.service @@ -169,7 +135,6 @@ - safekeeper - name: start systemd service - when: current_version > remote_version or force_deploy ansible.builtin.systemd: daemon_reload: yes name: safekeeper @@ -180,7 +145,7 @@ - safekeeper - name: post version to console - when: (current_version > remote_version or force_deploy) and console_mgmt_base_url is defined + when: console_mgmt_base_url is defined shell: cmd: | INSTANCE_ID=$(curl -s http://169.254.169.254/latest/meta-data/instance-id) diff --git a/.circleci/ansible/production.hosts b/.circleci/ansible/production.hosts index 3a0543f39a..13224b7cf5 100644 --- a/.circleci/ansible/production.hosts +++ b/.circleci/ansible/production.hosts @@ -1,7 +1,16 @@ [pageservers] -zenith-1-ps-1 bucket_name=zenith-storage-oregon bucket_region=us-west-2 +zenith-1-ps-1 console_region_id=1 [safekeepers] -zenith-1-sk-1 -zenith-1-sk-2 -zenith-1-sk-3 +zenith-1-sk-1 console_region_id=1 +zenith-1-sk-2 console_region_id=1 +zenith-1-sk-3 console_region_id=1 + +[storage:children] +pageservers +safekeepers + +[storage:vars] +console_mgmt_base_url = http://console-release.local +bucket_name = zenith-storage-oregon +bucket_region = us-west-2 diff --git a/.circleci/ansible/scripts/init_pageserver.sh b/.circleci/ansible/scripts/init_pageserver.sh new file mode 100644 index 0000000000..1cbdd0db94 --- /dev/null +++ b/.circleci/ansible/scripts/init_pageserver.sh @@ -0,0 +1,30 @@ +#!/bin/sh + +# get instance id from meta-data service +INSTANCE_ID=$(curl -s http://169.254.169.254/latest/meta-data/instance-id) + +# store fqdn hostname in var +HOST=$(hostname -f) + + +cat < Page ID + + ++---+ +| | Layer file ++---+ +``` + + +# Memtable + +When new WAL arrives, it is first put into the Memtable. Despite the +name, the Memtable is not a purely in-memory data structure. It can +spill to a temporary file on disk if the system is low on memory, and +is accessed through a buffer cache. + +If the page server crashes, the Memtable is lost. It is rebuilt by +processing again the WAL that's newer than the latest layer in L0. + +The size of the Memtable is configured by the "checkpoint distance" +setting. Because anything that hasn't been flushed to disk and +uploaded to S3 yet needs to be kept in the safekeeper, the "checkpoint +distance" also determines the amount of WAL that needs to kept in the +safekeeper. + +# L0 + +When the Memtable fills up, it is written out to a new file in L0. The +files are immutable; when a file is created, it is never +modified. Each file in L0 is roughly 1 GB in size (*). Like the +Memtable, each file in L0 covers the whole key range. + +When enough files have been accumulated in L0, compaction +starts. Compaction processes all the files in L0 and reshuffles the +data to create a new set of files in L1. + + +(*) except in corner cases like if we want to shut down the page +server and want to flush out the memtable to disk even though it's not +full yet. + + +# L1 + +L1 consists of ~ 1 GB files like L0. But each file covers only part of +the overall key space, and a larger range of LSNs. This speeds up +searches. When you're looking for a given page, you need to check all +the files in L0, to see if they contain a page version for the requested +page. But in L1, you only need to check the files whose key range covers +the requested page. This is particularly important at cold start, when +checking a file means downloading it from S3. + +Partitioning by key range also helps with garbage collection. If only a +part of the database is updated, we will accumulate more files for +the hot part in L1, and old files can be removed without affecting the +cold part. + + +# Image layers + +So far, we've only talked about delta layers. In addition to the delta +layers, we create image layers, when "enough" WAL has been accumulated +for some part of the database. Each image layer covers a 1 GB range of +key space. It contains images of the pages at a single LSN, a snapshot +if you will. + +The exact heuristic for what "enough" means is not clear yet. Maybe +create a new image layer when 10 GB of WAL has been accumulated for a +1 GB segment. + +The image layers limit the number of layers that a search needs to +check. That put a cap on read latency, and it also allows garbage +collecting layers that are older than the GC horizon. + + +# Partitioning scheme + +When compaction happens and creates a new set of files in L1, how do +we partition the data into the files? + +- Goal is that each file is ~ 1 GB in size +- Try to match partition boundaries at relation boundaries. (See [1] + for how PebblesDB does this, and for why that's important) +- Greedy algorithm + +# Additional Reading + +[1] Paper on PebblesDB and how it does partitioning. +https://www.cs.utexas.edu/~rak/papers/sosp17-pebblesdb.pdf diff --git a/docs/settings.md b/docs/settings.md index 571cfba8df..69aadc602f 100644 --- a/docs/settings.md +++ b/docs/settings.md @@ -68,11 +68,11 @@ S3. The unit is # of bytes. -#### checkpoint_period +#### compaction_period -The pageserver checks whether `checkpoint_distance` has been reached -every `checkpoint_period` seconds. Default is 1 s, which should be -fine. +Every `compaction_period` seconds, the page server checks if +maintenance operations, like compaction, are needed on the layer +files. Default is 1 s, which should be fine. #### gc_horizon diff --git a/pageserver/Cargo.toml b/pageserver/Cargo.toml index 46e6e2a8f1..de22d0dd77 100644 --- a/pageserver/Cargo.toml +++ b/pageserver/Cargo.toml @@ -12,6 +12,7 @@ bytes = { version = "1.0.1", features = ['serde'] } byteorder = "1.4.3" futures = "0.3.13" hyper = "0.14" +itertools = "0.10.3" lazy_static = "1.4.0" log = "0.4.14" clap = "3.0" diff --git a/pageserver/src/basebackup.rs b/pageserver/src/basebackup.rs index 5711f1807d..e2a56f17d6 100644 --- a/pageserver/src/basebackup.rs +++ b/pageserver/src/basebackup.rs @@ -20,8 +20,9 @@ use std::sync::Arc; use std::time::SystemTime; use tar::{Builder, EntryType, Header}; -use crate::relish::*; +use crate::reltag::SlruKind; use crate::repository::Timeline; +use crate::DatadirTimelineImpl; use postgres_ffi::xlog_utils::*; use postgres_ffi::*; use zenith_utils::lsn::Lsn; @@ -31,7 +32,7 @@ use zenith_utils::lsn::Lsn; /// used for constructing tarball. pub struct Basebackup<'a> { ar: Builder<&'a mut dyn Write>, - timeline: &'a Arc, + timeline: &'a Arc, pub lsn: Lsn, prev_record_lsn: Lsn, } @@ -46,7 +47,7 @@ pub struct Basebackup<'a> { impl<'a> Basebackup<'a> { pub fn new( write: &'a mut dyn Write, - timeline: &'a Arc, + timeline: &'a Arc, req_lsn: Option, ) -> Result> { // Compute postgres doesn't have any previous WAL files, but the first @@ -64,13 +65,13 @@ impl<'a> Basebackup<'a> { // prev_lsn to Lsn(0) if we cannot provide the correct value. let (backup_prev, backup_lsn) = if let Some(req_lsn) = req_lsn { // Backup was requested at a particular LSN. Wait for it to arrive. - timeline.wait_lsn(req_lsn)?; + timeline.tline.wait_lsn(req_lsn)?; // If the requested point is the end of the timeline, we can // provide prev_lsn. (get_last_record_rlsn() might return it as // zero, though, if no WAL has been generated on this timeline // yet.) - let end_of_timeline = timeline.get_last_record_rlsn(); + let end_of_timeline = timeline.tline.get_last_record_rlsn(); if req_lsn == end_of_timeline.last { (end_of_timeline.prev, req_lsn) } else { @@ -78,7 +79,7 @@ impl<'a> Basebackup<'a> { } } else { // Backup was requested at end of the timeline. - let end_of_timeline = timeline.get_last_record_rlsn(); + let end_of_timeline = timeline.tline.get_last_record_rlsn(); (end_of_timeline.prev, end_of_timeline.last) }; @@ -115,21 +116,24 @@ impl<'a> Basebackup<'a> { } // Gather non-relational files from object storage pages. - for obj in self.timeline.list_nonrels(self.lsn)? { - match obj { - RelishTag::Slru { slru, segno } => { - self.add_slru_segment(slru, segno)?; - } - RelishTag::FileNodeMap { spcnode, dbnode } => { - self.add_relmap_file(spcnode, dbnode)?; - } - RelishTag::TwoPhase { xid } => { - self.add_twophase_file(xid)?; - } - _ => {} + for kind in [ + SlruKind::Clog, + SlruKind::MultiXactOffsets, + SlruKind::MultiXactMembers, + ] { + for segno in self.timeline.list_slru_segments(kind, self.lsn)? { + self.add_slru_segment(kind, segno)?; } } + // Create tablespace directories + for ((spcnode, dbnode), has_relmap_file) in self.timeline.list_dbdirs(self.lsn)? { + self.add_dbdir(spcnode, dbnode, has_relmap_file)?; + } + for xid in self.timeline.list_twophase_files(self.lsn)? { + self.add_twophase_file(xid)?; + } + // Generate pg_control and bootstrap WAL segment. self.add_pgcontrol_file()?; self.ar.finish()?; @@ -141,28 +145,14 @@ impl<'a> Basebackup<'a> { // Generate SLRU segment files from repository. // fn add_slru_segment(&mut self, slru: SlruKind, segno: u32) -> anyhow::Result<()> { - let seg_size = self - .timeline - .get_relish_size(RelishTag::Slru { slru, segno }, self.lsn)?; - - let nblocks = match seg_size { - Some(seg_size) => seg_size, - None => { - trace!( - "SLRU segment {}/{:>04X} was truncated", - slru.to_str(), - segno - ); - return Ok(()); - } - }; + let nblocks = self.timeline.get_slru_segment_size(slru, segno, self.lsn)?; let mut slru_buf: Vec = Vec::with_capacity(nblocks as usize * pg_constants::BLCKSZ as usize); for blknum in 0..nblocks { - let img = - self.timeline - .get_page_at_lsn(RelishTag::Slru { slru, segno }, blknum, self.lsn)?; + let img = self + .timeline + .get_slru_page_at_lsn(slru, segno, blknum, self.lsn)?; ensure!(img.len() == pg_constants::BLCKSZ as usize); slru_buf.extend_from_slice(&img); @@ -177,16 +167,26 @@ impl<'a> Basebackup<'a> { } // - // Extract pg_filenode.map files from repository - // Along with them also send PG_VERSION for each database. + // Include database/tablespace directories. // - fn add_relmap_file(&mut self, spcnode: u32, dbnode: u32) -> anyhow::Result<()> { - let img = self.timeline.get_page_at_lsn( - RelishTag::FileNodeMap { spcnode, dbnode }, - 0, - self.lsn, - )?; - let path = if spcnode == pg_constants::GLOBALTABLESPACE_OID { + // Each directory contains a PG_VERSION file, and the default database + // directories also contain pg_filenode.map files. + // + fn add_dbdir( + &mut self, + spcnode: u32, + dbnode: u32, + has_relmap_file: bool, + ) -> anyhow::Result<()> { + let relmap_img = if has_relmap_file { + let img = self.timeline.get_relmap_file(spcnode, dbnode, self.lsn)?; + ensure!(img.len() == 512); + Some(img) + } else { + None + }; + + if spcnode == pg_constants::GLOBALTABLESPACE_OID { let version_bytes = pg_constants::PG_MAJORVERSION.as_bytes(); let header = new_tar_header("PG_VERSION", version_bytes.len() as u64)?; self.ar.append(&header, version_bytes)?; @@ -194,8 +194,32 @@ impl<'a> Basebackup<'a> { let header = new_tar_header("global/PG_VERSION", version_bytes.len() as u64)?; self.ar.append(&header, version_bytes)?; - String::from("global/pg_filenode.map") // filenode map for global tablespace + if let Some(img) = relmap_img { + // filenode map for global tablespace + let header = new_tar_header("global/pg_filenode.map", img.len() as u64)?; + self.ar.append(&header, &img[..])?; + } else { + warn!("global/pg_filenode.map is missing"); + } } else { + // User defined tablespaces are not supported. However, as + // a special case, if a tablespace/db directory is + // completely empty, we can leave it out altogether. This + // makes taking a base backup after the 'tablespace' + // regression test pass, because the test drops the + // created tablespaces after the tests. + // + // FIXME: this wouldn't be necessary, if we handled + // XLOG_TBLSPC_DROP records. But we probably should just + // throw an error on CREATE TABLESPACE in the first place. + if !has_relmap_file + && self + .timeline + .list_rels(spcnode, dbnode, self.lsn)? + .is_empty() + { + return Ok(()); + } // User defined tablespaces are not supported ensure!(spcnode == pg_constants::DEFAULTTABLESPACE_OID); @@ -204,16 +228,17 @@ impl<'a> Basebackup<'a> { let header = new_tar_header_dir(&path)?; self.ar.append(&header, &mut io::empty())?; - let dst_path = format!("base/{}/PG_VERSION", dbnode); - let version_bytes = pg_constants::PG_MAJORVERSION.as_bytes(); - let header = new_tar_header(&dst_path, version_bytes.len() as u64)?; - self.ar.append(&header, version_bytes)?; + if let Some(img) = relmap_img { + let dst_path = format!("base/{}/PG_VERSION", dbnode); + let version_bytes = pg_constants::PG_MAJORVERSION.as_bytes(); + let header = new_tar_header(&dst_path, version_bytes.len() as u64)?; + self.ar.append(&header, version_bytes)?; - format!("base/{}/pg_filenode.map", dbnode) + let relmap_path = format!("base/{}/pg_filenode.map", dbnode); + let header = new_tar_header(&relmap_path, img.len() as u64)?; + self.ar.append(&header, &img[..])?; + } }; - ensure!(img.len() == 512); - let header = new_tar_header(&path, img.len() as u64)?; - self.ar.append(&header, &img[..])?; Ok(()) } @@ -221,9 +246,7 @@ impl<'a> Basebackup<'a> { // Extract twophase state files // fn add_twophase_file(&mut self, xid: TransactionId) -> anyhow::Result<()> { - let img = self - .timeline - .get_page_at_lsn(RelishTag::TwoPhase { xid }, 0, self.lsn)?; + let img = self.timeline.get_twophase_file(xid, self.lsn)?; let mut buf = BytesMut::new(); buf.extend_from_slice(&img[..]); @@ -243,11 +266,11 @@ impl<'a> Basebackup<'a> { fn add_pgcontrol_file(&mut self) -> anyhow::Result<()> { let checkpoint_bytes = self .timeline - .get_page_at_lsn(RelishTag::Checkpoint, 0, self.lsn) + .get_checkpoint(self.lsn) .context("failed to get checkpoint bytes")?; let pg_control_bytes = self .timeline - .get_page_at_lsn(RelishTag::ControlFile, 0, self.lsn) + .get_control_file(self.lsn) .context("failed get control bytes")?; let mut pg_control = ControlFileData::decode(&pg_control_bytes)?; let mut checkpoint = CheckPoint::decode(&checkpoint_bytes)?; @@ -268,7 +291,7 @@ impl<'a> Basebackup<'a> { // add zenith.signal file let mut zenith_signal = String::new(); if self.prev_record_lsn == Lsn(0) { - if self.lsn == self.timeline.get_ancestor_lsn() { + if self.lsn == self.timeline.tline.get_ancestor_lsn() { write!(zenith_signal, "PREV LSN: none")?; } else { write!(zenith_signal, "PREV LSN: invalid")?; diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index e217806147..0af96cff66 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -20,7 +20,7 @@ use pageserver::{ config::{defaults::*, PageServerConf}, http, page_cache, page_service, remote_storage::{self, SyncStartupData}, - repository::TimelineSyncStatusUpdate, + repository::{Repository, TimelineSyncStatusUpdate}, tenant_mgr, thread_mgr, thread_mgr::ThreadKind, timelines, virtual_file, LOG_FILE_NAME, diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index dc85c83c17..0fdfb4ceed 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -31,7 +31,8 @@ pub mod defaults { // would be more appropriate. But a low value forces the code to be exercised more, // which is good for now to trigger bugs. pub const DEFAULT_CHECKPOINT_DISTANCE: u64 = 256 * 1024 * 1024; - pub const DEFAULT_CHECKPOINT_PERIOD: &str = "1 s"; + + pub const DEFAULT_COMPACTION_PERIOD: &str = "1 s"; pub const DEFAULT_GC_HORIZON: u64 = 64 * 1024 * 1024; pub const DEFAULT_GC_PERIOD: &str = "100 s"; @@ -57,7 +58,7 @@ pub mod defaults { #listen_http_addr = '{DEFAULT_HTTP_LISTEN_ADDR}' #checkpoint_distance = {DEFAULT_CHECKPOINT_DISTANCE} # in bytes -#checkpoint_period = '{DEFAULT_CHECKPOINT_PERIOD}' +#compaction_period = '{DEFAULT_COMPACTION_PERIOD}' #gc_period = '{DEFAULT_GC_PERIOD}' #gc_horizon = {DEFAULT_GC_HORIZON} @@ -91,7 +92,9 @@ pub struct PageServerConf { // This puts a backstop on how much WAL needs to be re-digested if the // page server crashes. pub checkpoint_distance: u64, - pub checkpoint_period: Duration, + + // How often to check if there's compaction work to be done. + pub compaction_period: Duration, pub gc_horizon: u64, pub gc_period: Duration, @@ -145,7 +148,8 @@ struct PageServerConfigBuilder { listen_http_addr: BuilderValue, checkpoint_distance: BuilderValue, - checkpoint_period: BuilderValue, + + compaction_period: BuilderValue, gc_horizon: BuilderValue, gc_period: BuilderValue, @@ -179,8 +183,8 @@ impl Default for PageServerConfigBuilder { listen_pg_addr: Set(DEFAULT_PG_LISTEN_ADDR.to_string()), listen_http_addr: Set(DEFAULT_HTTP_LISTEN_ADDR.to_string()), checkpoint_distance: Set(DEFAULT_CHECKPOINT_DISTANCE), - checkpoint_period: Set(humantime::parse_duration(DEFAULT_CHECKPOINT_PERIOD) - .expect("cannot parse default checkpoint period")), + compaction_period: Set(humantime::parse_duration(DEFAULT_COMPACTION_PERIOD) + .expect("cannot parse default compaction period")), gc_horizon: Set(DEFAULT_GC_HORIZON), gc_period: Set(humantime::parse_duration(DEFAULT_GC_PERIOD) .expect("cannot parse default gc period")), @@ -216,8 +220,8 @@ impl PageServerConfigBuilder { self.checkpoint_distance = BuilderValue::Set(checkpoint_distance) } - pub fn checkpoint_period(&mut self, checkpoint_period: Duration) { - self.checkpoint_period = BuilderValue::Set(checkpoint_period) + pub fn compaction_period(&mut self, compaction_period: Duration) { + self.compaction_period = BuilderValue::Set(compaction_period) } pub fn gc_horizon(&mut self, gc_horizon: u64) { @@ -286,9 +290,9 @@ impl PageServerConfigBuilder { checkpoint_distance: self .checkpoint_distance .ok_or(anyhow::anyhow!("missing checkpoint_distance"))?, - checkpoint_period: self - .checkpoint_period - .ok_or(anyhow::anyhow!("missing checkpoint_period"))?, + compaction_period: self + .compaction_period + .ok_or(anyhow::anyhow!("missing compaction_period"))?, gc_horizon: self .gc_horizon .ok_or(anyhow::anyhow!("missing gc_horizon"))?, @@ -337,10 +341,10 @@ pub struct RemoteStorageConfig { #[derive(Debug, Clone, PartialEq, Eq)] pub enum RemoteStorageKind { /// Storage based on local file system. - /// Specify a root folder to place all stored relish data into. + /// Specify a root folder to place all stored files into. LocalFs(PathBuf), - /// AWS S3 based storage, storing all relishes into the root - /// of the S3 bucket from the config. + /// AWS S3 based storage, storing all files in the S3 bucket + /// specified by the config AwsS3(S3Config), } @@ -425,7 +429,7 @@ impl PageServerConf { "listen_pg_addr" => builder.listen_pg_addr(parse_toml_string(key, item)?), "listen_http_addr" => builder.listen_http_addr(parse_toml_string(key, item)?), "checkpoint_distance" => builder.checkpoint_distance(parse_toml_u64(key, item)?), - "checkpoint_period" => builder.checkpoint_period(parse_toml_duration(key, item)?), + "compaction_period" => builder.compaction_period(parse_toml_duration(key, item)?), "gc_horizon" => builder.gc_horizon(parse_toml_u64(key, item)?), "gc_period" => builder.gc_period(parse_toml_duration(key, item)?), "wait_lsn_timeout" => builder.wait_lsn_timeout(parse_toml_duration(key, item)?), @@ -561,7 +565,7 @@ impl PageServerConf { PageServerConf { id: ZNodeId(0), checkpoint_distance: defaults::DEFAULT_CHECKPOINT_DISTANCE, - checkpoint_period: Duration::from_secs(10), + compaction_period: Duration::from_secs(10), gc_horizon: defaults::DEFAULT_GC_HORIZON, gc_period: Duration::from_secs(10), wait_lsn_timeout: Duration::from_secs(60), @@ -631,7 +635,8 @@ listen_pg_addr = '127.0.0.1:64000' listen_http_addr = '127.0.0.1:9898' checkpoint_distance = 111 # in bytes -checkpoint_period = '111 s' + +compaction_period = '111 s' gc_period = '222 s' gc_horizon = 222 @@ -668,7 +673,7 @@ id = 10 listen_pg_addr: defaults::DEFAULT_PG_LISTEN_ADDR.to_string(), listen_http_addr: defaults::DEFAULT_HTTP_LISTEN_ADDR.to_string(), checkpoint_distance: defaults::DEFAULT_CHECKPOINT_DISTANCE, - checkpoint_period: humantime::parse_duration(defaults::DEFAULT_CHECKPOINT_PERIOD)?, + compaction_period: humantime::parse_duration(defaults::DEFAULT_COMPACTION_PERIOD)?, gc_horizon: defaults::DEFAULT_GC_HORIZON, gc_period: humantime::parse_duration(defaults::DEFAULT_GC_PERIOD)?, wait_lsn_timeout: humantime::parse_duration(defaults::DEFAULT_WAIT_LSN_TIMEOUT)?, @@ -712,7 +717,7 @@ id = 10 listen_pg_addr: "127.0.0.1:64000".to_string(), listen_http_addr: "127.0.0.1:9898".to_string(), checkpoint_distance: 111, - checkpoint_period: Duration::from_secs(111), + compaction_period: Duration::from_secs(111), gc_horizon: 222, gc_period: Duration::from_secs(222), wait_lsn_timeout: Duration::from_secs(111), diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 13e79f8f55..82e818a47b 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -22,6 +22,7 @@ use super::models::{ StatusResponse, TenantCreateRequest, TenantCreateResponse, TimelineCreateRequest, }; use crate::remote_storage::{schedule_timeline_download, RemoteIndex}; +use crate::repository::Repository; use crate::timelines::{LocalTimelineInfo, RemoteTimelineInfo, TimelineInfo}; use crate::{config::PageServerConf, tenant_mgr, timelines, ZTenantId}; @@ -162,8 +163,11 @@ async fn timeline_detail_handler(request: Request) -> Result( path: &Path, - writer: &dyn TimelineWriter, + tline: &mut DatadirTimeline, lsn: Lsn, ) -> Result<()> { let mut pg_control: Option = None; + let mut modification = tline.begin_modification(lsn); + modification.init_empty()?; + // Scan 'global' + let mut relfiles: Vec = Vec::new(); for direntry in fs::read_dir(path.join("global"))? { let direntry = direntry?; match direntry.file_name().to_str() { None => continue, Some("pg_control") => { - pg_control = Some(import_control_file(writer, lsn, &direntry.path())?); + pg_control = Some(import_control_file(&mut modification, &direntry.path())?); + } + Some("pg_filenode.map") => { + import_relmap_file( + &mut modification, + pg_constants::GLOBALTABLESPACE_OID, + 0, + &direntry.path(), + )?; } - Some("pg_filenode.map") => import_nonrel_file( - writer, - lsn, - RelishTag::FileNodeMap { - spcnode: pg_constants::GLOBALTABLESPACE_OID, - dbnode: 0, - }, - &direntry.path(), - )?, - // Load any relation files into the page server - _ => import_relfile( - &direntry.path(), - writer, - lsn, - pg_constants::GLOBALTABLESPACE_OID, - 0, - )?, + // Load any relation files into the page server (but only after the other files) + _ => relfiles.push(direntry.path()), } } + for relfile in relfiles { + import_relfile( + &mut modification, + &relfile, + pg_constants::GLOBALTABLESPACE_OID, + 0, + )?; + } // Scan 'base'. It contains database dirs, the database OID is the filename. // E.g. 'base/12345', where 12345 is the database OID. @@ -76,54 +82,56 @@ pub fn import_timeline_from_postgres_datadir( let dboid = direntry.file_name().to_string_lossy().parse::()?; + let mut relfiles: Vec = Vec::new(); for direntry in fs::read_dir(direntry.path())? { let direntry = direntry?; match direntry.file_name().to_str() { None => continue, - Some("PG_VERSION") => continue, - Some("pg_filenode.map") => import_nonrel_file( - writer, - lsn, - RelishTag::FileNodeMap { - spcnode: pg_constants::DEFAULTTABLESPACE_OID, - dbnode: dboid, - }, + Some("PG_VERSION") => { + //modification.put_dbdir_creation(pg_constants::DEFAULTTABLESPACE_OID, dboid)?; + } + Some("pg_filenode.map") => import_relmap_file( + &mut modification, + pg_constants::DEFAULTTABLESPACE_OID, + dboid, &direntry.path(), )?, // Load any relation files into the page server - _ => import_relfile( - &direntry.path(), - writer, - lsn, - pg_constants::DEFAULTTABLESPACE_OID, - dboid, - )?, + _ => relfiles.push(direntry.path()), } } + for relfile in relfiles { + import_relfile( + &mut modification, + &relfile, + pg_constants::DEFAULTTABLESPACE_OID, + dboid, + )?; + } } for entry in fs::read_dir(path.join("pg_xact"))? { let entry = entry?; - import_slru_file(writer, lsn, SlruKind::Clog, &entry.path())?; + import_slru_file(&mut modification, SlruKind::Clog, &entry.path())?; } for entry in fs::read_dir(path.join("pg_multixact").join("members"))? { let entry = entry?; - import_slru_file(writer, lsn, SlruKind::MultiXactMembers, &entry.path())?; + import_slru_file(&mut modification, SlruKind::MultiXactMembers, &entry.path())?; } for entry in fs::read_dir(path.join("pg_multixact").join("offsets"))? { let entry = entry?; - import_slru_file(writer, lsn, SlruKind::MultiXactOffsets, &entry.path())?; + import_slru_file(&mut modification, SlruKind::MultiXactOffsets, &entry.path())?; } for entry in fs::read_dir(path.join("pg_twophase"))? { let entry = entry?; let xid = u32::from_str_radix(&entry.path().to_string_lossy(), 16)?; - import_nonrel_file(writer, lsn, RelishTag::TwoPhase { xid }, &entry.path())?; + import_twophase_file(&mut modification, xid, &entry.path())?; } // TODO: Scan pg_tblspc // We're done importing all the data files. - writer.advance_last_record_lsn(lsn); + modification.commit()?; // We expect the Postgres server to be shut down cleanly. let pg_control = pg_control.context("pg_control file not found")?; @@ -141,7 +149,7 @@ pub fn import_timeline_from_postgres_datadir( // *after* the checkpoint record. And crucially, it initializes the 'prev_lsn'. import_wal( &path.join("pg_wal"), - writer, + tline, Lsn(pg_control.checkPointCopy.redo), lsn, )?; @@ -150,10 +158,9 @@ pub fn import_timeline_from_postgres_datadir( } // subroutine of import_timeline_from_postgres_datadir(), to load one relation file. -fn import_relfile( +fn import_relfile( + modification: &mut DatadirModification, path: &Path, - timeline: &dyn TimelineWriter, - lsn: Lsn, spcoid: Oid, dboid: Oid, ) -> anyhow::Result<()> { @@ -169,26 +176,35 @@ fn import_relfile( let mut file = File::open(path)?; let mut buf: [u8; 8192] = [0u8; 8192]; + let len = file.metadata().unwrap().len(); + ensure!(len % pg_constants::BLCKSZ as u64 == 0); + let nblocks = len / pg_constants::BLCKSZ as u64; + + if segno != 0 { + todo!(); + } + + let rel = RelTag { + spcnode: spcoid, + dbnode: dboid, + relnode, + forknum, + }; + modification.put_rel_creation(rel, nblocks as u32)?; + let mut blknum: u32 = segno * (1024 * 1024 * 1024 / pg_constants::BLCKSZ as u32); loop { let r = file.read_exact(&mut buf); match r { Ok(_) => { - let rel = RelTag { - spcnode: spcoid, - dbnode: dboid, - relnode, - forknum, - }; - let tag = RelishTag::Relation(rel); - timeline.put_page_image(tag, blknum, lsn, Bytes::copy_from_slice(&buf))?; + modification.put_rel_page_image(rel, blknum, Bytes::copy_from_slice(&buf))?; } // TODO: UnexpectedEof is expected Err(err) => match err.kind() { std::io::ErrorKind::UnexpectedEof => { // reached EOF. That's expected. - // FIXME: maybe check that we read the full length of the file? + ensure!(blknum == nblocks as u32, "unexpected EOF"); break; } _ => { @@ -202,16 +218,28 @@ fn import_relfile( Ok(()) } -/// -/// Import a "non-blocky" file into the repository -/// -/// This is used for small files like the control file, twophase files etc. that -/// are just slurped into the repository as one blob. -/// -fn import_nonrel_file( - timeline: &dyn TimelineWriter, - lsn: Lsn, - tag: RelishTag, +/// Import a relmapper (pg_filenode.map) file into the repository +fn import_relmap_file( + modification: &mut DatadirModification, + spcnode: Oid, + dbnode: Oid, + path: &Path, +) -> Result<()> { + let mut file = File::open(path)?; + let mut buffer = Vec::new(); + // read the whole file + file.read_to_end(&mut buffer)?; + + trace!("importing relmap file {}", path.display()); + + modification.put_relmap_file(spcnode, dbnode, Bytes::copy_from_slice(&buffer[..]))?; + Ok(()) +} + +/// Import a twophase state file (pg_twophase/) into the repository +fn import_twophase_file( + modification: &mut DatadirModification, + xid: TransactionId, path: &Path, ) -> Result<()> { let mut file = File::open(path)?; @@ -221,7 +249,7 @@ fn import_nonrel_file( trace!("importing non-rel file {}", path.display()); - timeline.put_page_image(tag, 0, lsn, Bytes::copy_from_slice(&buffer[..]))?; + modification.put_twophase_file(xid, Bytes::copy_from_slice(&buffer[..]))?; Ok(()) } @@ -230,9 +258,8 @@ fn import_nonrel_file( /// /// The control file is imported as is, but we also extract the checkpoint record /// from it and store it separated. -fn import_control_file( - timeline: &dyn TimelineWriter, - lsn: Lsn, +fn import_control_file( + modification: &mut DatadirModification, path: &Path, ) -> Result { let mut file = File::open(path)?; @@ -243,17 +270,12 @@ fn import_control_file( trace!("importing control file {}", path.display()); // Import it as ControlFile - timeline.put_page_image( - RelishTag::ControlFile, - 0, - lsn, - Bytes::copy_from_slice(&buffer[..]), - )?; + modification.put_control_file(Bytes::copy_from_slice(&buffer[..]))?; // Extract the checkpoint record and import it separately. let pg_control = ControlFileData::decode(&buffer)?; let checkpoint_bytes = pg_control.checkPointCopy.encode(); - timeline.put_page_image(RelishTag::Checkpoint, 0, lsn, checkpoint_bytes)?; + modification.put_checkpoint(checkpoint_bytes)?; Ok(pg_control) } @@ -261,28 +283,34 @@ fn import_control_file( /// /// Import an SLRU segment file /// -fn import_slru_file( - timeline: &dyn TimelineWriter, - lsn: Lsn, +fn import_slru_file( + modification: &mut DatadirModification, slru: SlruKind, path: &Path, ) -> Result<()> { - // Does it look like an SLRU file? + trace!("importing slru file {}", path.display()); + let mut file = File::open(path)?; let mut buf: [u8; 8192] = [0u8; 8192]; let segno = u32::from_str_radix(&path.file_name().unwrap().to_string_lossy(), 16)?; - trace!("importing slru file {}", path.display()); + let len = file.metadata().unwrap().len(); + ensure!(len % pg_constants::BLCKSZ as u64 == 0); // we assume SLRU block size is the same as BLCKSZ + let nblocks = len / pg_constants::BLCKSZ as u64; + + ensure!(nblocks <= pg_constants::SLRU_PAGES_PER_SEGMENT as u64); + + modification.put_slru_segment_creation(slru, segno, nblocks as u32)?; let mut rpageno = 0; loop { let r = file.read_exact(&mut buf); match r { Ok(_) => { - timeline.put_page_image( - RelishTag::Slru { slru, segno }, + modification.put_slru_page_image( + slru, + segno, rpageno, - lsn, Bytes::copy_from_slice(&buf), )?; } @@ -291,7 +319,7 @@ fn import_slru_file( Err(err) => match err.kind() { std::io::ErrorKind::UnexpectedEof => { // reached EOF. That's expected. - // FIXME: maybe check that we read the full length of the file? + ensure!(rpageno == nblocks as u32, "unexpected EOF"); break; } _ => { @@ -300,8 +328,6 @@ fn import_slru_file( }, }; rpageno += 1; - - // TODO: Check that the file isn't unexpectedly large, not larger than SLRU_PAGES_PER_SEGMENT pages } Ok(()) @@ -309,9 +335,9 @@ fn import_slru_file( /// Scan PostgreSQL WAL files in given directory and load all records between /// 'startpoint' and 'endpoint' into the repository. -fn import_wal( +fn import_wal( walpath: &Path, - writer: &dyn TimelineWriter, + tline: &mut DatadirTimeline, startpoint: Lsn, endpoint: Lsn, ) -> Result<()> { @@ -321,7 +347,7 @@ fn import_wal( let mut offset = startpoint.segment_offset(pg_constants::WAL_SEGMENT_SIZE); let mut last_lsn = startpoint; - let mut walingest = WalIngest::new(writer.deref(), startpoint)?; + let mut walingest = WalIngest::new(tline, startpoint)?; while last_lsn <= endpoint { // FIXME: assume postgresql tli 1 for now @@ -354,7 +380,7 @@ fn import_wal( let mut nrecords = 0; while last_lsn <= endpoint { if let Some((lsn, recdata)) = waldecoder.poll_decode()? { - walingest.ingest_record(writer, recdata, lsn)?; + walingest.ingest_record(tline, recdata, lsn)?; last_lsn = lsn; nrecords += 1; diff --git a/pageserver/src/keyspace.rs b/pageserver/src/keyspace.rs new file mode 100644 index 0000000000..9973568b07 --- /dev/null +++ b/pageserver/src/keyspace.rs @@ -0,0 +1,134 @@ +use crate::repository::{key_range_size, singleton_range, Key}; +use postgres_ffi::pg_constants; +use std::ops::Range; + +// Target file size, when creating image and delta layers +pub const TARGET_FILE_SIZE_BYTES: u64 = 128 * 1024 * 1024; // 128 MB + +/// +/// Represents a set of Keys, in a compact form. +/// +#[derive(Clone, Debug)] +pub struct KeySpace { + /// Contiguous ranges of keys that belong to the key space. In key order, + /// and with no overlap. + pub ranges: Vec>, +} + +impl KeySpace { + /// + /// Partition a key space into roughly chunks of roughly 'target_size' bytes + /// in each patition. + /// + pub fn partition(&self, target_size: u64) -> KeyPartitioning { + // Assume that each value is 8k in size. + let target_nblocks = (target_size / pg_constants::BLCKSZ as u64) as usize; + + let mut parts = Vec::new(); + let mut current_part = Vec::new(); + let mut current_part_size: usize = 0; + for range in &self.ranges { + // If appending the next contiguous range in the keyspace to the current + // partition would cause it to be too large, start a new partition. + let this_size = key_range_size(range) as usize; + if current_part_size + this_size > target_nblocks && !current_part.is_empty() { + parts.push(KeySpace { + ranges: current_part, + }); + current_part = Vec::new(); + current_part_size = 0; + } + + // If the next range is larger than 'target_size', split it into + // 'target_size' chunks. + let mut remain_size = this_size; + let mut start = range.start; + while remain_size > target_nblocks { + let next = start.add(target_nblocks as u32); + parts.push(KeySpace { + ranges: vec![start..next], + }); + start = next; + remain_size -= target_nblocks + } + current_part.push(start..range.end); + current_part_size += remain_size; + } + + // add last partition that wasn't full yet. + if !current_part.is_empty() { + parts.push(KeySpace { + ranges: current_part, + }); + } + + KeyPartitioning { parts } + } +} + +/// +/// Represents a partitioning of the key space. +/// +/// The only kind of partitioning we do is to partition the key space into +/// partitions that are roughly equal in physical size (see KeySpace::partition). +/// But this data structure could represent any partitioning. +/// +#[derive(Clone, Debug, Default)] +pub struct KeyPartitioning { + pub parts: Vec, +} + +impl KeyPartitioning { + pub fn new() -> Self { + KeyPartitioning { parts: Vec::new() } + } +} + +/// +/// A helper object, to collect a set of keys and key ranges into a KeySpace +/// object. This takes care of merging adjacent keys and key ranges into +/// contiguous ranges. +/// +#[derive(Clone, Debug, Default)] +pub struct KeySpaceAccum { + accum: Option>, + + ranges: Vec>, +} + +impl KeySpaceAccum { + pub fn new() -> Self { + Self { + accum: None, + ranges: Vec::new(), + } + } + + pub fn add_key(&mut self, key: Key) { + self.add_range(singleton_range(key)) + } + + pub fn add_range(&mut self, range: Range) { + match self.accum.as_mut() { + Some(accum) => { + if range.start == accum.end { + accum.end = range.end; + } else { + assert!(range.start > accum.end); + self.ranges.push(accum.clone()); + *accum = range; + } + } + None => self.accum = Some(range), + } + } + + pub fn to_keyspace(mut self) -> KeySpace { + if let Some(accum) = self.accum.take() { + self.ranges.push(accum); + } + KeySpace { + ranges: self.ranges, + } + } +} diff --git a/pageserver/src/layered_repository.rs b/pageserver/src/layered_repository.rs index bf5f52b18d..837298a10e 100644 --- a/pageserver/src/layered_repository.rs +++ b/pageserver/src/layered_repository.rs @@ -14,32 +14,33 @@ use anyhow::{anyhow, bail, ensure, Context, Result}; use bookfile::Book; use bytes::Bytes; +use fail::fail_point; +use itertools::Itertools; use lazy_static::lazy_static; -use postgres_ffi::pg_constants::BLCKSZ; use tracing::*; -use std::cmp; +use std::cmp::{max, min, Ordering}; use std::collections::hash_map::Entry; +use std::collections::BTreeSet; use std::collections::HashMap; -use std::collections::{BTreeSet, HashSet}; use std::fs; use std::fs::{File, OpenOptions}; use std::io::Write; -use std::ops::{Bound::Included, Deref}; +use std::ops::{Bound::Included, Deref, Range}; use std::path::{Path, PathBuf}; -use std::sync::atomic::{self, AtomicBool, AtomicUsize}; -use std::sync::{Arc, Mutex, MutexGuard, RwLock, RwLockReadGuard}; +use std::sync::atomic::{self, AtomicBool}; +use std::sync::{Arc, Mutex, MutexGuard, RwLock, RwLockReadGuard, TryLockError}; use std::time::Instant; use self::metadata::{metadata_path, TimelineMetadata, METADATA_FILE_NAME}; use crate::config::PageServerConf; +use crate::keyspace::{KeyPartitioning, KeySpace}; use crate::page_cache; -use crate::relish::*; use crate::remote_storage::{schedule_timeline_checkpoint_upload, RemoteIndex}; use crate::repository::{ - BlockNumber, GcResult, Repository, RepositoryTimeline, Timeline, TimelineSyncStatusUpdate, - TimelineWriter, ZenithWalRecord, + GcResult, Repository, RepositoryTimeline, Timeline, TimelineSyncStatusUpdate, TimelineWriter, }; +use crate::repository::{Key, Value}; use crate::thread_mgr; use crate::virtual_file::VirtualFile; use crate::walreceiver::IS_WAL_RECEIVER; @@ -48,7 +49,6 @@ use crate::CheckpointConfig; use crate::{ZTenantId, ZTimelineId}; use zenith_metrics::{register_histogram_vec, Histogram, HistogramVec}; -use zenith_metrics::{register_int_gauge_vec, IntGauge, IntGaugeVec}; use zenith_utils::crashsafe_dir; use zenith_utils::lsn::{AtomicLsn, Lsn, RecordLsn}; use zenith_utils::seqwait::SeqWait; @@ -56,30 +56,25 @@ use zenith_utils::seqwait::SeqWait; mod delta_layer; pub(crate) mod ephemeral_file; mod filename; -mod global_layer_map; mod image_layer; mod inmemory_layer; -mod interval_tree; mod layer_map; pub mod metadata; mod par_fsync; mod storage_layer; -use delta_layer::DeltaLayer; +use delta_layer::{DeltaLayer, DeltaLayerWriter}; use ephemeral_file::is_ephemeral_file; use filename::{DeltaFileName, ImageFileName}; -use image_layer::ImageLayer; +use image_layer::{ImageLayer, ImageLayerWriter}; use inmemory_layer::InMemoryLayer; use layer_map::LayerMap; -use storage_layer::{ - Layer, PageReconstructData, PageReconstructResult, SegmentBlk, SegmentTag, RELISH_SEG_SIZE, -}; +use layer_map::SearchResult; +use storage_layer::{Layer, ValueReconstructResult, ValueReconstructState}; // re-export this function so that page_cache.rs can use it. pub use crate::layered_repository::ephemeral_file::writeback as writeback_ephemeral_file; -static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; 8192]); - // Metrics collected on operations on the storage repository. lazy_static! { static ref STORAGE_TIME: HistogramVec = register_histogram_vec!( @@ -100,17 +95,6 @@ lazy_static! { .expect("failed to define a metric"); } -lazy_static! { - // NOTE: can be zero if pageserver was restarted and there hasn't been any - // activity yet. - static ref LOGICAL_TIMELINE_SIZE: IntGaugeVec = register_int_gauge_vec!( - "pageserver_logical_timeline_size", - "Logical timeline size (bytes)", - &["tenant_id", "timeline_id"] - ) - .expect("failed to define a metric"); -} - /// Parts of the `.zenith/tenants//timelines/` directory prefix. pub const TIMELINES_SEGMENT_NAME: &str = "timelines"; @@ -118,7 +102,7 @@ pub const TIMELINES_SEGMENT_NAME: &str = "timelines"; /// Repository consists of multiple timelines. Keep them in a hash table. /// pub struct LayeredRepository { - conf: &'static PageServerConf, + pub conf: &'static PageServerConf, tenantid: ZTenantId, timelines: Mutex>, // This mutex prevents creation of new timelines during GC. @@ -135,21 +119,23 @@ pub struct LayeredRepository { remote_index: RemoteIndex, /// Makes every timeline to backup their files to remote storage. - upload_relishes: bool, + upload_layers: bool, } /// Public interface impl Repository for LayeredRepository { - fn get_timeline(&self, timelineid: ZTimelineId) -> Option { + type Timeline = LayeredTimeline; + + fn get_timeline(&self, timelineid: ZTimelineId) -> Option> { let timelines = self.timelines.lock().unwrap(); self.get_timeline_internal(timelineid, &timelines) .map(RepositoryTimeline::from) } - fn get_timeline_load(&self, timelineid: ZTimelineId) -> Result> { + fn get_timeline_load(&self, timelineid: ZTimelineId) -> Result> { let mut timelines = self.timelines.lock().unwrap(); match self.get_timeline_load_internal(timelineid, &mut timelines)? { - Some(local_loaded_timeline) => Ok(local_loaded_timeline as _), + Some(local_loaded_timeline) => Ok(local_loaded_timeline), None => anyhow::bail!( "cannot get local timeline: unknown timeline id: {}", timelineid @@ -157,7 +143,7 @@ impl Repository for LayeredRepository { } } - fn list_timelines(&self) -> Vec<(ZTimelineId, RepositoryTimeline)> { + fn list_timelines(&self) -> Vec<(ZTimelineId, RepositoryTimeline)> { self.timelines .lock() .unwrap() @@ -175,7 +161,7 @@ impl Repository for LayeredRepository { &self, timelineid: ZTimelineId, initdb_lsn: Lsn, - ) -> Result> { + ) -> Result> { let mut timelines = self.timelines.lock().unwrap(); // Create the timeline directory, and write initial metadata to file. @@ -191,9 +177,9 @@ impl Repository for LayeredRepository { timelineid, self.tenantid, Arc::clone(&self.walredo_mgr), - 0, - self.upload_relishes, + self.upload_layers, ); + timeline.layers.lock().unwrap().next_open_layer_at = Some(initdb_lsn); let timeline = Arc::new(timeline); let r = timelines.insert( @@ -282,13 +268,46 @@ impl Repository for LayeredRepository { }) } - fn checkpoint_iteration(&self, cconf: CheckpointConfig) -> Result<()> { + fn compaction_iteration(&self) -> Result<()> { + // Scan through the hashmap and collect a list of all the timelines, + // while holding the lock. Then drop the lock and actually perform the + // compactions. We don't want to block everything else while the + // compaction runs. + let timelines = self.timelines.lock().unwrap(); + let timelines_to_compact = timelines + .iter() + .map(|(timelineid, timeline)| (*timelineid, timeline.clone())) + .collect::>(); + drop(timelines); + + for (timelineid, timeline) in &timelines_to_compact { + let _entered = + info_span!("compact", timeline = %timelineid, tenant = %self.tenantid).entered(); + match timeline { + LayeredTimelineEntry::Loaded(timeline) => { + timeline.compact()?; + } + LayeredTimelineEntry::Unloaded { .. } => { + debug!("Cannot compact remote timeline {}", timelineid) + } + } + } + + Ok(()) + } + + /// + /// Flush all in-memory data to disk. + /// + /// Used at shutdown. + /// + fn checkpoint(&self) -> Result<()> { // Scan through the hashmap and collect a list of all the timelines, // while holding the lock. Then drop the lock and actually perform the // checkpoints. We don't want to block everything else while the // checkpoint runs. let timelines = self.timelines.lock().unwrap(); - let timelines_to_checkpoint = timelines + let timelines_to_compact = timelines .iter() // filter to get only loaded timelines .filter_map(|(timelineid, entry)| match entry { @@ -302,10 +321,10 @@ impl Repository for LayeredRepository { .collect::>(); drop(timelines); - for (timelineid, timeline) in &timelines_to_checkpoint { + for (timelineid, timeline) in &timelines_to_compact { let _entered = info_span!("checkpoint", timeline = %timelineid, tenant = %self.tenantid).entered(); - timeline.checkpoint(cconf)?; + timeline.checkpoint(CheckpointConfig::Flush)?; } Ok(()) @@ -403,7 +422,7 @@ impl LayeredTimelineEntry { } } -impl From for RepositoryTimeline { +impl From for RepositoryTimeline { fn from(entry: LayeredTimelineEntry) -> Self { match entry { LayeredTimelineEntry::Loaded(timeline) => RepositoryTimeline::Loaded(timeline as _), @@ -489,20 +508,18 @@ impl LayeredRepository { let _enter = info_span!("loading timeline", timeline = %timelineid, tenant = %self.tenantid) .entered(); - let mut timeline = LayeredTimeline::new( + let timeline = LayeredTimeline::new( self.conf, metadata, ancestor, timelineid, self.tenantid, Arc::clone(&self.walredo_mgr), - 0, // init with 0 and update after layers are loaded, - self.upload_relishes, + self.upload_layers, ); timeline .load_layer_map(disk_consistent_lsn) .context("failed to load layermap")?; - timeline.init_current_logical_size()?; Ok(Arc::new(timeline)) } @@ -512,7 +529,7 @@ impl LayeredRepository { walredo_mgr: Arc, tenantid: ZTenantId, remote_index: RemoteIndex, - upload_relishes: bool, + upload_layers: bool, ) -> LayeredRepository { LayeredRepository { tenantid, @@ -521,7 +538,7 @@ impl LayeredRepository { gc_cs: Mutex::new(()), walredo_mgr, remote_index, - upload_relishes, + upload_layers, } } @@ -673,7 +690,8 @@ impl LayeredRepository { timeline.checkpoint(CheckpointConfig::Forced)?; info!("timeline {} checkpoint_before_gc done", timelineid); } - let result = timeline.gc_timeline(branchpoints, cutoff)?; + timeline.update_gc_info(branchpoints, cutoff); + let result = timeline.gc()?; totals += result; timelines = self.timelines.lock().unwrap(); @@ -693,6 +711,8 @@ pub struct LayeredTimeline { layers: Mutex, + last_freeze_at: AtomicLsn, + // WAL redo manager walredo_mgr: Arc, @@ -725,33 +745,14 @@ pub struct LayeredTimeline { ancestor_timeline: Option, ancestor_lsn: Lsn, - // this variable indicates how much space is used from user's point of view, - // e.g. we do not account here for multiple versions of data and so on. - // this is counted incrementally based on physical relishes (excluding FileNodeMap) - // current_logical_size is not stored no disk and initialized on timeline creation using - // get_current_logical_size_non_incremental in init_current_logical_size - // this is needed because when we save it in metadata it can become out of sync - // because current_logical_size is consistent on last_record_lsn, not ondisk_consistent_lsn - // NOTE: current_logical_size also includes size of the ancestor - current_logical_size: AtomicUsize, // bytes - - // To avoid calling .with_label_values and formatting the tenant and timeline IDs to strings - // every time the logical size is updated, keep a direct reference to the Gauge here. - // unfortunately it doesnt forward atomic methods like .fetch_add - // so use two fields: actual size and metric - // see https://github.com/zenithdb/zenith/issues/622 for discussion - // TODO: it is possible to combine these two fields into single one using custom metric which uses SeqCst - // ordering for its operations, but involves private modules, and macro trickery - current_logical_size_gauge: IntGauge, - // Metrics histograms reconstruct_time_histo: Histogram, - checkpoint_time_histo: Histogram, - flush_checkpoint_time_histo: Histogram, - forced_checkpoint_time_histo: Histogram, + flush_time_histo: Histogram, + compact_time_histo: Histogram, + create_images_time_histo: Histogram, /// If `true`, will backup its files that appear after each checkpointing to the remote storage. - upload_relishes: AtomicBool, + upload_layers: AtomicBool, /// Ensures layers aren't frozen by checkpointer between /// [`LayeredTimeline::get_layer_for_write`] and layer reads. @@ -760,15 +761,24 @@ pub struct LayeredTimeline { /// to avoid deadlock. write_lock: Mutex<()>, - // Prevent concurrent checkpoints. - // Checkpoints are normally performed by one thread. But checkpoint can also be manually requested by admin - // (that's used in tests), and shutdown also forces a checkpoint. These forced checkpoints run in a different thread - // and could be triggered at the same time as a normal checkpoint. - checkpoint_cs: Mutex<()>, + /// Used to ensure that there is only one thread + layer_flush_lock: Mutex<()>, + + // Prevent concurrent compactions. + // Compactions are normally performed by one thread. But compaction can also be manually + // requested by admin (that's used in tests). These forced compactions run in a different + // thread and could be triggered at the same time as a normal, timed compaction. + compaction_cs: Mutex<()>, // Needed to ensure that we can't create a branch at a point that was already garbage collected latest_gc_cutoff_lsn: RwLock, + // List of child timelines and their branch points. This is needed to avoid + // garbage collecting data that is still needed by the child timelines. + gc_info: RwLock, + + partitioning: RwLock>, + // It may change across major versions so for simplicity // keep it after running initdb for a timeline. // It is needed in checks when we want to error on some operations @@ -778,6 +788,28 @@ pub struct LayeredTimeline { initdb_lsn: Lsn, } +/// +/// Information about how much history needs to be retained, needed by +/// Garbage Collection. +/// +struct GcInfo { + /// Specific LSNs that are needed. + /// + /// Currently, this includes all points where child branches have + /// been forked off from. In the future, could also include + /// explicit user-defined snapshot points. + retain_lsns: Vec, + + /// In addition to 'retain_lsns', keep everything newer than this + /// point. + /// + /// This is calculated by subtracting 'gc_horizon' setting from + /// last-record LSN + /// + /// FIXME: is this inclusive or exclusive? + cutoff: Lsn, +} + /// Public interface functions impl Timeline for LayeredTimeline { fn get_ancestor_lsn(&self) -> Lsn { @@ -815,162 +847,35 @@ impl Timeline for LayeredTimeline { self.latest_gc_cutoff_lsn.read().unwrap() } - /// Look up given page version. - fn get_page_at_lsn(&self, rel: RelishTag, rel_blknum: BlockNumber, lsn: Lsn) -> Result { - if !rel.is_blocky() && rel_blknum != 0 { - bail!( - "invalid request for block {} for non-blocky relish {}", - rel_blknum, - rel - ); - } - debug_assert!(lsn <= self.get_last_record_lsn()); - let (seg, seg_blknum) = SegmentTag::from_blknum(rel, rel_blknum); - - if let Some((layer, lsn)) = self.get_layer_for_read(seg, lsn)? { - self.materialize_page(seg, seg_blknum, lsn, &*layer) - } else { - // FIXME: This can happen if PostgreSQL extends a relation but never writes - // the page. See https://github.com/zenithdb/zenith/issues/841 - // - // Would be nice to detect that situation better. - if seg.segno > 0 && self.get_rel_exists(rel, lsn)? { - warn!("Page {} blk {} at {} not found", rel, rel_blknum, lsn); - return Ok(ZERO_PAGE.clone()); - } - - bail!("segment {} not found at {}", rel, lsn); - } - } - - fn get_relish_size(&self, rel: RelishTag, lsn: Lsn) -> Result> { - if !rel.is_blocky() { - bail!( - "invalid get_relish_size request for non-blocky relish {}", - rel - ); - } + /// Look up the value with the given a key + fn get(&self, key: Key, lsn: Lsn) -> Result { debug_assert!(lsn <= self.get_last_record_lsn()); - let mut segno = 0; - loop { - let seg = SegmentTag { rel, segno }; - - let segsize; - if let Some((layer, lsn)) = self.get_layer_for_read(seg, lsn)? { - segsize = layer.get_seg_size(lsn)?; - trace!("get_seg_size: {} at {} -> {}", seg, lsn, segsize); - } else { - if segno == 0 { - return Ok(None); + // Check the page cache. We will get back the most recent page with lsn <= `lsn`. + // The cached image can be returned directly if there is no WAL between the cached image + // and requested LSN. The cached image can also be used to reduce the amount of WAL needed + // for redo. + let cached_page_img = match self.lookup_cached_page(&key, lsn) { + Some((cached_lsn, cached_img)) => { + match cached_lsn.cmp(&lsn) { + Ordering::Less => {} // there might be WAL between cached_lsn and lsn, we need to check + Ordering::Equal => return Ok(cached_img), // exact LSN match, return the image + Ordering::Greater => panic!(), // the returned lsn should never be after the requested lsn } - segsize = 0; + Some((cached_lsn, cached_img)) } - - if segsize != RELISH_SEG_SIZE { - let result = segno * RELISH_SEG_SIZE + segsize; - return Ok(Some(result)); - } - segno += 1; - } - } - - fn get_rel_exists(&self, rel: RelishTag, lsn: Lsn) -> Result { - debug_assert!(lsn <= self.get_last_record_lsn()); - - let seg = SegmentTag { rel, segno: 0 }; - - let result = if let Some((layer, lsn)) = self.get_layer_for_read(seg, lsn)? { - layer.get_seg_exists(lsn)? - } else { - false + None => None, }; - trace!("get_rel_exists: {} at {} -> {}", rel, lsn, result); - Ok(result) - } - - fn list_rels(&self, spcnode: u32, dbnode: u32, lsn: Lsn) -> Result> { - let request_tag = RelTag { - spcnode, - dbnode, - relnode: 0, - forknum: 0, + let mut reconstruct_state = ValueReconstructState { + records: Vec::new(), + img: cached_page_img, }; - self.list_relishes(Some(request_tag), lsn) - } + self.get_reconstruct_data(key, lsn, &mut reconstruct_state)?; - fn list_nonrels(&self, lsn: Lsn) -> Result> { - info!("list_nonrels called at {}", lsn); - - self.list_relishes(None, lsn) - } - - fn list_relishes(&self, tag: Option, lsn: Lsn) -> Result> { - trace!("list_relishes called at {}", lsn); - debug_assert!(lsn <= self.get_last_record_lsn()); - - // List of all relishes along with a flag that marks if they exist at the given lsn. - let mut all_relishes_map: HashMap = HashMap::new(); - let mut result = HashSet::new(); - let mut timeline = self; - - // Iterate through layers back in time and find the most - // recent state of the relish. Don't add relish to the list - // if newer version is already there. - // - // This most recent version can represent dropped or existing relish. - // We will filter dropped relishes below. - // - loop { - let rels = timeline.layers.lock().unwrap().list_relishes(tag, lsn)?; - - for (&new_relish, &new_relish_exists) in rels.iter() { - match all_relishes_map.entry(new_relish) { - Entry::Occupied(o) => { - trace!( - "Newer version of the object {} is already found: exists {}", - new_relish, - o.get(), - ); - } - Entry::Vacant(v) => { - v.insert(new_relish_exists); - trace!( - "Newer version of the object {} NOT found. Insert NEW: exists {}", - new_relish, - new_relish_exists - ); - } - } - } - - match &timeline.ancestor_timeline { - None => break, - Some(ancestor_entry) => { - timeline = ancestor_entry.ensure_loaded().with_context( - || format!( - "cannot list relishes for timeline {} tenant {} due to its ancestor {} being either unloaded", - self.timelineid, self.tenantid, ancestor_entry.timeline_id(), - ) - )?; - continue; - } - } - } - - // Filter out dropped relishes - for (&new_relish, &new_relish_exists) in all_relishes_map.iter() { - if new_relish_exists { - result.insert(new_relish); - trace!("List object {}", new_relish); - } else { - trace!("Filtered out dropped object {}", new_relish); - } - } - - Ok(result) + self.reconstruct_time_histo + .observe_closure_duration(|| self.reconstruct_value(key, lsn, reconstruct_state)) } /// Public entry point for checkpoint(). All the logic is in the private @@ -978,15 +883,15 @@ impl Timeline for LayeredTimeline { /// metrics collection. fn checkpoint(&self, cconf: CheckpointConfig) -> anyhow::Result<()> { match cconf { - CheckpointConfig::Flush => self - .flush_checkpoint_time_histo - .observe_closure_duration(|| self.checkpoint_internal(0, false)), - CheckpointConfig::Forced => self - .forced_checkpoint_time_histo - .observe_closure_duration(|| self.checkpoint_internal(0, true)), - CheckpointConfig::Distance(distance) => self - .checkpoint_time_histo - .observe_closure_duration(|| self.checkpoint_internal(distance, true)), + CheckpointConfig::Flush => { + self.freeze_inmem_layer(false); + self.flush_frozen_layers(true) + } + CheckpointConfig::Forced => { + self.freeze_inmem_layer(false); + self.flush_frozen_layers(true)?; + self.compact() + } } } @@ -1019,51 +924,24 @@ impl Timeline for LayeredTimeline { self.last_record_lsn.load() } - fn get_current_logical_size(&self) -> usize { - self.current_logical_size.load(atomic::Ordering::Acquire) as usize - } - - fn get_current_logical_size_non_incremental(&self, lsn: Lsn) -> Result { - let mut total_blocks: usize = 0; - - let _enter = info_span!("calc logical size", %lsn).entered(); - - // list of all relations in this timeline, including ancestor timelines - let all_rels = self.list_rels(0, 0, lsn)?; - - for rel in all_rels { - if let Some(size) = self.get_relish_size(rel, lsn)? { - total_blocks += size as usize; - } - } - - let non_rels = self.list_nonrels(lsn)?; - for non_rel in non_rels { - // TODO support TwoPhase - if matches!(non_rel, RelishTag::Slru { slru: _, segno: _ }) { - if let Some(size) = self.get_relish_size(non_rel, lsn)? { - total_blocks += size as usize; - } - } - } - - Ok(total_blocks * BLCKSZ as usize) - } - fn get_disk_consistent_lsn(&self) -> Lsn { self.disk_consistent_lsn.load() } + fn hint_partitioning(&self, partitioning: KeyPartitioning, lsn: Lsn) -> Result<()> { + self.partitioning + .write() + .unwrap() + .replace((partitioning, lsn)); + Ok(()) + } + fn writer<'a>(&'a self) -> Box { Box::new(LayeredTimelineWriter { tl: self, _write_guard: self.write_lock.lock().unwrap(), }) } - - fn upgrade_to_layered_timeline(&self) -> &crate::layered_repository::LayeredTimeline { - self - } } impl LayeredTimeline { @@ -1078,32 +956,28 @@ impl LayeredTimeline { timelineid: ZTimelineId, tenantid: ZTenantId, walredo_mgr: Arc, - current_logical_size: usize, - upload_relishes: bool, + upload_layers: bool, ) -> LayeredTimeline { - let current_logical_size_gauge = LOGICAL_TIMELINE_SIZE - .get_metric_with_label_values(&[&tenantid.to_string(), &timelineid.to_string()]) - .unwrap(); let reconstruct_time_histo = RECONSTRUCT_TIME .get_metric_with_label_values(&[&tenantid.to_string(), &timelineid.to_string()]) .unwrap(); - let checkpoint_time_histo = STORAGE_TIME + let flush_time_histo = STORAGE_TIME .get_metric_with_label_values(&[ - "checkpoint", + "layer flush", &tenantid.to_string(), &timelineid.to_string(), ]) .unwrap(); - let flush_checkpoint_time_histo = STORAGE_TIME + let compact_time_histo = STORAGE_TIME .get_metric_with_label_values(&[ - "flush checkpoint", + "compact", &tenantid.to_string(), &timelineid.to_string(), ]) .unwrap(); - let forced_checkpoint_time_histo = STORAGE_TIME + let create_images_time_histo = STORAGE_TIME .get_metric_with_label_values(&[ - "forced checkpoint", + "create images", &tenantid.to_string(), &timelineid.to_string(), ]) @@ -1124,18 +998,27 @@ impl LayeredTimeline { }), disk_consistent_lsn: AtomicLsn::new(metadata.disk_consistent_lsn().0), + last_freeze_at: AtomicLsn::new(0), + ancestor_timeline: ancestor, ancestor_lsn: metadata.ancestor_lsn(), - current_logical_size: AtomicUsize::new(current_logical_size), - current_logical_size_gauge, + reconstruct_time_histo, - checkpoint_time_histo, - flush_checkpoint_time_histo, - forced_checkpoint_time_histo, - upload_relishes: AtomicBool::new(upload_relishes), + flush_time_histo, + compact_time_histo, + create_images_time_histo, + + upload_layers: AtomicBool::new(upload_layers), write_lock: Mutex::new(()), - checkpoint_cs: Mutex::new(()), + layer_flush_lock: Mutex::new(()), + compaction_cs: Mutex::new(()), + + gc_info: RwLock::new(GcInfo { + retain_lsns: Vec::new(), + cutoff: Lsn(0), + }), + partitioning: RwLock::new(None), latest_gc_cutoff_lsn: RwLock::new(metadata.latest_gc_cutoff_lsn()), initdb_lsn: metadata.initdb_lsn(), @@ -1179,13 +1062,12 @@ impl LayeredTimeline { num_layers += 1; } else if let Some(deltafilename) = DeltaFileName::parse_str(&fname) { // Create a DeltaLayer struct for each delta file. - ensure!(deltafilename.start_lsn < deltafilename.end_lsn); // The end-LSN is exclusive, while disk_consistent_lsn is // inclusive. For example, if disk_consistent_lsn is 100, it is // OK for a delta layer to have end LSN 101, but if the end LSN // is 102, then it might not have been fully flushed to disk // before crash. - if deltafilename.end_lsn > disk_consistent_lsn + 1 { + if deltafilename.lsn_range.end > disk_consistent_lsn + 1 { warn!( "found future delta layer {} on timeline {} disk_consistent_lsn is {}", deltafilename, self.timelineid, disk_consistent_lsn @@ -1212,41 +1094,14 @@ impl LayeredTimeline { } } - info!("loaded layer map with {} layers", num_layers); + layers.next_open_layer_at = Some(Lsn(disk_consistent_lsn.0) + 1); - Ok(()) - } - - /// - /// Used to init current logical size on startup - /// - fn init_current_logical_size(&mut self) -> Result<()> { - if self.current_logical_size.load(atomic::Ordering::Relaxed) != 0 { - bail!("cannot init already initialized current logical size") - }; - let lsn = self.get_last_record_lsn(); - self.current_logical_size = - AtomicUsize::new(self.get_current_logical_size_non_incremental(lsn)?); - trace!( - "current_logical_size initialized to {}", - self.current_logical_size.load(atomic::Ordering::Relaxed) + info!( + "loaded layer map with {} layers at {}", + num_layers, disk_consistent_lsn ); - Ok(()) - } - /// - /// Get a handle to a Layer for reading. - /// - /// The returned Layer might be from an ancestor timeline, if the - /// segment hasn't been updated on this timeline yet. - /// - fn get_layer_for_read( - &self, - seg: SegmentTag, - lsn: Lsn, - ) -> Result, Lsn)>> { - let self_layers = self.layers.lock().unwrap(); - self.get_layer_for_read_locked(seg, lsn, &self_layers) + Ok(()) } /// @@ -1257,88 +1112,160 @@ impl LayeredTimeline { /// /// This function takes the current timeline's locked LayerMap as an argument, /// so callers can avoid potential race conditions. - fn get_layer_for_read_locked( + fn get_reconstruct_data( &self, - seg: SegmentTag, - lsn: Lsn, - self_layers: &MutexGuard, - ) -> anyhow::Result, Lsn)>> { - trace!("get_layer_for_read called for {} at {}", seg, lsn); - - // If you requested a page at an older LSN, before the branch point, dig into - // the right ancestor timeline. This can only happen if you launch a read-only - // node with an old LSN, a primary always uses a recent LSN in its requests. + key: Key, + request_lsn: Lsn, + reconstruct_state: &mut ValueReconstructState, + ) -> anyhow::Result<()> { + // Start from the current timeline. + let mut timeline_owned; let mut timeline = self; - let mut lsn = lsn; - while lsn < timeline.ancestor_lsn { - trace!("going into ancestor {} ", timeline.ancestor_lsn); - timeline = timeline - .ancestor_timeline - .as_ref() - .expect("there should be an ancestor") - .ensure_loaded() - .with_context(|| format!( - "Cannot get the whole layer for read locked: timeline {} is not present locally", - self.get_ancestor_timeline_id().unwrap()) - )?; - } + let mut path: Vec<(ValueReconstructResult, Lsn, Arc)> = Vec::new(); - // Now we have the right starting timeline for our search. - loop { - let layers_owned: MutexGuard; - let layers = if self as *const LayeredTimeline != timeline as *const LayeredTimeline { - layers_owned = timeline.layers.lock().unwrap(); - &layers_owned - } else { - self_layers - }; + // 'prev_lsn' tracks the last LSN that we were at in our search. It's used + // to check that each iteration make some progress, to break infinite + // looping if something goes wrong. + let mut prev_lsn = Lsn(u64::MAX); - // - // FIXME: If the relation has been dropped, does this return the right - // thing? The compute node should not normally request dropped relations, - // but if OID wraparound happens the same relfilenode might get reused - // for an unrelated relation. - // + let mut result = ValueReconstructResult::Continue; + let mut cont_lsn = Lsn(request_lsn.0 + 1); - // Do we have a layer on this timeline? - if let Some(layer) = layers.get(&seg, lsn) { - trace!( - "found layer in cache: {} {}-{}", - timeline.timelineid, - layer.get_start_lsn(), - layer.get_end_lsn() - ); + 'outer: loop { + // The function should have updated 'state' + //info!("CALLED for {} at {}: {:?} with {} records", reconstruct_state.key, reconstruct_state.lsn, result, reconstruct_state.records.len()); + match result { + ValueReconstructResult::Complete => return Ok(()), + ValueReconstructResult::Continue => { + if prev_lsn <= cont_lsn { + // Didn't make any progress in last iteration. Error out to avoid + // getting stuck in the loop. - ensure!(layer.get_start_lsn() <= lsn); - - if layer.is_dropped() && layer.get_end_lsn() <= lsn { - return Ok(None); + // For debugging purposes, print the path of layers that we traversed + // through. + for (r, c, l) in path { + error!( + "PATH: result {:?}, cont_lsn {}, layer: {}", + r, + c, + l.filename().display() + ); + } + bail!("could not find layer with more data for key {} at LSN {}, request LSN {}, ancestor {}", + key, + Lsn(cont_lsn.0 - 1), + request_lsn, + timeline.ancestor_lsn) + } + prev_lsn = cont_lsn; + } + ValueReconstructResult::Missing => { + bail!( + "could not find data for key {} at LSN {}, for request at LSN {}", + key, + cont_lsn, + request_lsn + ) } - - return Ok(Some((layer.clone(), lsn))); } - // If not, check if there's a layer on the ancestor timeline - match &timeline.ancestor_timeline { - Some(ancestor_entry) => { - let ancestor = ancestor_entry - .ensure_loaded() - .context("cannot get a layer for read from ancestor because it is either remote or unloaded")?; - lsn = timeline.ancestor_lsn; - timeline = ancestor; - trace!("recursing into ancestor at {}/{}", timeline.timelineid, lsn); + // Recurse into ancestor if needed + if Lsn(cont_lsn.0 - 1) <= timeline.ancestor_lsn { + trace!( + "going into ancestor {}, cont_lsn is {}", + timeline.ancestor_lsn, + cont_lsn + ); + let ancestor = timeline.get_ancestor_timeline()?; + timeline_owned = ancestor; + timeline = &*timeline_owned; + prev_lsn = Lsn(u64::MAX); + continue; + } + + let layers = timeline.layers.lock().unwrap(); + + // Check the open and frozen in-memory layers first + if let Some(open_layer) = &layers.open_layer { + let start_lsn = open_layer.get_lsn_range().start; + if cont_lsn > start_lsn { + //info!("CHECKING for {} at {} on open layer {}", key, cont_lsn, open_layer.filename().display()); + result = open_layer.get_value_reconstruct_data( + key, + open_layer.get_lsn_range().start..cont_lsn, + reconstruct_state, + )?; + cont_lsn = start_lsn; + path.push((result, cont_lsn, open_layer.clone())); continue; } - None => return Ok(None), + } + for frozen_layer in layers.frozen_layers.iter() { + let start_lsn = frozen_layer.get_lsn_range().start; + if cont_lsn > start_lsn { + //info!("CHECKING for {} at {} on frozen layer {}", key, cont_lsn, frozen_layer.filename().display()); + result = frozen_layer.get_value_reconstruct_data( + key, + frozen_layer.get_lsn_range().start..cont_lsn, + reconstruct_state, + )?; + cont_lsn = start_lsn; + path.push((result, cont_lsn, frozen_layer.clone())); + continue 'outer; + } + } + + if let Some(SearchResult { lsn_floor, layer }) = layers.search(key, cont_lsn)? { + //info!("CHECKING for {} at {} on historic layer {}", key, cont_lsn, layer.filename().display()); + + result = layer.get_value_reconstruct_data( + key, + lsn_floor..cont_lsn, + reconstruct_state, + )?; + cont_lsn = lsn_floor; + path.push((result, cont_lsn, layer)); + } else if self.ancestor_timeline.is_some() { + // Nothing on this timeline. Traverse to parent + result = ValueReconstructResult::Continue; + cont_lsn = Lsn(self.ancestor_lsn.0 + 1); + } else { + // Nothing found + result = ValueReconstructResult::Missing; } } } + fn lookup_cached_page(&self, key: &Key, lsn: Lsn) -> Option<(Lsn, Bytes)> { + let cache = page_cache::get(); + + // FIXME: It's pointless to check the cache for things that are not 8kB pages. + // We should look at the key to determine if it's a cacheable object + let (lsn, read_guard) = + cache.lookup_materialized_page(self.tenantid, self.timelineid, key, lsn)?; + let img = Bytes::from(read_guard.to_vec()); + Some((lsn, img)) + } + + fn get_ancestor_timeline(&self) -> Result> { + let ancestor = self + .ancestor_timeline + .as_ref() + .expect("there should be an ancestor") + .ensure_loaded() + .with_context(|| { + format!( + "Cannot get the whole layer for read locked: timeline {} is not present locally", + self.get_ancestor_timeline_id().unwrap()) + })?; + Ok(Arc::clone(ancestor)) + } + /// /// Get a handle to the latest layer for appending. /// - fn get_layer_for_write(&self, seg: SegmentTag, lsn: Lsn) -> anyhow::Result> { + fn get_layer_for_write(&self, lsn: Lsn) -> anyhow::Result> { let mut layers = self.layers.lock().unwrap(); ensure!(lsn.is_aligned()); @@ -1353,235 +1280,191 @@ impl LayeredTimeline { // Do we have a layer open for writing already? let layer; - if let Some(open_layer) = layers.get_open(&seg) { - if open_layer.get_start_lsn() > lsn { + if let Some(open_layer) = &layers.open_layer { + if open_layer.get_lsn_range().start > lsn { bail!("unexpected open layer in the future"); } - // Open layer exists, but it is dropped, so create a new one. - if open_layer.is_dropped() { - ensure!(!open_layer.is_writeable()); - // Layer that is created after dropped one represents a new relish segment. - trace!( - "creating layer for write for new relish segment after dropped layer {} at {}/{}", - seg, - self.timelineid, - lsn - ); - - layer = InMemoryLayer::create( - self.conf, - self.timelineid, - self.tenantid, - seg, - lsn, - last_record_lsn, - )?; - } else { - return Ok(open_layer); - } - } - // No writeable layer for this relation. Create one. - // - // Is this a completely new relation? Or the first modification after branching? - // - else if let Some((prev_layer, _prev_lsn)) = - self.get_layer_for_read_locked(seg, lsn, &layers)? - { - // Create new entry after the previous one. - let start_lsn; - if prev_layer.get_timeline_id() != self.timelineid { - // First modification on this timeline - start_lsn = self.ancestor_lsn + 1; - trace!( - "creating layer for write for {} at branch point {}", - seg, - start_lsn - ); - } else { - start_lsn = prev_layer.get_end_lsn(); - trace!( - "creating layer for write for {} after previous layer {}", - seg, - start_lsn - ); - } - trace!( - "prev layer is at {}/{} - {}", - prev_layer.get_timeline_id(), - prev_layer.get_start_lsn(), - prev_layer.get_end_lsn() - ); - layer = InMemoryLayer::create_successor_layer( - self.conf, - prev_layer, - self.timelineid, - self.tenantid, - start_lsn, - last_record_lsn, - )?; + layer = Arc::clone(open_layer); } else { - // New relation. + // No writeable layer yet. Create one. + let start_lsn = layers.next_open_layer_at.unwrap(); + trace!( - "creating layer for write for new rel {} at {}/{}", - seg, + "creating layer for write at {}/{} for record at {}", self.timelineid, + start_lsn, lsn ); + let new_layer = + InMemoryLayer::create(self.conf, self.timelineid, self.tenantid, start_lsn)?; + let layer_rc = Arc::new(new_layer); - layer = InMemoryLayer::create( - self.conf, - self.timelineid, - self.tenantid, - seg, - lsn, - last_record_lsn, - )?; + layers.open_layer = Some(Arc::clone(&layer_rc)); + layers.next_open_layer_at = None; + + layer = layer_rc; } + Ok(layer) + } - let layer_rc: Arc = Arc::new(layer); - layers.insert_open(Arc::clone(&layer_rc)); + fn put_value(&self, key: Key, lsn: Lsn, val: Value) -> Result<()> { + //info!("PUT: key {} at {}", key, lsn); + let layer = self.get_layer_for_write(lsn)?; + layer.put_value(key, lsn, val)?; + Ok(()) + } - Ok(layer_rc) + fn put_tombstone(&self, key_range: Range, lsn: Lsn) -> Result<()> { + let layer = self.get_layer_for_write(lsn)?; + layer.put_tombstone(key_range, lsn)?; + + Ok(()) + } + + fn finish_write(&self, new_lsn: Lsn) { + assert!(new_lsn.is_aligned()); + + self.last_record_lsn.advance(new_lsn); + } + + fn freeze_inmem_layer(&self, write_lock_held: bool) { + // Freeze the current open in-memory layer. It will be written to disk on next + // iteration. + let _write_guard = if write_lock_held { + None + } else { + Some(self.write_lock.lock().unwrap()) + }; + let mut layers = self.layers.lock().unwrap(); + if let Some(open_layer) = &layers.open_layer { + let open_layer_rc = Arc::clone(open_layer); + // Does this layer need freezing? + let end_lsn = Lsn(self.get_last_record_lsn().0 + 1); + open_layer.freeze(end_lsn); + + // The layer is no longer open, update the layer map to reflect this. + // We will replace it with on-disk historics below. + layers.frozen_layers.push_back(open_layer_rc); + layers.open_layer = None; + layers.next_open_layer_at = Some(end_lsn); + self.last_freeze_at.store(end_lsn); + } + drop(layers); } /// - /// Flush to disk all data that was written with the put_* functions + /// Check if more than 'checkpoint_distance' of WAL has been accumulated + /// in the in-memory layer, and initiate flushing it if so. /// - /// NOTE: This has nothing to do with checkpoint in PostgreSQL. - fn checkpoint_internal(&self, checkpoint_distance: u64, reconstruct_pages: bool) -> Result<()> { - // Prevent concurrent checkpoints - let _checkpoint_cs = self.checkpoint_cs.lock().unwrap(); - let write_guard = self.write_lock.lock().unwrap(); - let mut layers = self.layers.lock().unwrap(); + pub fn check_checkpoint_distance(self: &Arc) -> Result<()> { + let last_lsn = self.get_last_record_lsn(); - // Bump the generation number in the layer map, so that we can distinguish - // entries inserted after the checkpoint started - let current_generation = layers.increment_generation(); + let distance = last_lsn.widening_sub(self.last_freeze_at.load()); + if distance >= self.conf.checkpoint_distance.into() { + self.freeze_inmem_layer(true); + self.last_freeze_at.store(last_lsn); + } + if let Ok(guard) = self.layer_flush_lock.try_lock() { + drop(guard); + let self_clone = Arc::clone(self); + thread_mgr::spawn( + thread_mgr::ThreadKind::LayerFlushThread, + Some(self.tenantid), + Some(self.timelineid), + "layer flush thread", + false, + move || self_clone.flush_frozen_layers(false), + )?; + } + Ok(()) + } - let RecordLsn { - last: last_record_lsn, - prev: prev_record_lsn, - } = self.last_record_lsn.load(); + /// Flush all frozen layers to disk. + /// + /// Only one thread at a time can be doing layer-flushing for a + /// given timeline. If 'wait' is true, and another thread is + /// currently doing the flushing, this function will wait for it + /// to finish. If 'wait' is false, this function will return + /// immediately instead. + fn flush_frozen_layers(&self, wait: bool) -> Result<()> { + let flush_lock_guard = if wait { + self.layer_flush_lock.lock().unwrap() + } else { + match self.layer_flush_lock.try_lock() { + Ok(guard) => guard, + Err(TryLockError::WouldBlock) => return Ok(()), + Err(TryLockError::Poisoned(err)) => panic!("{:?}", err), + } + }; - trace!("checkpoint starting at {}", last_record_lsn); + let timer = self.flush_time_histo.start_timer(); - // Take the in-memory layer with the oldest WAL record. If it's older - // than the threshold, write it out to disk as a new image and delta file. - // Repeat until all remaining in-memory layers are within the threshold. - // - // That's necessary to limit the amount of WAL that needs to be kept - // in the safekeepers, and that needs to be reprocessed on page server - // crash. TODO: It's not a great policy for keeping memory usage in - // check, though. We should also aim at flushing layers that consume - // a lot of memory and/or aren't receiving much updates anymore. - let mut disk_consistent_lsn = last_record_lsn; - - let mut layer_paths = Vec::new(); - let mut freeze_end_lsn = Lsn(0); - let mut evicted_layers = Vec::new(); - - // - // Determine which layers we need to evict and calculate max(latest_lsn) - // among those layers. - // - while let Some((oldest_layer_id, oldest_layer, oldest_generation)) = - layers.peek_oldest_open() - { - let oldest_lsn = oldest_layer.get_oldest_lsn(); - // Does this layer need freezing? - // - // Write out all in-memory layers that contain WAL older than CHECKPOINT_DISTANCE. - // If we reach a layer with the same - // generation number, we know that we have cycled through all layers that were open - // when we started. We don't want to process layers inserted after we started, to - // avoid getting into an infinite loop trying to process again entries that we - // inserted ourselves. - // - // Once we have decided to write out at least one layer, we must also write out - // any other layers that contain WAL older than the end LSN of the layers we have - // already decided to write out. In other words, we must write out all layers - // whose [oldest_lsn, latest_lsn) range overlaps with any of the other layers - // that we are writing out. Otherwise, when we advance 'disk_consistent_lsn', it's - // ambiguous whether those layers are already durable on disk or not. For example, - // imagine that there are two layers in memory that contain page versions in the - // following LSN ranges: - // - // A: 100-150 - // B: 110-200 - // - // If we flush layer A, we must also flush layer B, because they overlap. If we - // flushed only A, and advanced 'disk_consistent_lsn' to 150, we would break the - // rule that all WAL older than 'disk_consistent_lsn' are durable on disk, because - // B contains some WAL older than 150. On the other hand, if we flushed out A and - // advanced 'disk_consistent_lsn' only up to 110, after crash and restart we would - // delete the first layer because its end LSN is larger than 110. If we changed - // the deletion logic to not delete it, then we would start streaming at 110, and - // process again the WAL records in the range 110-150 that are already in layer A, - // and the WAL processing code does not cope with that. We solve that dilemma by - // insisting that if we write out the first layer, we also write out the second - // layer, and advance disk_consistent_lsn all the way up to 200. - // - let distance = last_record_lsn.widening_sub(oldest_lsn); - if (distance < 0 - || distance < checkpoint_distance.into() - || oldest_generation == current_generation) - && oldest_lsn >= freeze_end_lsn - // this layer intersects with evicted layer and so also need to be evicted - { - debug!( - "the oldest layer is now {} which is {} bytes behind last_record_lsn", - oldest_layer.filename().display(), - distance - ); - disk_consistent_lsn = oldest_lsn; + loop { + let layers = self.layers.lock().unwrap(); + if let Some(frozen_layer) = layers.frozen_layers.front() { + let frozen_layer = Arc::clone(frozen_layer); + drop(layers); // to allow concurrent reads and writes + self.flush_frozen_layer(frozen_layer)?; + } else { + // Drop the 'layer_flush_lock' *before* 'layers'. That + // way, if you freeze a layer, and then call + // flush_frozen_layers(false), it is guaranteed that + // if another thread was busy flushing layers and the + // call therefore returns immediately, the other + // thread will have seen the newly-frozen layer and + // will flush that too (assuming no errors). + drop(flush_lock_guard); + drop(layers); break; } - let latest_lsn = oldest_layer.get_latest_lsn(); - if latest_lsn > freeze_end_lsn { - freeze_end_lsn = latest_lsn; // calculate max of latest_lsn of the layers we're about to evict - } - layers.remove_open(oldest_layer_id); - evicted_layers.push((oldest_layer_id, oldest_layer)); } - // Freeze evicted layers - for (_evicted_layer_id, evicted_layer) in evicted_layers.iter() { - // Mark the layer as no longer accepting writes and record the end_lsn. - // This happens in-place, no new layers are created now. - evicted_layer.freeze(freeze_end_lsn); - layers.insert_historic(evicted_layer.clone()); + timer.stop_and_record(); + + Ok(()) + } + + /// Flush one frozen in-memory layer to disk, as a new delta layer. + fn flush_frozen_layer(&self, frozen_layer: Arc) -> Result<()> { + let new_delta = frozen_layer.write_to_disk()?; + let new_delta_path = new_delta.path(); + + // Sync the new layer to disk. + // + // We must also fsync the timeline dir to ensure the directory entries for + // new layer files are durable + // + // TODO: If we're running inside 'flush_frozen_layers' and there are multiple + // files to flush, it might be better to first write them all, and then fsync + // them all in parallel. + par_fsync::par_fsync(&[ + new_delta_path.clone(), + self.conf.timeline_path(&self.timelineid, &self.tenantid), + ])?; + + // Finally, replace the frozen in-memory layer with the new on-disk layers + { + let mut layers = self.layers.lock().unwrap(); + let l = layers.frozen_layers.pop_front(); + + // Only one thread may call this function at a time (for this + // timeline). If two threads tried to flush the same frozen + // layer to disk at the same time, that would not work. + assert!(Arc::ptr_eq(&l.unwrap(), &frozen_layer)); + + // Add the new delta layer to the LayerMap + layers.insert_historic(Arc::new(new_delta)); + + // release lock on 'layers' } - // Call unload() on all frozen layers, to release memory. - // This shouldn't be much memory, as only metadata is slurped - // into memory. - for layer in layers.iter_historic_layers() { - layer.unload()?; - } - - drop(layers); - drop(write_guard); - - // Create delta/image layers for evicted layers - for (_evicted_layer_id, evicted_layer) in evicted_layers.iter() { - let mut this_layer_paths = - self.evict_layer(evicted_layer.clone(), reconstruct_pages)?; - layer_paths.append(&mut this_layer_paths); - } - - // Sync layers - if !layer_paths.is_empty() { - // We must fsync the timeline dir to ensure the directory entries for - // new layer files are durable - layer_paths.push(self.conf.timeline_path(&self.timelineid, &self.tenantid)); - - // Fsync all the layer files and directory using multiple threads to - // minimize latency. - par_fsync::par_fsync(&layer_paths)?; - - layer_paths.pop().unwrap(); - } + // Update the metadata file, with new 'disk_consistent_lsn' + // + // TODO: This perhaps should be done in 'flush_frozen_layers', after flushing + // *all* the layers, to avoid fsyncing the file multiple times. + let disk_consistent_lsn; + disk_consistent_lsn = Lsn(frozen_layer.get_lsn_range().end.0 - 1); // If we were able to advance 'disk_consistent_lsn', save it the metadata file. // After crash, we will restart WAL streaming and processing from that point. @@ -1595,6 +1478,10 @@ impl LayeredTimeline { // don't remember what the correct value that corresponds to some old // LSN is. But if we flush everything, then the value corresponding // current 'last_record_lsn' is correct and we can store it on disk. + let RecordLsn { + last: last_record_lsn, + prev: prev_record_lsn, + } = self.last_record_lsn.load(); let ondisk_prev_record_lsn = if disk_consistent_lsn == last_record_lsn { Some(prev_record_lsn) } else { @@ -1615,6 +1502,11 @@ impl LayeredTimeline { self.initdb_lsn, ); + fail_point!("checkpoint-before-saving-metadata", |x| bail!( + "{}", + x.unwrap() + )); + LayeredRepository::save_metadata( self.conf, self.timelineid, @@ -1622,11 +1514,11 @@ impl LayeredTimeline { &metadata, false, )?; - if self.upload_relishes.load(atomic::Ordering::Relaxed) { + if self.upload_layers.load(atomic::Ordering::Relaxed) { schedule_timeline_checkpoint_upload( self.tenantid, self.timelineid, - layer_paths, + vec![new_delta_path], metadata, ); } @@ -1638,34 +1530,273 @@ impl LayeredTimeline { Ok(()) } - fn evict_layer( - &self, - layer: Arc, - reconstruct_pages: bool, - ) -> Result> { - let new_historics = layer.write_to_disk(self, reconstruct_pages)?; + pub fn compact(&self) -> Result<()> { + // + // High level strategy for compaction / image creation: + // + // 1. First, calculate the desired "partitioning" of the + // currently in-use key space. The goal is to partition the + // key space into roughly fixed-size chunks, but also take into + // account any existing image layers, and try to align the + // chunk boundaries with the existing image layers to avoid + // too much churn. Also try to align chunk boundaries with + // relation boundaries. In principle, we don't know about + // relation boundaries here, we just deal with key-value + // pairs, and the code in pgdatadir_mapping.rs knows how to + // map relations into key-value pairs. But in practice we know + // that 'field6' is the block number, and the fields 1-5 + // identify a relation. This is just an optimization, + // though. + // + // 2. Once we know the partitioning, for each partition, + // decide if it's time to create a new image layer. The + // criteria is: there has been too much "churn" since the last + // image layer? The "churn" is fuzzy concept, it's a + // combination of too many delta files, or too much WAL in + // total in the delta file. Or perhaps: if creating an image + // file would allow to delete some older files. + // + // 3. After that, we compact all level0 delta files if there + // are too many of them. While compacting, we also garbage + // collect any page versions that are no longer needed because + // of the new image layers we created in step 2. + // + // TODO: This hight level strategy hasn't been implemented yet. + // Below are functions compact_level0() and create_image_layers() + // but they are a bit ad hoc and don't quite work like it's explained + // above. Rewrite it. + let _compaction_cs = self.compaction_cs.lock().unwrap(); - let mut layer_paths = Vec::new(); - let _write_guard = self.write_lock.lock().unwrap(); - let mut layers = self.layers.lock().unwrap(); + let target_file_size = self.conf.checkpoint_distance; - // Finally, replace the frozen in-memory layer with the new on-disk layers - layers.remove_historic(layer); + // 1. The partitioning was already done by the code in + // pgdatadir_mapping.rs. We just use it here. + let partitioning_guard = self.partitioning.read().unwrap(); + if let Some((partitioning, lsn)) = partitioning_guard.as_ref() { + let timer = self.create_images_time_histo.start_timer(); + // Make a copy of the partitioning, so that we can release + // the lock. Otherwise we could block the WAL receiver. + let lsn = *lsn; + let parts = partitioning.parts.clone(); + drop(partitioning_guard); - // Add the historics to the LayerMap - for delta_layer in new_historics.delta_layers { - layer_paths.push(delta_layer.path()); - layers.insert_historic(Arc::new(delta_layer)); + // 2. Create new image layers for partitions that have been modified + // "enough". + for part in parts.iter() { + if self.time_for_new_image_layer(part, lsn, 3)? { + self.create_image_layer(part, lsn)?; + } + } + timer.stop_and_record(); + + // 3. Compact + let timer = self.compact_time_histo.start_timer(); + self.compact_level0(target_file_size)?; + timer.stop_and_record(); + } else { + info!("Could not compact because no partitioning specified yet"); } - for image_layer in new_historics.image_layers { - layer_paths.push(image_layer.path()); - layers.insert_historic(Arc::new(image_layer)); + + // Call unload() on all frozen layers, to release memory. + // This shouldn't be much memory, as only metadata is slurped + // into memory. + let layers = self.layers.lock().unwrap(); + for layer in layers.iter_historic_layers() { + layer.unload()?; } - Ok(layer_paths) + drop(layers); + + Ok(()) } + // Is it time to create a new image layer for the given partition? + fn time_for_new_image_layer( + &self, + partition: &KeySpace, + lsn: Lsn, + threshold: usize, + ) -> Result { + let layers = self.layers.lock().unwrap(); + + for part_range in &partition.ranges { + let image_coverage = layers.image_coverage(part_range, lsn)?; + for (img_range, last_img) in image_coverage { + let img_lsn = if let Some(ref last_img) = last_img { + last_img.get_lsn_range().end + } else { + Lsn(0) + }; + + let num_deltas = layers.count_deltas(&img_range, &(img_lsn..lsn))?; + + info!( + "range {}-{}, has {} deltas on this timeline", + img_range.start, img_range.end, num_deltas + ); + if num_deltas >= threshold { + return Ok(true); + } + } + } + + Ok(false) + } + + fn create_image_layer(&self, partition: &KeySpace, lsn: Lsn) -> Result<()> { + let img_range = + partition.ranges.first().unwrap().start..partition.ranges.last().unwrap().end; + let mut image_layer_writer = + ImageLayerWriter::new(self.conf, self.timelineid, self.tenantid, &img_range, lsn)?; + + for range in &partition.ranges { + let mut key = range.start; + while key < range.end { + let img = self.get(key, lsn)?; + image_layer_writer.put_image(key, &img)?; + key = key.next(); + } + } + let image_layer = image_layer_writer.finish()?; + + // Sync the new layer to disk before adding it to the layer map, to make sure + // we don't garbage collect something based on the new layer, before it has + // reached the disk. + // + // We must also fsync the timeline dir to ensure the directory entries for + // new layer files are durable + // + // Compaction creates multiple image layers. It would be better to create them all + // and fsync them all in parallel. + par_fsync::par_fsync(&[ + image_layer.path(), + self.conf.timeline_path(&self.timelineid, &self.tenantid), + ])?; + + // FIXME: Do we need to do something to upload it to remote storage here? + + let mut layers = self.layers.lock().unwrap(); + layers.insert_historic(Arc::new(image_layer)); + drop(layers); + + Ok(()) + } + + fn compact_level0(&self, target_file_size: u64) -> Result<()> { + let layers = self.layers.lock().unwrap(); + + // We compact or "shuffle" the level-0 delta layers when 10 have + // accumulated. + static COMPACT_THRESHOLD: usize = 10; + + let level0_deltas = layers.get_level0_deltas()?; + + if level0_deltas.len() < COMPACT_THRESHOLD { + return Ok(()); + } + drop(layers); + + // FIXME: this function probably won't work correctly if there's overlap + // in the deltas. + let lsn_range = level0_deltas + .iter() + .map(|l| l.get_lsn_range()) + .reduce(|a, b| min(a.start, b.start)..max(a.end, b.end)) + .unwrap(); + + let all_values_iter = level0_deltas.iter().map(|l| l.iter()).kmerge_by(|a, b| { + if let Ok((a_key, a_lsn, _)) = a { + if let Ok((b_key, b_lsn, _)) = b { + match a_key.cmp(b_key) { + Ordering::Less => true, + Ordering::Equal => a_lsn <= b_lsn, + Ordering::Greater => false, + } + } else { + false + } + } else { + true + } + }); + + // Merge the contents of all the input delta layers into a new set + // of delta layers, based on the current partitioning. + // + // TODO: this actually divides the layers into fixed-size chunks, not + // based on the partitioning. + // + // TODO: we should also opportunistically materialize and + // garbage collect what we can. + let mut new_layers = Vec::new(); + let mut prev_key: Option = None; + let mut writer: Option = None; + for x in all_values_iter { + let (key, lsn, value) = x?; + + if let Some(prev_key) = prev_key { + if key != prev_key && writer.is_some() { + let size = writer.as_mut().unwrap().size(); + if size > target_file_size { + new_layers.push(writer.take().unwrap().finish(prev_key.next())?); + writer = None; + } + } + } + + if writer.is_none() { + writer = Some(DeltaLayerWriter::new( + self.conf, + self.timelineid, + self.tenantid, + key, + lsn_range.clone(), + )?); + } + + writer.as_mut().unwrap().put_value(key, lsn, value)?; + prev_key = Some(key); + } + if let Some(writer) = writer { + new_layers.push(writer.finish(prev_key.unwrap().next())?); + } + + // Sync layers + if !new_layers.is_empty() { + let mut layer_paths: Vec = new_layers.iter().map(|l| l.path()).collect(); + + // also sync the directory + layer_paths.push(self.conf.timeline_path(&self.timelineid, &self.tenantid)); + + // Fsync all the layer files and directory using multiple threads to + // minimize latency. + par_fsync::par_fsync(&layer_paths)?; + + layer_paths.pop().unwrap(); + } + + let mut layers = self.layers.lock().unwrap(); + for l in new_layers { + layers.insert_historic(Arc::new(l)); + } + + // Now that we have reshuffled the data to set of new delta layers, we can + // delete the old ones + for l in level0_deltas { + l.delete()?; + layers.remove_historic(l.clone()); + } + drop(layers); + + Ok(()) + } + + /// Update information about which layer files need to be retained on + /// garbage collection. This is separate from actually performing the GC, + /// and is updated more frequently, so that compaction can remove obsolete + /// page versions more aggressively. /// - /// Garbage collect layer files on a timeline that are no longer needed. + /// TODO: that's wishful thinking, compaction doesn't actually do that + /// currently. /// /// The caller specifies how much history is needed with the two arguments: /// @@ -1682,15 +1813,29 @@ impl LayeredTimeline { /// the latest LSN subtracted by a constant, and doesn't do anything smart /// to figure out what read-only nodes might actually need.) /// + fn update_gc_info(&self, retain_lsns: Vec, cutoff: Lsn) { + let mut gc_info = self.gc_info.write().unwrap(); + gc_info.retain_lsns = retain_lsns; + gc_info.cutoff = cutoff; + } + + /// + /// Garbage collect layer files on a timeline that are no longer needed. + /// /// Currently, we don't make any attempt at removing unneeded page versions /// within a layer file. We can only remove the whole file if it's fully /// obsolete. /// - pub fn gc_timeline(&self, retain_lsns: Vec, cutoff: Lsn) -> Result { + fn gc(&self) -> Result { let now = Instant::now(); let mut result: GcResult = Default::default(); let disk_consistent_lsn = self.get_disk_consistent_lsn(); - let _checkpoint_cs = self.checkpoint_cs.lock().unwrap(); + + let _compaction_cs = self.compaction_cs.lock().unwrap(); + + let gc_info = self.gc_info.read().unwrap(); + let retain_lsns = &gc_info.retain_lsns; + let cutoff = gc_info.cutoff; let _enter = info_span!("garbage collection", timeline = %self.timelineid, tenant = %self.tenantid, cutoff = %cutoff).entered(); @@ -1709,8 +1854,7 @@ impl LayeredTimeline { // Garbage collect the layer if all conditions are satisfied: // 1. it is older than cutoff LSN; // 2. it doesn't need to be retained for 'retain_lsns'; - // 3. newer on-disk layer exists (only for non-dropped segments); - // 4. this layer doesn't serve as a tombstone for some older layer; + // 3. newer on-disk image layers cover the layer's whole key range // let mut layers = self.layers.lock().unwrap(); 'outer: for l in layers.iter_historic_layers() { @@ -1724,28 +1868,16 @@ impl LayeredTimeline { continue; } - let seg = l.get_seg_tag(); - - if seg.rel.is_relation() { - result.ondisk_relfiles_total += 1; - } else { - result.ondisk_nonrelfiles_total += 1; - } + result.layers_total += 1; // 1. Is it newer than cutoff point? - if l.get_end_lsn() > cutoff { + if l.get_lsn_range().end > cutoff { debug!( - "keeping {} {}-{} because it's newer than cutoff {}", - seg, - l.get_start_lsn(), - l.get_end_lsn(), + "keeping {} because it's newer than cutoff {}", + l.filename().display(), cutoff ); - if seg.rel.is_relation() { - result.ondisk_relfiles_needed_by_cutoff += 1; - } else { - result.ondisk_nonrelfiles_needed_by_cutoff += 1; - } + result.layers_needed_by_cutoff += 1; continue 'outer; } @@ -1754,135 +1886,49 @@ impl LayeredTimeline { // might be referenced by child branches forever. // We can track this in child timeline GC and delete parent layers when // they are no longer needed. This might be complicated with long inheritance chains. - for retain_lsn in &retain_lsns { + for retain_lsn in retain_lsns { // start_lsn is inclusive - if &l.get_start_lsn() <= retain_lsn { + if &l.get_lsn_range().start <= retain_lsn { debug!( - "keeping {} {}-{} because it's still might be referenced by child branch forked at {} is_dropped: {} is_incremental: {}", - seg, - l.get_start_lsn(), - l.get_end_lsn(), + "keeping {} because it's still might be referenced by child branch forked at {} is_dropped: xx is_incremental: {}", + l.filename().display(), retain_lsn, - l.is_dropped(), l.is_incremental(), ); - if seg.rel.is_relation() { - result.ondisk_relfiles_needed_by_branches += 1; - } else { - result.ondisk_nonrelfiles_needed_by_branches += 1; - } + result.layers_needed_by_branches += 1; continue 'outer; } } // 3. Is there a later on-disk layer for this relation? - if !l.is_dropped() - && !layers.newer_image_layer_exists( - l.get_seg_tag(), - l.get_end_lsn(), - disk_consistent_lsn, - ) - { + // + // The end-LSN is exclusive, while disk_consistent_lsn is + // inclusive. For example, if disk_consistent_lsn is 100, it is + // OK for a delta layer to have end LSN 101, but if the end LSN + // is 102, then it might not have been fully flushed to disk + // before crash. + // + // FIXME: This logic is wrong. See https://github.com/zenithdb/zenith/issues/707 + if !layers.newer_image_layer_exists( + &l.get_key_range(), + l.get_lsn_range().end, + disk_consistent_lsn + 1, + )? { debug!( - "keeping {} {}-{} because it is the latest layer", - seg, - l.get_start_lsn(), - l.get_end_lsn() + "keeping {} because it is the latest layer", + l.filename().display() ); - if seg.rel.is_relation() { - result.ondisk_relfiles_not_updated += 1; - } else { - result.ondisk_nonrelfiles_not_updated += 1; - } + result.layers_not_updated += 1; continue 'outer; } - // 4. Does this layer serve as a tombstone for some older layer? - if l.is_dropped() { - let prior_lsn = l.get_start_lsn().checked_sub(1u64).unwrap(); - - // Check if this layer serves as a tombstone for this timeline - // We have to do this separately from timeline check below, - // because LayerMap of this timeline is already locked. - let mut is_tombstone = layers.layer_exists_at_lsn(l.get_seg_tag(), prior_lsn)?; - if is_tombstone { - debug!( - "earlier layer exists at {} in {}", - prior_lsn, self.timelineid - ); - } - // Now check ancestor timelines, if any are present locally - else if let Some(ancestor) = self - .ancestor_timeline - .as_ref() - .and_then(|timeline_entry| timeline_entry.ensure_loaded().ok()) - { - let prior_lsn = ancestor.get_last_record_lsn(); - if seg.rel.is_blocky() { - debug!( - "check blocky relish size {} at {} in {} for layer {}-{}", - seg, - prior_lsn, - ancestor.timelineid, - l.get_start_lsn(), - l.get_end_lsn() - ); - match ancestor.get_relish_size(seg.rel, prior_lsn).unwrap() { - Some(size) => { - let (last_live_seg, _rel_blknum) = - SegmentTag::from_blknum(seg.rel, size - 1); - debug!( - "blocky rel size is {} last_live_seg.segno {} seg.segno {}", - size, last_live_seg.segno, seg.segno - ); - if last_live_seg.segno >= seg.segno { - is_tombstone = true; - } - } - _ => { - debug!("blocky rel doesn't exist"); - } - } - } else { - debug!( - "check non-blocky relish existence {} at {} in {} for layer {}-{}", - seg, - prior_lsn, - ancestor.timelineid, - l.get_start_lsn(), - l.get_end_lsn() - ); - is_tombstone = ancestor.get_rel_exists(seg.rel, prior_lsn).unwrap_or(false); - } - } - - if is_tombstone { - debug!( - "keeping {} {}-{} because this layer serves as a tombstone for older layer", - seg, - l.get_start_lsn(), - l.get_end_lsn() - ); - - if seg.rel.is_relation() { - result.ondisk_relfiles_needed_as_tombstone += 1; - } else { - result.ondisk_nonrelfiles_needed_as_tombstone += 1; - } - continue 'outer; - } - } - // We didn't find any reason to keep this file, so remove it. debug!( - "garbage collecting {} {}-{} is_dropped: {} is_incremental: {}", - l.get_seg_tag(), - l.get_start_lsn(), - l.get_end_lsn(), - l.is_dropped(), + "garbage collecting {} is_dropped: xx is_incremental: {}", + l.filename().display(), l.is_incremental(), ); - layers_to_remove.push(Arc::clone(&l)); + layers_to_remove.push(Arc::clone(l)); } // Actually delete the layers from disk and remove them from the map. @@ -1892,222 +1938,75 @@ impl LayeredTimeline { doomed_layer.delete()?; layers.remove_historic(doomed_layer.clone()); - match ( - doomed_layer.is_dropped(), - doomed_layer.get_seg_tag().rel.is_relation(), - ) { - (true, true) => result.ondisk_relfiles_dropped += 1, - (true, false) => result.ondisk_nonrelfiles_dropped += 1, - (false, true) => result.ondisk_relfiles_removed += 1, - (false, false) => result.ondisk_nonrelfiles_removed += 1, - } + result.layers_removed += 1; } result.elapsed = now.elapsed(); Ok(result) } - fn lookup_cached_page( + /// + /// Reconstruct a value, using the given base image and WAL records in 'data'. + /// + fn reconstruct_value( &self, - rel: &RelishTag, - rel_blknum: BlockNumber, - lsn: Lsn, - ) -> Option<(Lsn, Bytes)> { - let cache = page_cache::get(); - if let RelishTag::Relation(rel_tag) = &rel { - let (lsn, read_guard) = cache.lookup_materialized_page( - self.tenantid, - self.timelineid, - *rel_tag, - rel_blknum, - lsn, - )?; - let img = Bytes::from(read_guard.to_vec()); - Some((lsn, img)) - } else { - None - } - } - - /// - /// Reconstruct a page version from given Layer - /// - fn materialize_page( - &self, - seg: SegmentTag, - seg_blknum: SegmentBlk, - lsn: Lsn, - layer: &dyn Layer, - ) -> anyhow::Result { - // Check the page cache. We will get back the most recent page with lsn <= `lsn`. - // The cached image can be returned directly if there is no WAL between the cached image - // and requested LSN. The cached image can also be used to reduce the amount of WAL needed - // for redo. - let rel = seg.rel; - let rel_blknum = seg.segno * RELISH_SEG_SIZE + seg_blknum; - let cached_page_img = match self.lookup_cached_page(&rel, rel_blknum, lsn) { - Some((cached_lsn, cached_img)) => { - match cached_lsn.cmp(&lsn) { - cmp::Ordering::Less => {} // there might be WAL between cached_lsn and lsn, we need to check - cmp::Ordering::Equal => return Ok(cached_img), // exact LSN match, return the image - cmp::Ordering::Greater => { - bail!("the returned lsn should never be after the requested lsn") - } - } - Some((cached_lsn, cached_img)) - } - None => None, - }; - - let mut data = PageReconstructData { - records: Vec::new(), - page_img: cached_page_img, - }; - - // Holds an Arc reference to 'layer_ref' when iterating in the loop below. - let mut layer_arc: Arc; - - // Call the layer's get_page_reconstruct_data function to get the base image - // and WAL records needed to materialize the page. If it returns 'Continue', - // call it again on the predecessor layer until we have all the required data. - let mut layer_ref = layer; - let mut curr_lsn = lsn; - loop { - let result = self.reconstruct_time_histo.observe_closure_duration(|| { - layer_ref - .get_page_reconstruct_data(seg_blknum, curr_lsn, &mut data) - .with_context(|| { - format!( - "Failed to get reconstruct data {} {:?} {} {}", - layer_ref.get_seg_tag(), - layer_ref.filename(), - seg_blknum, - curr_lsn, - ) - }) - })?; - match result { - PageReconstructResult::Complete => break, - PageReconstructResult::Continue(cont_lsn) => { - // Fetch base image / more WAL from the returned predecessor layer - if let Some((cont_layer, cont_lsn)) = self.get_layer_for_read(seg, cont_lsn)? { - if cont_lsn == curr_lsn { - // We landed on the same layer again. Shouldn't happen, but if it does, - // don't get stuck in an infinite loop. - bail!( - "could not find predecessor of layer {} at {}, layer returned its own LSN", - layer_ref.filename().display(), - cont_lsn - ); - } - layer_arc = cont_layer; - layer_ref = &*layer_arc; - curr_lsn = cont_lsn; - continue; - } else { - bail!( - "could not find predecessor of layer {} at {}", - layer_ref.filename().display(), - cont_lsn - ); - } - } - PageReconstructResult::Missing(lsn) => { - // Oops, we could not reconstruct the page. - if data.records.is_empty() { - // no records, and no base image. This can happen if PostgreSQL extends a relation - // but never writes the page. - // - // Would be nice to detect that situation better. - warn!("Page {} blk {} at {} not found", rel, rel_blknum, lsn); - return Ok(ZERO_PAGE.clone()); - } - bail!( - "No base image found for page {} blk {} at {}/{}", - rel, - rel_blknum, - self.timelineid, - lsn, - ); - } - } - } - - self.reconstruct_page(rel, rel_blknum, lsn, data) - } - - /// - /// Reconstruct a page version, using the given base image and WAL records in 'data'. - /// - fn reconstruct_page( - &self, - rel: RelishTag, - rel_blknum: BlockNumber, + key: Key, request_lsn: Lsn, - mut data: PageReconstructData, + mut data: ValueReconstructState, ) -> Result { // Perform WAL redo if needed data.records.reverse(); // If we have a page image, and no WAL, we're all set if data.records.is_empty() { - if let Some((img_lsn, img)) = &data.page_img { + if let Some((img_lsn, img)) = &data.img { trace!( - "found page image for blk {} in {} at {}, no WAL redo required", - rel_blknum, - rel, + "found page image for key {} at {}, no WAL redo required", + key, img_lsn ); Ok(img.clone()) } else { - // FIXME: this ought to be an error? - warn!( - "Page {} blk {} at {} not found", - rel, rel_blknum, request_lsn - ); - Ok(ZERO_PAGE.clone()) + bail!("base image for {} at {} not found", key, request_lsn); } } else { // We need to do WAL redo. // // If we don't have a base image, then the oldest WAL record better initialize // the page - if data.page_img.is_none() && !data.records.first().unwrap().1.will_init() { - // FIXME: this ought to be an error? - warn!( - "Base image for page {}/{} at {} not found, but got {} WAL records", - rel, - rel_blknum, + if data.img.is_none() && !data.records.first().unwrap().1.will_init() { + bail!( + "Base image for {} at {} not found, but got {} WAL records", + key, request_lsn, data.records.len() ); - Ok(ZERO_PAGE.clone()) } else { - let base_img = if let Some((_lsn, img)) = data.page_img { - trace!("found {} WAL records and a base image for blk {} in {} at {}, performing WAL redo", data.records.len(), rel_blknum, rel, request_lsn); + let base_img = if let Some((_lsn, img)) = data.img { + trace!( + "found {} WAL records and a base image for {} at {}, performing WAL redo", + data.records.len(), + key, + request_lsn + ); Some(img) } else { - trace!("found {} WAL records that will init the page for blk {} in {} at {}, performing WAL redo", data.records.len(), rel_blknum, rel, request_lsn); + trace!("found {} WAL records that will init the page for {} at {}, performing WAL redo", data.records.len(), key, request_lsn); None }; let last_rec_lsn = data.records.last().unwrap().0; - let img = self.walredo_mgr.request_redo( - rel, - rel_blknum, - request_lsn, - base_img, - data.records, - )?; + let img = + self.walredo_mgr + .request_redo(key, request_lsn, base_img, data.records)?; - if let RelishTag::Relation(rel_tag) = &rel { + if img.len() == page_cache::PAGE_SZ { let cache = page_cache::get(); cache.memorize_materialized_page( self.tenantid, self.timelineid, - *rel_tag, - rel_blknum, + key, last_rec_lsn, &img, ); @@ -2117,40 +2016,6 @@ impl LayeredTimeline { } } } - - /// - /// This is a helper function to increase current_total_relation_size - /// - fn increase_current_logical_size(&self, diff: u32) { - let val = self - .current_logical_size - .fetch_add(diff as usize, atomic::Ordering::SeqCst); - trace!( - "increase_current_logical_size: {} + {} = {}", - val, - diff, - val + diff as usize, - ); - self.current_logical_size_gauge - .set(val as i64 + diff as i64); - } - - /// - /// This is a helper function to decrease current_total_relation_size - /// - fn decrease_current_logical_size(&self, diff: u32) { - let val = self - .current_logical_size - .fetch_sub(diff as usize, atomic::Ordering::SeqCst); - trace!( - "decrease_current_logical_size: {} - {} = {}", - val, - diff, - val - diff as usize, - ); - self.current_logical_size_gauge - .set(val as i64 - diff as i64); - } } struct LayeredTimelineWriter<'a> { @@ -2166,159 +2031,20 @@ impl Deref for LayeredTimelineWriter<'_> { } } -impl<'a> TimelineWriter for LayeredTimelineWriter<'a> { - fn put_wal_record( - &self, - lsn: Lsn, - rel: RelishTag, - rel_blknum: u32, - rec: ZenithWalRecord, - ) -> Result<()> { - if !rel.is_blocky() && rel_blknum != 0 { - bail!( - "invalid request for block {} for non-blocky relish {}", - rel_blknum, - rel - ); - } - ensure!(lsn.is_aligned(), "unaligned record LSN"); - - let (seg, seg_blknum) = SegmentTag::from_blknum(rel, rel_blknum); - let layer = self.tl.get_layer_for_write(seg, lsn)?; - let delta_size = layer.put_wal_record(lsn, seg_blknum, rec)?; - self.tl - .increase_current_logical_size(delta_size * BLCKSZ as u32); - Ok(()) +impl<'a> TimelineWriter<'_> for LayeredTimelineWriter<'a> { + fn put(&self, key: Key, lsn: Lsn, value: Value) -> Result<()> { + self.tl.put_value(key, lsn, value) } - fn put_page_image( - &self, - rel: RelishTag, - rel_blknum: BlockNumber, - lsn: Lsn, - img: Bytes, - ) -> Result<()> { - if !rel.is_blocky() && rel_blknum != 0 { - bail!( - "invalid request for block {} for non-blocky relish {}", - rel_blknum, - rel - ); - } - ensure!(lsn.is_aligned(), "unaligned record LSN"); - - let (seg, seg_blknum) = SegmentTag::from_blknum(rel, rel_blknum); - - let layer = self.tl.get_layer_for_write(seg, lsn)?; - let delta_size = layer.put_page_image(seg_blknum, lsn, img)?; - - self.tl - .increase_current_logical_size(delta_size * BLCKSZ as u32); - Ok(()) - } - - fn put_truncation(&self, rel: RelishTag, lsn: Lsn, relsize: BlockNumber) -> Result<()> { - if !rel.is_blocky() { - bail!("invalid truncation for non-blocky relish {}", rel); - } - ensure!(lsn.is_aligned(), "unaligned record LSN"); - - debug!("put_truncation: {} to {} blocks at {}", rel, relsize, lsn); - - let oldsize = self - .tl - .get_relish_size(rel, self.tl.get_last_record_lsn())? - .with_context(|| { - format!( - "attempted to truncate non-existent relish {} at {}", - rel, lsn - ) - })?; - - if oldsize <= relsize { - return Ok(()); - } - let old_last_seg = (oldsize - 1) / RELISH_SEG_SIZE; - - let last_remain_seg = if relsize == 0 { - 0 - } else { - (relsize - 1) / RELISH_SEG_SIZE - }; - - // Drop segments beyond the last remaining segment. - for remove_segno in (last_remain_seg + 1)..=old_last_seg { - let seg = SegmentTag { - rel, - segno: remove_segno, - }; - - let layer = self.tl.get_layer_for_write(seg, lsn)?; - layer.drop_segment(lsn); - } - - // Truncate the last remaining segment to the specified size - if relsize == 0 || relsize % RELISH_SEG_SIZE != 0 { - let seg = SegmentTag { - rel, - segno: last_remain_seg, - }; - let layer = self.tl.get_layer_for_write(seg, lsn)?; - layer.put_truncation(lsn, relsize % RELISH_SEG_SIZE) - } - self.tl - .decrease_current_logical_size((oldsize - relsize) * BLCKSZ as u32); - Ok(()) - } - - fn drop_relish(&self, rel: RelishTag, lsn: Lsn) -> Result<()> { - trace!("drop_segment: {} at {}", rel, lsn); - - if rel.is_blocky() { - if let Some(oldsize) = self - .tl - .get_relish_size(rel, self.tl.get_last_record_lsn())? - { - let old_last_seg = if oldsize == 0 { - 0 - } else { - (oldsize - 1) / RELISH_SEG_SIZE - }; - - // Drop all segments of the relish - for remove_segno in 0..=old_last_seg { - let seg = SegmentTag { - rel, - segno: remove_segno, - }; - let layer = self.tl.get_layer_for_write(seg, lsn)?; - layer.drop_segment(lsn); - } - self.tl - .decrease_current_logical_size(oldsize * BLCKSZ as u32); - } else { - warn!( - "drop_segment called on non-existent relish {} at {}", - rel, lsn - ); - } - } else { - // TODO handle TwoPhase relishes - let (seg, _seg_blknum) = SegmentTag::from_blknum(rel, 0); - let layer = self.tl.get_layer_for_write(seg, lsn)?; - layer.drop_segment(lsn); - } - - Ok(()) + fn delete(&self, key_range: Range, lsn: Lsn) -> Result<()> { + self.tl.put_tombstone(key_range, lsn) } /// /// Remember the (end of) last valid WAL record remembered in the timeline. /// - fn advance_last_record_lsn(&self, new_lsn: Lsn) { - assert!(new_lsn.is_aligned()); - - self.tl.last_record_lsn.advance(new_lsn); + fn finish_write(&self, new_lsn: Lsn) { + self.tl.finish_write(new_lsn); } } @@ -2328,10 +2054,10 @@ pub fn dump_layerfile_from_path(path: &Path) -> Result<()> { let book = Book::new(file)?; match book.magic() { - delta_layer::DELTA_FILE_MAGIC => { + crate::DELTA_FILE_MAGIC => { DeltaLayer::new_for_path(path, &book)?.dump()?; } - image_layer::IMAGE_FILE_MAGIC => { + crate::IMAGE_FILE_MAGIC => { ImageLayer::new_for_path(path, &book)?.dump()?; } magic => bail!("unrecognized magic identifier: {:?}", magic), @@ -2368,9 +2094,11 @@ fn rename_to_backup(path: PathBuf) -> anyhow::Result<()> { /// file format and directory layout. The test here are more low level. /// #[cfg(test)] -mod tests { +pub mod tests { use super::*; + use crate::keyspace::KeySpaceAccum; use crate::repository::repo_harness::*; + use rand::{thread_rng, Rng}; #[test] fn corrupt_metadata() -> Result<()> { @@ -2387,7 +2115,7 @@ mod tests { let mut metadata_bytes = std::fs::read(&metadata_path)?; assert_eq!(metadata_bytes.len(), 512); - metadata_bytes[512 - 4 - 2] ^= 1; + metadata_bytes[8] ^= 1; std::fs::write(metadata_path, metadata_bytes)?; let err = harness.try_load().err().expect("should fail"); @@ -2400,113 +2128,259 @@ mod tests { Ok(()) } - /// - /// Test the logic in 'load_layer_map' that removes layer files that are - /// newer than 'disk_consistent_lsn'. - /// + // Target file size in the unit tests. In production, the target + // file size is much larger, maybe 1 GB. But a small size makes it + // much faster to exercise all the logic for creating the files, + // garbage collection, compaction etc. + pub const TEST_FILE_SIZE: u64 = 4 * 1024 * 1024; + #[test] - fn future_layerfiles() -> Result<()> { - const TEST_NAME: &str = "future_layerfiles"; - let harness = RepoHarness::create(TEST_NAME)?; - let repo = harness.load(); + fn test_images() -> Result<()> { + let repo = RepoHarness::create("test_images")?.load(); + let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; + + #[allow(non_snake_case)] + let TEST_KEY: Key = Key::from_hex("112222222233333333444444445500000001").unwrap(); - // Create a timeline with disk_consistent_lsn = 8000 - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0x8000))?; let writer = tline.writer(); - writer.advance_last_record_lsn(Lsn(0x8000)); + writer.put(TEST_KEY, Lsn(0x10), Value::Image(TEST_IMG("foo at 0x10")))?; + writer.finish_write(Lsn(0x10)); drop(writer); - repo.checkpoint_iteration(CheckpointConfig::Forced)?; - drop(repo); - let timeline_path = harness.timeline_path(&TIMELINE_ID); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; - let make_empty_file = |filename: &str| -> std::io::Result<()> { - let path = timeline_path.join(filename); + let writer = tline.writer(); + writer.put(TEST_KEY, Lsn(0x20), Value::Image(TEST_IMG("foo at 0x20")))?; + writer.finish_write(Lsn(0x20)); + drop(writer); - assert!(!path.exists()); - std::fs::write(&path, &[])?; + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; - Ok(()) - }; + let writer = tline.writer(); + writer.put(TEST_KEY, Lsn(0x30), Value::Image(TEST_IMG("foo at 0x30")))?; + writer.finish_write(Lsn(0x30)); + drop(writer); - // Helper function to check that a relation file exists, and a corresponding - // .0.old file does not. - let assert_exists = |filename: &str| { - let path = timeline_path.join(filename); - assert!(path.exists(), "file {} was removed", filename); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; - // Check that there is no .old file - let backup_path = timeline_path.join(format!("{}.0.old", filename)); - assert!( - !backup_path.exists(), - "unexpected backup file {}", - backup_path.display() - ); - }; + let writer = tline.writer(); + writer.put(TEST_KEY, Lsn(0x40), Value::Image(TEST_IMG("foo at 0x40")))?; + writer.finish_write(Lsn(0x40)); + drop(writer); - // Helper function to check that a relation file does *not* exists, and a corresponding - // ..old file does. - let assert_is_renamed = |filename: &str, num: u32| { - let path = timeline_path.join(filename); - assert!( - !path.exists(), - "file {} was not removed as expected", - filename - ); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; - let backup_path = timeline_path.join(format!("{}.{}.old", filename, num)); - assert!( - backup_path.exists(), - "backup file {} was not created", - backup_path.display() - ); - }; + assert_eq!(tline.get(TEST_KEY, Lsn(0x10))?, TEST_IMG("foo at 0x10")); + assert_eq!(tline.get(TEST_KEY, Lsn(0x1f))?, TEST_IMG("foo at 0x10")); + assert_eq!(tline.get(TEST_KEY, Lsn(0x20))?, TEST_IMG("foo at 0x20")); + assert_eq!(tline.get(TEST_KEY, Lsn(0x30))?, TEST_IMG("foo at 0x30")); + assert_eq!(tline.get(TEST_KEY, Lsn(0x40))?, TEST_IMG("foo at 0x40")); - // These files are considered to be in the future and will be renamed out - // of the way - let future_filenames = vec![ - format!("pg_control_0_{:016X}", 0x8001), - format!("pg_control_0_{:016X}_{:016X}", 0x8001, 0x8008), - ]; - // But these are not: - let past_filenames = vec![ - format!("pg_control_0_{:016X}", 0x8000), - format!("pg_control_0_{:016X}_{:016X}", 0x7000, 0x8001), - ]; + Ok(()) + } - for filename in future_filenames.iter().chain(past_filenames.iter()) { - make_empty_file(filename)?; + // + // Insert 1000 key-value pairs with increasing keys, checkpoint, + // repeat 50 times. + // + #[test] + fn test_bulk_insert() -> Result<()> { + let repo = RepoHarness::create("test_bulk_insert")?.load(); + let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; + + let mut lsn = Lsn(0x10); + + let mut keyspace = KeySpaceAccum::new(); + + let mut test_key = Key::from_hex("012222222233333333444444445500000000").unwrap(); + let mut blknum = 0; + for _ in 0..50 { + for _ in 0..1000 { + test_key.field6 = blknum; + let writer = tline.writer(); + writer.put( + test_key, + lsn, + Value::Image(TEST_IMG(&format!("{} at {}", blknum, lsn))), + )?; + writer.finish_write(lsn); + drop(writer); + + keyspace.add_key(test_key); + + lsn = Lsn(lsn.0 + 0x10); + blknum += 1; + } + + let cutoff = tline.get_last_record_lsn(); + let parts = keyspace + .clone() + .to_keyspace() + .partition(TEST_FILE_SIZE as u64); + tline.hint_partitioning(parts.clone(), lsn)?; + + tline.update_gc_info(Vec::new(), cutoff); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; + tline.gc()?; } - // Load the timeline. This will cause the files in the "future" to be renamed - // away. - let new_repo = harness.load(); - new_repo.get_timeline_load(TIMELINE_ID).unwrap(); - drop(new_repo); + Ok(()) + } - for filename in future_filenames.iter() { - assert_is_renamed(filename, 0); - } - for filename in past_filenames.iter() { - assert_exists(filename); + #[test] + fn test_random_updates() -> Result<()> { + let repo = RepoHarness::create("test_random_updates")?.load(); + let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; + + const NUM_KEYS: usize = 1000; + + let mut test_key = Key::from_hex("012222222233333333444444445500000000").unwrap(); + + let mut keyspace = KeySpaceAccum::new(); + + // Track when each page was last modified. Used to assert that + // a read sees the latest page version. + let mut updated = [Lsn(0); NUM_KEYS]; + + let mut lsn = Lsn(0); + #[allow(clippy::needless_range_loop)] + for blknum in 0..NUM_KEYS { + lsn = Lsn(lsn.0 + 0x10); + test_key.field6 = blknum as u32; + let writer = tline.writer(); + writer.put( + test_key, + lsn, + Value::Image(TEST_IMG(&format!("{} at {}", blknum, lsn))), + )?; + writer.finish_write(lsn); + updated[blknum] = lsn; + drop(writer); + + keyspace.add_key(test_key); } - // Create the future files again, and load again. They should be renamed to - // *.1.old this time. - for filename in future_filenames.iter() { - make_empty_file(filename)?; + let parts = keyspace.to_keyspace().partition(TEST_FILE_SIZE as u64); + tline.hint_partitioning(parts, lsn)?; + + for _ in 0..50 { + for _ in 0..NUM_KEYS { + lsn = Lsn(lsn.0 + 0x10); + let blknum = thread_rng().gen_range(0..NUM_KEYS); + test_key.field6 = blknum as u32; + let writer = tline.writer(); + writer.put( + test_key, + lsn, + Value::Image(TEST_IMG(&format!("{} at {}", blknum, lsn))), + )?; + println!("updating {} at {}", blknum, lsn); + writer.finish_write(lsn); + drop(writer); + updated[blknum] = lsn; + } + + // Read all the blocks + for (blknum, last_lsn) in updated.iter().enumerate() { + test_key.field6 = blknum as u32; + assert_eq!( + tline.get(test_key, lsn)?, + TEST_IMG(&format!("{} at {}", blknum, last_lsn)) + ); + } + + // Perform a cycle of checkpoint, compaction, and GC + println!("checkpointing {}", lsn); + let cutoff = tline.get_last_record_lsn(); + tline.update_gc_info(Vec::new(), cutoff); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; + tline.gc()?; } - let new_repo = harness.load(); - new_repo.get_timeline_load(TIMELINE_ID).unwrap(); - drop(new_repo); + Ok(()) + } - for filename in future_filenames.iter() { - assert_is_renamed(filename, 0); - assert_is_renamed(filename, 1); + #[test] + fn test_traverse_branches() -> Result<()> { + let repo = RepoHarness::create("test_traverse_branches")?.load(); + let mut tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; + + const NUM_KEYS: usize = 1000; + + let mut test_key = Key::from_hex("012222222233333333444444445500000000").unwrap(); + + let mut keyspace = KeySpaceAccum::new(); + + // Track when each page was last modified. Used to assert that + // a read sees the latest page version. + let mut updated = [Lsn(0); NUM_KEYS]; + + let mut lsn = Lsn(0); + #[allow(clippy::needless_range_loop)] + for blknum in 0..NUM_KEYS { + lsn = Lsn(lsn.0 + 0x10); + test_key.field6 = blknum as u32; + let writer = tline.writer(); + writer.put( + test_key, + lsn, + Value::Image(TEST_IMG(&format!("{} at {}", blknum, lsn))), + )?; + writer.finish_write(lsn); + updated[blknum] = lsn; + drop(writer); + + keyspace.add_key(test_key); } - for filename in past_filenames.iter() { - assert_exists(filename); + + let parts = keyspace.to_keyspace().partition(TEST_FILE_SIZE as u64); + tline.hint_partitioning(parts, lsn)?; + + let mut tline_id = TIMELINE_ID; + for _ in 0..50 { + let new_tline_id = ZTimelineId::generate(); + repo.branch_timeline(tline_id, new_tline_id, lsn)?; + tline = repo.get_timeline_load(new_tline_id)?; + tline_id = new_tline_id; + + for _ in 0..NUM_KEYS { + lsn = Lsn(lsn.0 + 0x10); + let blknum = thread_rng().gen_range(0..NUM_KEYS); + test_key.field6 = blknum as u32; + let writer = tline.writer(); + writer.put( + test_key, + lsn, + Value::Image(TEST_IMG(&format!("{} at {}", blknum, lsn))), + )?; + println!("updating {} at {}", blknum, lsn); + writer.finish_write(lsn); + drop(writer); + updated[blknum] = lsn; + } + + // Read all the blocks + for (blknum, last_lsn) in updated.iter().enumerate() { + test_key.field6 = blknum as u32; + assert_eq!( + tline.get(test_key, lsn)?, + TEST_IMG(&format!("{} at {}", blknum, last_lsn)) + ); + } + + // Perform a cycle of checkpoint, compaction, and GC + println!("checkpointing {}", lsn); + let cutoff = tline.get_last_record_lsn(); + tline.update_gc_info(Vec::new(), cutoff); + tline.checkpoint(CheckpointConfig::Forced)?; + tline.compact()?; + tline.gc()?; } Ok(()) diff --git a/pageserver/src/layered_repository/README.md b/pageserver/src/layered_repository/README.md index 20f89ddc70..519478e417 100644 --- a/pageserver/src/layered_repository/README.md +++ b/pageserver/src/layered_repository/README.md @@ -1,40 +1,42 @@ # Overview -The on-disk format is based on immutable files. The page server receives a -stream of incoming WAL, parses the WAL records to determine which pages they -apply to, and accumulates the incoming changes in memory. Every now and then, -the accumulated changes are written out to new immutable files. This process is -called checkpointing. Old versions of on-disk files that are not needed by any -timeline are removed by GC process. - The main responsibility of the Page Server is to process the incoming WAL, and reprocess it into a format that allows reasonably quick access to any page -version. +version. The page server slices the incoming WAL per relation and page, and +packages the sliced WAL into suitably-sized "layer files". The layer files +contain all the history of the database, back to some reasonable retention +period. This system replaces the base backups and the WAL archive used in a +traditional PostgreSQL installation. The layer files are immutable, they are not +modified in-place after creation. New layer files are created for new incoming +WAL, and old layer files are removed when they are no longer needed. + +The on-disk format is based on immutable files. The page server receives a +stream of incoming WAL, parses the WAL records to determine which pages they +apply to, and accumulates the incoming changes in memory. Whenever enough WAL +has been accumulated in memory, it is written out to a new immutable file. That +process accumulates "L0 delta files" on disk. When enough L0 files have been +accumulated, they are merged and re-partitioned into L1 files, and old files +that are no longer needed are removed by Garbage Collection (GC). The incoming WAL contains updates to arbitrary pages in the system. The distribution depends on the workload: the updates could be totally random, or there could be a long stream of updates to a single relation when data is bulk -loaded, for example, or something in between. The page server slices the -incoming WAL per relation and page, and packages the sliced WAL into -suitably-sized "layer files". The layer files contain all the history of the -database, back to some reasonable retention period. This system replaces the -base backups and the WAL archive used in a traditional PostgreSQL -installation. The layer files are immutable, they are not modified in-place -after creation. New layer files are created for new incoming WAL, and old layer -files are removed when they are no longer needed. We could also replace layer -files with new files that contain the same information, merging small files for -example, but that hasn't been implemented yet. +loaded, for example, or something in between. +Cloud Storage Page Server Safekeeper + L1 L0 Memory WAL -Cloud Storage Page Server Safekeeper - Local disk Memory WAL - -|AAAA| |AAAA|AAAA| |AA -|BBBB| |BBBB|BBBB| | -|CCCC|CCCC| <---- |CCCC|CCCC|CCCC| <--- |CC <---- ADEBAABED -|DDDD|DDDD| |DDDD|DDDD| |DDD -|EEEE| |EEEE|EEEE|EEEE| |E - ++----+ +----+----+ +|AAAA| |AAAA|AAAA| +---+-----+ | ++----+ +----+----+ | | | |AA +|BBBB| |BBBB|BBBB| |BB | AA | |BB ++----+----+ +----+----+ |C | BB | |CC +|CCCC|CCCC| <---- |CCCC|CCCC| <--- |D | CC | <--- |DDD <---- ADEBAABED ++----+----+ +----+----+ | | DDD | |E +|DDDD|DDDD| |DDDD|DDDD| |E | | | ++----+----+ +----+----+ | | | +|EEEE| |EEEE|EEEE| +---+-----+ ++----+ +----+----+ In this illustration, WAL is received as a stream from the Safekeeper, from the right. It is immediately captured by the page server and stored quickly in @@ -42,39 +44,29 @@ memory. The page server memory can be thought of as a quick "reorder buffer", used to hold the incoming WAL and reorder it so that we keep the WAL records for the same page and relation close to each other. -From the page server memory, whenever enough WAL has been accumulated for one -relation segment, it is moved to local disk, as a new layer file, and the memory -is released. +From the page server memory, whenever enough WAL has been accumulated, it is flushed +to disk into a new L0 layer file, and the memory is released. + +When enough L0 files have been accumulated, they are merged together rand sliced +per key-space, producing a new set of files where each file contains a more +narrow key range, but larger LSN range. From the local disk, the layers are further copied to Cloud Storage, for long-term archival. After a layer has been copied to Cloud Storage, it can be removed from local disk, although we currently keep everything locally for fast access. If a layer is needed that isn't found locally, it is fetched from Cloud -Storage and stored in local disk. - -# Terms used in layered repository - -- Relish - one PostgreSQL relation or similarly treated file. -- Segment - one slice of a Relish that is stored in a LayeredTimeline. -- Layer - specific version of a relish Segment in a range of LSNs. +Storage and stored in local disk. L0 and L1 files are both uploaded to Cloud +Storage. # Layer map -The LayerMap tracks what layers exist for all the relishes in a timeline. - -LayerMap consists of two data structures: -- segs - All the layers keyed by segment tag -- open_layers - data structure that hold all open layers ordered by oldest_pending_lsn for quick access during checkpointing. oldest_pending_lsn is the LSN of the oldest page version stored in this layer. - -All operations that update InMemory Layers should update both structures to keep them up-to-date. - -- LayeredTimeline - implements Timeline interface. - -All methods of LayeredTimeline are aware of its ancestors and return data taking them into account. -TODO: Are there any exceptions to this? -For example, timeline.list_rels(lsn) will return all segments that are visible in this timeline at the LSN, -including ones that were not modified in this timeline and thus don't have a layer in the timeline's LayerMap. +The LayerMap tracks what layers exist in a timeline. +Currently, the layer map is just a resizeable array (Vec). On a GetPage@LSN or +other read request, the layer map scans through the array to find the right layer +that contains the data for the requested page. The read-code in LayeredTimeline +is aware of the ancestor, and returns data from the ancestor timeline if it's +not found on the current timeline. # Different kinds of layers @@ -92,11 +84,11 @@ To avoid OOM errors, InMemory layers can be spilled to disk into ephemeral file. TODO: Clarify the difference between Closed, Historic and Frozen. There are two kinds of OnDisk layers: -- ImageLayer represents an image or a snapshot of a 10 MB relish segment, at one particular LSN. -- DeltaLayer represents a collection of WAL records or page images in a range of LSNs, for one - relish segment. - -Dropped segments are always represented on disk by DeltaLayer. +- ImageLayer represents a snapshot of all the keys in a particular range, at one + particular LSN. Any keys that are not present in the ImageLayer are known not + to exist at that LSN. +- DeltaLayer represents a collection of WAL records or page images in a range of + LSNs, for a range of keys. # Layer life cycle @@ -109,71 +101,71 @@ layer or a delta layer, it is a valid end bound. An image layer represents snapshot at one LSN, so end_lsn is always the snapshot LSN + 1 Every layer starts its life as an Open In-Memory layer. When the page server -receives the first WAL record for a segment, it creates a new In-Memory layer -for it, and puts it to the layer map. Later, the layer is old enough, its -contents are written to disk, as On-Disk layers. This process is called -"evicting" a layer. +receives the first WAL record for a timeline, it creates a new In-Memory layer +for it, and puts it to the layer map. Later, when the layer becomes full, its +contents are written to disk, as an on-disk layers. -Layer eviction is a two-step process: First, the layer is marked as closed, so -that it no longer accepts new WAL records, and the layer map is updated -accordingly. If a new WAL record for that segment arrives after this step, a new -Open layer is created to hold it. After this first step, the layer is a Closed +Flushing a layer is a two-step process: First, the layer is marked as closed, so +that it no longer accepts new WAL records, and a new in-memory layer is created +to hold any WAL after that point. After this first step, the layer is a Closed InMemory state. This first step is called "freezing" the layer. -In the second step, new Delta and Image layers are created, containing all the -data in the Frozen InMemory layer. When the new layers are ready, the original -frozen layer is replaced with the new layers in the layer map, and the original -frozen layer is dropped, releasing the memory. +In the second step, a new Delta layers is created, containing all the data from +the Frozen InMemory layer. When it has been created and flushed to disk, the +original frozen layer is replaced with the new layers in the layer map, and the +original frozen layer is dropped, releasing the memory. # Layer files (On-disk layers) -The files are called "layer files". Each layer file corresponds -to one RELISH_SEG_SIZE slice of a PostgreSQL relation fork or -non-rel file in a range of LSNs. The layer files -for each timeline are stored in the timeline's subdirectory under +The files are called "layer files". Each layer file covers a range of keys, and +a range of LSNs (or a single LSN, in case of image layers). You can think of it +as a rectangle in the two-dimensional key-LSN space. The layer files for each +timeline are stored in the timeline's subdirectory under .zenith/tenants//timelines. -There are two kind of layer file: base images, and deltas. A base -image file contains a layer of a segment as it was at one LSN, -whereas a delta file contains modifications to a segment - mostly in -the form of WAL records - in a range of LSN +There are two kind of layer files: images, and delta layers. An image file +contains a snapshot of all keys at a particular LSN, whereas a delta file +contains modifications to a segment - mostly in the form of WAL records - in a +range of LSN. -base image file: +image file: - rel______ + 000000067F000032BE0000400000000070B6-000000067F000032BE0000400000000080B6__00000000346BC568 + start key end key LSN + +The first parts define the key range that the layer covers. See +pgdatadir_mapping.rs for how the key space is used. The last part is the LSN. delta file: - rel_______ +Delta files are named similarly, but they cover a range of LSNs: -For example: + 000000067F000032BE0000400000000020B6-000000067F000032BE0000400000000030B6__000000578C6B29-0000000057A50051 + start key end key start LSN end LSN - rel_1663_13990_2609_0_10_000000000169C348 - rel_1663_13990_2609_0_10_000000000169C348_0000000001702000 +A delta file contains all the key-values in the key-range that were updated in +the LSN range. If a key has not been modified, there is no trace of it in the +delta layer. -In addition to the relations, with "rel_*" prefix, we use the same -format for storing various smaller files from the PostgreSQL data -directory. They will use different suffixes and the naming scheme up -to the LSNs vary. The Zenith source code uses the term "relish" to -mean "a relation, or other file that's treated like a relation in the -storage" For example, a base image of a CLOG segment would be named -like this: - pg_xact_0000_0_00000000198B06B0 +A delta layer file can cover a part of the overall key space, as in the previous +example, or the whole key range like this: -There is no difference in how the relation and non-relation files are -managed, except that the first part of file names is different. -Internally, the relations and non-relation files that are managed in -the versioned store are together called "relishes". + 000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__000000578C6B29-0000000057A50051 -If a file has been dropped, the last layer file for it is created -with the _DROPPED suffix, e.g. - - rel_1663_13990_2609_0_10_000000000169C348_0000000001702000_DROPPED +A file that covers the whole key range is called a L0 file (Level 0), while a +file that covers only part of the key range is called a L1 file. The "level" of +a file is not explicitly stored anywhere, you can only distinguish them by +looking at the key range that a file covers. The read-path doesn't need to +treat L0 and L1 files any differently. ## Notation used in this document +FIXME: This is somewhat obsolete, the layer files cover a key-range rather than +a particular relation nowadays. However, the description on how you find a page +version, and how branching and GC works is still valid. + The full path of a delta file looks like this: .zenith/tenants/941ddc8604413b88b3d208bddf90396c/timelines/4af489b06af8eed9e27a841775616962/rel_1663_13990_2609_0_10_000000000169C348_0000000001702000 diff --git a/pageserver/src/layered_repository/delta_layer.rs b/pageserver/src/layered_repository/delta_layer.rs index 1a6e941fbe..bb5fa02be1 100644 --- a/pageserver/src/layered_repository/delta_layer.rs +++ b/pageserver/src/layered_repository/delta_layer.rs @@ -1,6 +1,5 @@ -//! //! A DeltaLayer represents a collection of WAL records or page images in a range of -//! LSNs, for one segment. It is stored on a file on disk. +//! LSNs, and in a range of Keys. It is stored on a file on disk. //! //! Usually a delta layer only contains differences - in the form of WAL records against //! a base LSN. However, if a segment is newly created, by creating a new relation or @@ -11,84 +10,74 @@ //! can happen when you create a new branch in the middle of a delta layer, and the WAL //! records on the new branch are put in a new delta layer. //! -//! When a delta file needs to be accessed, we slurp the metadata and segsize chapters +//! When a delta file needs to be accessed, we slurp the 'index' metadata //! into memory, into the DeltaLayerInner struct. See load() and unload() functions. -//! To access a page/WAL record, we search `page_version_metas` for the block # and LSN. -//! The byte ranges in the metadata can be used to find the page/WAL record in -//! PAGE_VERSIONS_CHAPTER. +//! To access a particular value, we search `index` for the given key. +//! The byte offset in the index can be used to find the value in +//! VALUES_CHAPTER. //! //! On disk, the delta files are stored in timelines/ directory. //! Currently, there are no subdirectories, and each delta file is named like this: //! -//! ______ +//! -__- page/WAL record +/// byte ranges in VALUES_CHAPTER +static INDEX_CHAPTER: u64 = 1; -/// Mapping from (block #, lsn) -> page/WAL record -/// byte ranges in PAGE_VERSIONS_CHAPTER -static PAGE_VERSION_METAS_CHAPTER: u64 = 1; /// Page/WAL bytes - cannot be interpreted -/// without PAGE_VERSION_METAS_CHAPTER -static PAGE_VERSIONS_CHAPTER: u64 = 2; -static SEG_SIZES_CHAPTER: u64 = 3; +/// without the page versions from the INDEX_CHAPTER +static VALUES_CHAPTER: u64 = 2; /// Contains the [`Summary`] struct -static SUMMARY_CHAPTER: u64 = 4; +static SUMMARY_CHAPTER: u64 = 3; #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] struct Summary { tenantid: ZTenantId, timelineid: ZTimelineId, - seg: SegmentTag, - - start_lsn: Lsn, - end_lsn: Lsn, - - dropped: bool, + key_range: Range, + lsn_range: Range, } impl From<&DeltaLayer> for Summary { @@ -96,33 +85,17 @@ impl From<&DeltaLayer> for Summary { Self { tenantid: layer.tenantid, timelineid: layer.timelineid, - seg: layer.seg, - - start_lsn: layer.start_lsn, - end_lsn: layer.end_lsn, - - dropped: layer.dropped, + key_range: layer.key_range.clone(), + lsn_range: layer.lsn_range.clone(), } } } -#[derive(Serialize, Deserialize)] -struct BlobRange { - offset: u64, - size: usize, -} - -fn read_blob(reader: &BoundedReader<&'_ F>, range: &BlobRange) -> Result> { - let mut buf = vec![0u8; range.size]; - reader.read_exact_at(&mut buf, range.offset)?; - Ok(buf) -} - /// /// DeltaLayer is the in-memory data structure associated with an /// on-disk delta file. We keep a DeltaLayer in memory for each /// file, in the LayerMap. If a layer is in "loaded" state, we have a -/// copy of the file in memory, in 'inner'. Otherwise the struct is +/// copy of the index in memory, in 'inner'. Otherwise the struct is /// just a placeholder for a file that exists on disk, and it needs to /// be loaded before using it in queries. /// @@ -131,47 +104,24 @@ pub struct DeltaLayer { pub tenantid: ZTenantId, pub timelineid: ZTimelineId, - pub seg: SegmentTag, - - // - // This entry contains all the changes from 'start_lsn' to 'end_lsn'. The - // start is inclusive, and end is exclusive. - // - pub start_lsn: Lsn, - pub end_lsn: Lsn, - - dropped: bool, + pub key_range: Range, + pub lsn_range: Range, inner: RwLock, } pub struct DeltaLayerInner { - /// If false, the 'page_version_metas' and 'seg_sizes' have not been - /// loaded into memory yet. + /// If false, the 'index' has not been loaded into memory yet. loaded: bool, + /// + /// All versions of all pages in the layer are kept here. + /// Indexed by block number and LSN. The value is an offset into the + /// chapter where the page version is stored. + /// + index: HashMap>, + book: Option>, - - /// All versions of all pages in the file are are kept here. - /// Indexed by block number and LSN. - page_version_metas: VecMap<(SegmentBlk, Lsn), BlobRange>, - - /// `seg_sizes` tracks the size of the segment at different points in time. - seg_sizes: VecMap, -} - -impl DeltaLayerInner { - fn get_seg_size(&self, lsn: Lsn) -> Result { - // Scan the VecMap backwards, starting from the given entry. - let slice = self - .seg_sizes - .slice_range((Included(&Lsn(0)), Included(&lsn))); - if let Some((_entry_lsn, entry)) = slice.last() { - Ok(*entry) - } else { - bail!("could not find seg size in delta layer") - } - } } impl Layer for DeltaLayer { @@ -183,132 +133,93 @@ impl Layer for DeltaLayer { self.timelineid } - fn get_seg_tag(&self) -> SegmentTag { - self.seg + fn get_key_range(&self) -> Range { + self.key_range.clone() } - fn is_dropped(&self) -> bool { - self.dropped - } - - fn get_start_lsn(&self) -> Lsn { - self.start_lsn - } - - fn get_end_lsn(&self) -> Lsn { - self.end_lsn + fn get_lsn_range(&self) -> Range { + self.lsn_range.clone() } fn filename(&self) -> PathBuf { PathBuf::from(self.layer_name().to_string()) } - /// Look up given page in the cache. - fn get_page_reconstruct_data( + fn get_value_reconstruct_data( &self, - blknum: SegmentBlk, - lsn: Lsn, - reconstruct_data: &mut PageReconstructData, - ) -> anyhow::Result { + key: Key, + lsn_range: Range, + reconstruct_state: &mut ValueReconstructState, + ) -> anyhow::Result { let mut need_image = true; - ensure!((0..RELISH_SEG_SIZE).contains(&blknum)); - - match &reconstruct_data.page_img { - Some((cached_lsn, _)) if &self.end_lsn <= cached_lsn => { - return Ok(PageReconstructResult::Complete) - } - _ => {} - } + ensure!(self.key_range.contains(&key)); { // Open the file and lock the metadata in memory let inner = self.load()?; - let page_version_reader = inner + let values_reader = inner .book .as_ref() .expect("should be loaded in load call above") - .chapter_reader(PAGE_VERSIONS_CHAPTER)?; + .chapter_reader(VALUES_CHAPTER)?; - // Scan the metadata VecMap backwards, starting from the given entry. - let minkey = (blknum, Lsn(0)); - let maxkey = (blknum, lsn); - let iter = inner - .page_version_metas - .slice_range((Included(&minkey), Included(&maxkey))) - .iter() - .rev(); - for ((_blknum, pv_lsn), blob_range) in iter { - match &reconstruct_data.page_img { - Some((cached_lsn, _)) if pv_lsn <= cached_lsn => { - return Ok(PageReconstructResult::Complete) - } - _ => {} - } - - let pv = PageVersion::des(&read_blob(&page_version_reader, blob_range)?)?; - - match pv { - PageVersion::Page(img) => { - // Found a page image, return it - reconstruct_data.page_img = Some((*pv_lsn, img)); - need_image = false; + // Scan the page versions backwards, starting from `lsn`. + if let Some(vec_map) = inner.index.get(&key) { + let slice = vec_map.slice_range(lsn_range); + let mut size = 0usize; + let mut first_pos = 0u64; + for (_entry_lsn, blob_ref) in slice.iter().rev() { + size += blob_ref.size(); + first_pos = blob_ref.pos(); + if blob_ref.will_init() { break; } - PageVersion::Wal(rec) => { - let will_init = rec.will_init(); - reconstruct_data.records.push((*pv_lsn, rec)); - if will_init { - // This WAL record initializes the page, so no need to go further back - need_image = false; - break; + } + if size != 0 { + let mut buf = vec![0u8; size]; + values_reader.read_exact_at(&mut buf, first_pos)?; + for (entry_lsn, blob_ref) in slice.iter().rev() { + let offs = (blob_ref.pos() - first_pos) as usize; + let val = Value::des(&buf[offs..offs + blob_ref.size()])?; + match val { + Value::Image(img) => { + reconstruct_state.img = Some((*entry_lsn, img)); + need_image = false; + break; + } + Value::WalRecord(rec) => { + let will_init = rec.will_init(); + reconstruct_state.records.push((*entry_lsn, rec)); + if will_init { + // This WAL record initializes the page, so no need to go further back + need_image = false; + break; + } + } } } } } - - // If we didn't find any records for this, check if the request is beyond EOF - if need_image - && reconstruct_data.records.is_empty() - && self.seg.rel.is_blocky() - && blknum >= inner.get_seg_size(lsn)? - { - return Ok(PageReconstructResult::Missing(self.start_lsn)); - } - // release metadata lock and close the file } // If an older page image is needed to reconstruct the page, let the // caller know. if need_image { - Ok(PageReconstructResult::Continue(Lsn(self.start_lsn.0 - 1))) + Ok(ValueReconstructResult::Continue) } else { - Ok(PageReconstructResult::Complete) + Ok(ValueReconstructResult::Complete) } } - /// Get size of the relation at given LSN - fn get_seg_size(&self, lsn: Lsn) -> anyhow::Result { - ensure!(lsn >= self.start_lsn); - ensure!( - self.seg.rel.is_blocky(), - "get_seg_size() called on a non-blocky rel" - ); + fn iter(&self) -> Box> + '_> { + let inner = self.load().unwrap(); - let inner = self.load()?; - inner.get_seg_size(lsn) - } - - /// Does this segment exist at given LSN? - fn get_seg_exists(&self, lsn: Lsn) -> Result { - // Is the requested LSN after the rel was dropped? - if self.dropped && lsn >= self.end_lsn { - return Ok(false); + match DeltaValueIter::new(inner) { + Ok(iter) => Box::new(iter), + Err(err) => Box::new(std::iter::once(Err(err))), } - - // Otherwise, it exists. - Ok(true) } /// @@ -316,13 +227,22 @@ impl Layer for DeltaLayer { /// it will need to be loaded back. /// fn unload(&self) -> Result<()> { + // FIXME: In debug mode, loading and unloading the index slows + // things down so much that you get timeout errors. At least + // with the test_parallel_copy test. So as an even more ad hoc + // stopgap fix for that, only unload every on average 10 + // checkpoint cycles. + use rand::RngCore; + if rand::thread_rng().next_u32() > (u32::MAX / 10) { + return Ok(()); + } + let mut inner = match self.inner.try_write() { Ok(inner) => inner, Err(TryLockError::WouldBlock) => return Ok(()), Err(TryLockError::Poisoned(_)) => panic!("DeltaLayer lock was poisoned"), }; - inner.page_version_metas = VecMap::default(); - inner.seg_sizes = VecMap::default(); + inner.index = HashMap::default(); inner.loaded = false; // Note: we keep the Book open. Is that a good idea? The virtual file @@ -349,45 +269,52 @@ impl Layer for DeltaLayer { /// debugging function to print out the contents of the layer fn dump(&self) -> Result<()> { println!( - "----- delta layer for ten {} tli {} seg {} {}-{} ----", - self.tenantid, self.timelineid, self.seg, self.start_lsn, self.end_lsn + "----- delta layer for ten {} tli {} keys {}-{} lsn {}-{} ----", + self.tenantid, + self.timelineid, + self.key_range.start, + self.key_range.end, + self.lsn_range.start, + self.lsn_range.end ); - println!("--- seg sizes ---"); let inner = self.load()?; - for (k, v) in inner.seg_sizes.as_slice() { - println!(" {}: {}", k, v); - } - println!("--- page versions ---"); let path = self.path(); let file = std::fs::File::open(&path)?; let book = Book::new(file)?; + let chapter = book.chapter_reader(VALUES_CHAPTER)?; - let chapter = book.chapter_reader(PAGE_VERSIONS_CHAPTER)?; - for ((blk, lsn), blob_range) in inner.page_version_metas.as_slice() { - let mut desc = String::new(); + let mut values: Vec<(&Key, &VecMap)> = inner.index.iter().collect(); + values.sort_by_key(|k| k.0); - let buf = read_blob(&chapter, blob_range)?; - let pv = PageVersion::des(&buf)?; + for (key, versions) in values { + for (lsn, blob_ref) in versions.as_slice() { + let mut desc = String::new(); + let mut buf = vec![0u8; blob_ref.size()]; + chapter.read_exact_at(&mut buf, blob_ref.pos())?; + let val = Value::des(&buf); - match pv { - PageVersion::Page(img) => { - write!(&mut desc, " img {} bytes", img.len())?; - } - PageVersion::Wal(rec) => { - let wal_desc = walrecord::describe_wal_record(&rec); - write!( - &mut desc, - " rec {} bytes will_init: {} {}", - blob_range.size, - rec.will_init(), - wal_desc - )?; + match val { + Ok(Value::Image(img)) => { + write!(&mut desc, " img {} bytes", img.len())?; + } + Ok(Value::WalRecord(rec)) => { + let wal_desc = walrecord::describe_wal_record(&rec); + write!( + &mut desc, + " rec {} bytes will_init: {} {}", + buf.len(), + rec.will_init(), + wal_desc + )?; + } + Err(err) => { + write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?; + } } + println!(" key {} at {}: {}", key, lsn, desc); } - - println!(" blk {} at {}: {}", blk, lsn, desc); } Ok(()) @@ -475,18 +402,13 @@ impl DeltaLayer { } } - let chapter = book.read_chapter(PAGE_VERSION_METAS_CHAPTER)?; - let page_version_metas = VecMap::des(&chapter)?; - - let chapter = book.read_chapter(SEG_SIZES_CHAPTER)?; - let seg_sizes = VecMap::des(&chapter)?; + let chapter = book.read_chapter(INDEX_CHAPTER)?; + let index = HashMap::des(&chapter)?; debug!("loaded from {}", &path.display()); - inner.page_version_metas = page_version_metas; - inner.seg_sizes = seg_sizes; + inner.index = index; inner.loaded = true; - Ok(()) } @@ -501,15 +423,12 @@ impl DeltaLayer { path_or_conf: PathOrConf::Conf(conf), timelineid, tenantid, - seg: filename.seg, - start_lsn: filename.start_lsn, - end_lsn: filename.end_lsn, - dropped: filename.dropped, + key_range: filename.key_range.clone(), + lsn_range: filename.lsn_range.clone(), inner: RwLock::new(DeltaLayerInner { loaded: false, book: None, - page_version_metas: VecMap::default(), - seg_sizes: VecMap::default(), + index: HashMap::default(), }), } } @@ -519,7 +438,7 @@ impl DeltaLayer { /// This variant is only used for debugging purposes, by the 'dump_layerfile' binary. pub fn new_for_path(path: &Path, book: &Book) -> Result where - F: std::os::unix::prelude::FileExt, + F: FileExt, { let chapter = book.read_chapter(SUMMARY_CHAPTER)?; let summary = Summary::des(&chapter)?; @@ -528,25 +447,20 @@ impl DeltaLayer { path_or_conf: PathOrConf::Path(path.to_path_buf()), timelineid: summary.timelineid, tenantid: summary.tenantid, - seg: summary.seg, - start_lsn: summary.start_lsn, - end_lsn: summary.end_lsn, - dropped: summary.dropped, + key_range: summary.key_range, + lsn_range: summary.lsn_range, inner: RwLock::new(DeltaLayerInner { loaded: false, book: None, - page_version_metas: VecMap::default(), - seg_sizes: VecMap::default(), + index: HashMap::default(), }), }) } fn layer_name(&self) -> DeltaFileName { DeltaFileName { - seg: self.seg, - start_lsn: self.start_lsn, - end_lsn: self.end_lsn, - dropped: self.dropped, + key_range: self.key_range.clone(), + lsn_range: self.lsn_range.clone(), } } @@ -567,24 +481,24 @@ impl DeltaLayer { /// /// 1. Create the DeltaLayerWriter by calling DeltaLayerWriter::new(...) /// -/// 2. Write the contents by calling `put_page_version` for every page +/// 2. Write the contents by calling `put_value` for every page /// version to store in the layer. /// /// 3. Call `finish`. /// pub struct DeltaLayerWriter { conf: &'static PageServerConf, + path: PathBuf, timelineid: ZTimelineId, tenantid: ZTenantId, - seg: SegmentTag, - start_lsn: Lsn, - end_lsn: Lsn, - dropped: bool, - page_version_writer: ChapterWriter>, - pv_offset: u64, + key_start: Key, + lsn_range: Range, - page_version_metas: VecMap<(SegmentBlk, Lsn), BlobRange>, + index: HashMap>, + + values_writer: ChapterWriter>, + end_offset: u64, } impl DeltaLayerWriter { @@ -595,94 +509,86 @@ impl DeltaLayerWriter { conf: &'static PageServerConf, timelineid: ZTimelineId, tenantid: ZTenantId, - seg: SegmentTag, - start_lsn: Lsn, - end_lsn: Lsn, - dropped: bool, + key_start: Key, + lsn_range: Range, ) -> Result { - // Create the file + // Create the file initially with a temporary filename. We don't know + // the end key yet, so we cannot form the final filename yet. We will + // rename it when we're done. // // Note: This overwrites any existing file. There shouldn't be any. // FIXME: throw an error instead? - let path = DeltaLayer::path_for( - &PathOrConf::Conf(conf), - timelineid, - tenantid, - &DeltaFileName { - seg, - start_lsn, - end_lsn, - dropped, - }, - ); + let path = conf.timeline_path(&timelineid, &tenantid).join(format!( + "{}-XXX__{:016X}-{:016X}.temp", + key_start, + u64::from(lsn_range.start), + u64::from(lsn_range.end) + )); let file = VirtualFile::create(&path)?; let buf_writer = BufWriter::new(file); let book = BookWriter::new(buf_writer, DELTA_FILE_MAGIC)?; // Open the page-versions chapter for writing. The calls to - // `put_page_version` will use this to write the contents. - let page_version_writer = book.new_chapter(PAGE_VERSIONS_CHAPTER); + // `put_value` will use this to write the contents. + let values_writer = book.new_chapter(VALUES_CHAPTER); Ok(DeltaLayerWriter { conf, + path, timelineid, tenantid, - seg, - start_lsn, - end_lsn, - dropped, - page_version_writer, - page_version_metas: VecMap::default(), - pv_offset: 0, + key_start, + lsn_range, + index: HashMap::new(), + values_writer, + end_offset: 0, }) } /// - /// Append a page version to the file. + /// Append a key-value pair to the file. /// - /// 'buf' is a serialized PageVersion. - /// The page versions must be appended in blknum, lsn order. + /// The values must be appended in key, lsn order. /// - pub fn put_page_version(&mut self, blknum: SegmentBlk, lsn: Lsn, buf: &[u8]) -> Result<()> { + pub fn put_value(&mut self, key: Key, lsn: Lsn, val: Value) -> Result<()> { + //info!("DELTA: key {} at {} on {}", key, lsn, self.path.display()); + assert!(self.lsn_range.start <= lsn); // Remember the offset and size metadata. The metadata is written // to a separate chapter, in `finish`. - let blob_range = BlobRange { - offset: self.pv_offset, - size: buf.len(), - }; - self.page_version_metas - .append((blknum, lsn), blob_range) - .unwrap(); - - // write the page version - self.page_version_writer.write_all(buf)?; - self.pv_offset += buf.len() as u64; + let off = self.end_offset; + let buf = Value::ser(&val)?; + let len = buf.len(); + self.values_writer.write_all(&buf)?; + self.end_offset += len as u64; + let vec_map = self.index.entry(key).or_default(); + let blob_ref = BlobRef::new(off, len, val.will_init()); + let old = vec_map.append_or_update_last(lsn, blob_ref).unwrap().0; + if old.is_some() { + // We already had an entry for this LSN. That's odd.. + bail!( + "Value for {} at {} already exists in delta layer being built", + key, + lsn + ); + } Ok(()) } + pub fn size(&self) -> u64 { + self.end_offset + } + /// /// Finish writing the delta layer. /// - /// 'seg_sizes' is a list of size changes to store with the actual data. - /// - pub fn finish(self, seg_sizes: VecMap) -> anyhow::Result { - // Close the page-versions chapter - let book = self.page_version_writer.close()?; + pub fn finish(self, key_end: Key) -> anyhow::Result { + // Close the values chapter + let book = self.values_writer.close()?; - // Write out page versions metadata - let mut chapter = book.new_chapter(PAGE_VERSION_METAS_CHAPTER); - let buf = VecMap::ser(&self.page_version_metas)?; - chapter.write_all(&buf)?; - let book = chapter.close()?; - - if self.seg.rel.is_blocky() { - ensure!(!seg_sizes.is_empty()); - } - - // and seg_sizes to separate chapter - let mut chapter = book.new_chapter(SEG_SIZES_CHAPTER); - let buf = VecMap::ser(&seg_sizes)?; + // Write out the index + let mut chapter = book.new_chapter(INDEX_CHAPTER); + let buf = HashMap::ser(&self.index)?; chapter.write_all(&buf)?; let book = chapter.close()?; @@ -690,12 +596,8 @@ impl DeltaLayerWriter { let summary = Summary { tenantid: self.tenantid, timelineid: self.timelineid, - seg: self.seg, - - start_lsn: self.start_lsn, - end_lsn: self.end_lsn, - - dropped: self.dropped, + key_range: self.key_start..key_end, + lsn_range: self.lsn_range.clone(), }; Summary::ser_into(&summary, &mut chapter)?; let book = chapter.close()?; @@ -710,20 +612,111 @@ impl DeltaLayerWriter { path_or_conf: PathOrConf::Conf(self.conf), tenantid: self.tenantid, timelineid: self.timelineid, - seg: self.seg, - start_lsn: self.start_lsn, - end_lsn: self.end_lsn, - dropped: self.dropped, + key_range: self.key_start..key_end, + lsn_range: self.lsn_range.clone(), inner: RwLock::new(DeltaLayerInner { loaded: false, + index: HashMap::new(), book: None, - page_version_metas: VecMap::default(), - seg_sizes: VecMap::default(), }), }; - trace!("created delta layer {}", &layer.path().display()); + // Rename the file to its final name + // + // Note: This overwrites any existing file. There shouldn't be any. + // FIXME: throw an error instead? + let final_path = DeltaLayer::path_for( + &PathOrConf::Conf(self.conf), + self.timelineid, + self.tenantid, + &DeltaFileName { + key_range: self.key_start..key_end, + lsn_range: self.lsn_range, + }, + ); + std::fs::rename(self.path, &final_path)?; + + trace!("created delta layer {}", final_path.display()); Ok(layer) } + + pub fn abort(self) { + match self.values_writer.close() { + Ok(book) => { + if let Err(err) = book.close() { + error!("error while closing delta layer file: {}", err); + } + } + Err(err) => { + error!("error while closing chapter writer: {}", err); + } + } + if let Err(err) = std::fs::remove_file(self.path) { + error!("error removing unfinished delta layer file: {}", err); + } + } +} + +/// +/// Iterator over all key-value pairse stored in a delta layer +/// +/// FIXME: This creates a Vector to hold the offsets of all key value pairs. +/// That takes up quite a lot of memory. Should do this in a more streaming +/// fashion. +/// +struct DeltaValueIter { + all_offsets: Vec<(Key, Lsn, BlobRef)>, + next_idx: usize, + data: Vec, +} + +impl Iterator for DeltaValueIter { + type Item = Result<(Key, Lsn, Value)>; + + fn next(&mut self) -> Option { + self.next_res().transpose() + } +} + +impl DeltaValueIter { + fn new(inner: RwLockReadGuard) -> Result { + let mut index: Vec<(&Key, &VecMap)> = inner.index.iter().collect(); + index.sort_by_key(|x| x.0); + + let mut all_offsets: Vec<(Key, Lsn, BlobRef)> = Vec::new(); + for (key, vec_map) in index.iter() { + for (lsn, blob_ref) in vec_map.as_slice().iter() { + all_offsets.push((**key, *lsn, *blob_ref)); + } + } + + let values_reader = inner + .book + .as_ref() + .expect("should be loaded in load call above") + .chapter_reader(VALUES_CHAPTER)?; + let file_size = values_reader.len() as usize; + let mut layer = DeltaValueIter { + all_offsets, + next_idx: 0, + data: vec![0u8; file_size], + }; + values_reader.read_exact_at(&mut layer.data, 0)?; + + Ok(layer) + } + + fn next_res(&mut self) -> Result> { + if self.next_idx < self.all_offsets.len() { + let (key, lsn, blob_ref) = self.all_offsets[self.next_idx]; + let offs = blob_ref.pos() as usize; + let size = blob_ref.size(); + let val = Value::des(&self.data[offs..offs + size])?; + self.next_idx += 1; + Ok(Some((key, lsn, val))) + } else { + Ok(None) + } + } } diff --git a/pageserver/src/layered_repository/filename.rs b/pageserver/src/layered_repository/filename.rs index df23700dfd..cd63f014c4 100644 --- a/pageserver/src/layered_repository/filename.rs +++ b/pageserver/src/layered_repository/filename.rs @@ -2,29 +2,52 @@ //! Helper functions for dealing with filenames of the image and delta layer files. //! use crate::config::PageServerConf; -use crate::layered_repository::storage_layer::SegmentTag; -use crate::relish::*; +use crate::repository::Key; +use std::cmp::Ordering; use std::fmt; +use std::ops::Range; use std::path::PathBuf; use zenith_utils::lsn::Lsn; // Note: LayeredTimeline::load_layer_map() relies on this sort order -#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone)] +#[derive(Debug, PartialEq, Eq, Clone)] pub struct DeltaFileName { - pub seg: SegmentTag, - pub start_lsn: Lsn, - pub end_lsn: Lsn, - pub dropped: bool, + pub key_range: Range, + pub lsn_range: Range, +} + +impl PartialOrd for DeltaFileName { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for DeltaFileName { + fn cmp(&self, other: &Self) -> Ordering { + let mut cmp; + + cmp = self.key_range.start.cmp(&other.key_range.start); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.key_range.end.cmp(&other.key_range.end); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.lsn_range.start.cmp(&other.lsn_range.start); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.lsn_range.end.cmp(&other.lsn_range.end); + + cmp + } } /// Represents the filename of a DeltaLayer /// -/// ______ -/// -/// or if it was dropped: -/// -/// _______DROPPED +/// -__- /// impl DeltaFileName { /// @@ -32,234 +55,123 @@ impl DeltaFileName { /// match the expected pattern. /// pub fn parse_str(fname: &str) -> Option { - let rel; - let mut parts; - if let Some(rest) = fname.strip_prefix("rel_") { - parts = rest.split('_'); - rel = RelishTag::Relation(RelTag { - spcnode: parts.next()?.parse::().ok()?, - dbnode: parts.next()?.parse::().ok()?, - relnode: parts.next()?.parse::().ok()?, - forknum: parts.next()?.parse::().ok()?, - }); - } else if let Some(rest) = fname.strip_prefix("pg_xact_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::Clog, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_multixact_members_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_multixact_offsets_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_filenodemap_") { - parts = rest.split('_'); - rel = RelishTag::FileNodeMap { - spcnode: parts.next()?.parse::().ok()?, - dbnode: parts.next()?.parse::().ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_twophase_") { - parts = rest.split('_'); - rel = RelishTag::TwoPhase { - xid: parts.next()?.parse::().ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_control_checkpoint_") { - parts = rest.split('_'); - rel = RelishTag::Checkpoint; - } else if let Some(rest) = fname.strip_prefix("pg_control_") { - parts = rest.split('_'); - rel = RelishTag::ControlFile; - } else { + let mut parts = fname.split("__"); + let mut key_parts = parts.next()?.split('-'); + let mut lsn_parts = parts.next()?.split('-'); + + let key_start_str = key_parts.next()?; + let key_end_str = key_parts.next()?; + let lsn_start_str = lsn_parts.next()?; + let lsn_end_str = lsn_parts.next()?; + if parts.next().is_some() || key_parts.next().is_some() || key_parts.next().is_some() { return None; } - let segno = parts.next()?.parse::().ok()?; + let key_start = Key::from_hex(key_start_str).ok()?; + let key_end = Key::from_hex(key_end_str).ok()?; - let seg = SegmentTag { rel, segno }; + let start_lsn = Lsn::from_hex(lsn_start_str).ok()?; + let end_lsn = Lsn::from_hex(lsn_end_str).ok()?; - let start_lsn = Lsn::from_hex(parts.next()?).ok()?; - let end_lsn = Lsn::from_hex(parts.next()?).ok()?; - - let mut dropped = false; - if let Some(suffix) = parts.next() { - if suffix == "DROPPED" { - dropped = true; - } else { - return None; - } - } - if parts.next().is_some() { + if start_lsn >= end_lsn { return None; + // or panic? + } + + if key_start >= key_end { + return None; + // or panic? } Some(DeltaFileName { - seg, - start_lsn, - end_lsn, - dropped, + key_range: key_start..key_end, + lsn_range: start_lsn..end_lsn, }) } } impl fmt::Display for DeltaFileName { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let basename = match self.seg.rel { - RelishTag::Relation(reltag) => format!( - "rel_{}_{}_{}_{}", - reltag.spcnode, reltag.dbnode, reltag.relnode, reltag.forknum - ), - RelishTag::Slru { - slru: SlruKind::Clog, - segno, - } => format!("pg_xact_{:04X}", segno), - RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno, - } => format!("pg_multixact_members_{:04X}", segno), - RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno, - } => format!("pg_multixact_offsets_{:04X}", segno), - RelishTag::FileNodeMap { spcnode, dbnode } => { - format!("pg_filenodemap_{}_{}", spcnode, dbnode) - } - RelishTag::TwoPhase { xid } => format!("pg_twophase_{}", xid), - RelishTag::Checkpoint => "pg_control_checkpoint".to_string(), - RelishTag::ControlFile => "pg_control".to_string(), - }; - write!( f, - "{}_{}_{:016X}_{:016X}{}", - basename, - self.seg.segno, - u64::from(self.start_lsn), - u64::from(self.end_lsn), - if self.dropped { "_DROPPED" } else { "" } + "{}-{}__{:016X}-{:016X}", + self.key_range.start, + self.key_range.end, + u64::from(self.lsn_range.start), + u64::from(self.lsn_range.end), ) } } -#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone)] +#[derive(Debug, PartialEq, Eq, Clone)] pub struct ImageFileName { - pub seg: SegmentTag, + pub key_range: Range, pub lsn: Lsn, } +impl PartialOrd for ImageFileName { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for ImageFileName { + fn cmp(&self, other: &Self) -> Ordering { + let mut cmp; + + cmp = self.key_range.start.cmp(&other.key_range.start); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.key_range.end.cmp(&other.key_range.end); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.lsn.cmp(&other.lsn); + + cmp + } +} + /// /// Represents the filename of an ImageLayer /// -/// _____ -/// +/// -__ impl ImageFileName { /// /// Parse a string as an image file name. Returns None if the filename does not /// match the expected pattern. /// pub fn parse_str(fname: &str) -> Option { - let rel; - let mut parts; - if let Some(rest) = fname.strip_prefix("rel_") { - parts = rest.split('_'); - rel = RelishTag::Relation(RelTag { - spcnode: parts.next()?.parse::().ok()?, - dbnode: parts.next()?.parse::().ok()?, - relnode: parts.next()?.parse::().ok()?, - forknum: parts.next()?.parse::().ok()?, - }); - } else if let Some(rest) = fname.strip_prefix("pg_xact_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::Clog, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_multixact_members_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_multixact_offsets_") { - parts = rest.split('_'); - rel = RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno: u32::from_str_radix(parts.next()?, 16).ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_filenodemap_") { - parts = rest.split('_'); - rel = RelishTag::FileNodeMap { - spcnode: parts.next()?.parse::().ok()?, - dbnode: parts.next()?.parse::().ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_twophase_") { - parts = rest.split('_'); - rel = RelishTag::TwoPhase { - xid: parts.next()?.parse::().ok()?, - }; - } else if let Some(rest) = fname.strip_prefix("pg_control_checkpoint_") { - parts = rest.split('_'); - rel = RelishTag::Checkpoint; - } else if let Some(rest) = fname.strip_prefix("pg_control_") { - parts = rest.split('_'); - rel = RelishTag::ControlFile; - } else { + let mut parts = fname.split("__"); + let mut key_parts = parts.next()?.split('-'); + + let key_start_str = key_parts.next()?; + let key_end_str = key_parts.next()?; + let lsn_str = parts.next()?; + if parts.next().is_some() || key_parts.next().is_some() { return None; } - let segno = parts.next()?.parse::().ok()?; + let key_start = Key::from_hex(key_start_str).ok()?; + let key_end = Key::from_hex(key_end_str).ok()?; - let seg = SegmentTag { rel, segno }; + let lsn = Lsn::from_hex(lsn_str).ok()?; - let lsn = Lsn::from_hex(parts.next()?).ok()?; - - if parts.next().is_some() { - return None; - } - - Some(ImageFileName { seg, lsn }) + Some(ImageFileName { + key_range: key_start..key_end, + lsn, + }) } } impl fmt::Display for ImageFileName { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let basename = match self.seg.rel { - RelishTag::Relation(reltag) => format!( - "rel_{}_{}_{}_{}", - reltag.spcnode, reltag.dbnode, reltag.relnode, reltag.forknum - ), - RelishTag::Slru { - slru: SlruKind::Clog, - segno, - } => format!("pg_xact_{:04X}", segno), - RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno, - } => format!("pg_multixact_members_{:04X}", segno), - RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno, - } => format!("pg_multixact_offsets_{:04X}", segno), - RelishTag::FileNodeMap { spcnode, dbnode } => { - format!("pg_filenodemap_{}_{}", spcnode, dbnode) - } - RelishTag::TwoPhase { xid } => format!("pg_twophase_{}", xid), - RelishTag::Checkpoint => "pg_control_checkpoint".to_string(), - RelishTag::ControlFile => "pg_control".to_string(), - }; - write!( f, - "{}_{}_{:016X}", - basename, - self.seg.segno, + "{}-{}__{:016X}", + self.key_range.start, + self.key_range.end, u64::from(self.lsn), ) } diff --git a/pageserver/src/layered_repository/global_layer_map.rs b/pageserver/src/layered_repository/global_layer_map.rs deleted file mode 100644 index 169a89650a..0000000000 --- a/pageserver/src/layered_repository/global_layer_map.rs +++ /dev/null @@ -1,142 +0,0 @@ -//! -//! Global registry of open layers. -//! -//! Whenever a new in-memory layer is created to hold incoming WAL, it is registered -//! in [`GLOBAL_LAYER_MAP`], so that we can keep track of the total number of -//! in-memory layers in the system, and know when we need to evict some to release -//! memory. -//! -//! Each layer is assigned a unique ID when it's registered in the global registry. -//! The ID can be used to relocate the layer later, without having to hold locks. -//! - -use std::sync::atomic::{AtomicU8, Ordering}; -use std::sync::{Arc, RwLock}; - -use super::inmemory_layer::InMemoryLayer; - -use lazy_static::lazy_static; - -const MAX_USAGE_COUNT: u8 = 5; - -lazy_static! { - pub static ref GLOBAL_LAYER_MAP: RwLock = - RwLock::new(InMemoryLayers::default()); -} - -// TODO these types can probably be smaller -#[derive(PartialEq, Eq, Clone, Copy)] -pub struct LayerId { - index: usize, - tag: u64, // to avoid ABA problem -} - -enum SlotData { - Occupied(Arc), - /// Vacant slots form a linked list, the value is the index - /// of the next vacant slot in the list. - Vacant(Option), -} - -struct Slot { - tag: u64, - data: SlotData, - usage_count: AtomicU8, // for clock algorithm -} - -#[derive(Default)] -pub struct InMemoryLayers { - slots: Vec, - num_occupied: usize, - - // Head of free-slot list. - next_empty_slot_idx: Option, -} - -impl InMemoryLayers { - pub fn insert(&mut self, layer: Arc) -> LayerId { - let slot_idx = match self.next_empty_slot_idx { - Some(slot_idx) => slot_idx, - None => { - let idx = self.slots.len(); - self.slots.push(Slot { - tag: 0, - data: SlotData::Vacant(None), - usage_count: AtomicU8::new(0), - }); - idx - } - }; - let slots_len = self.slots.len(); - - let slot = &mut self.slots[slot_idx]; - - match slot.data { - SlotData::Occupied(_) => { - panic!("an occupied slot was in the free list"); - } - SlotData::Vacant(next_empty_slot_idx) => { - self.next_empty_slot_idx = next_empty_slot_idx; - } - } - - slot.data = SlotData::Occupied(layer); - slot.usage_count.store(1, Ordering::Relaxed); - - self.num_occupied += 1; - assert!(self.num_occupied <= slots_len); - - LayerId { - index: slot_idx, - tag: slot.tag, - } - } - - pub fn get(&self, layer_id: &LayerId) -> Option> { - let slot = self.slots.get(layer_id.index)?; // TODO should out of bounds indexes just panic? - if slot.tag != layer_id.tag { - return None; - } - - if let SlotData::Occupied(layer) = &slot.data { - let _ = slot.usage_count.fetch_update( - Ordering::Relaxed, - Ordering::Relaxed, - |old_usage_count| { - if old_usage_count < MAX_USAGE_COUNT { - Some(old_usage_count + 1) - } else { - None - } - }, - ); - Some(Arc::clone(layer)) - } else { - None - } - } - - // TODO this won't be a public API in the future - pub fn remove(&mut self, layer_id: &LayerId) { - let slot = &mut self.slots[layer_id.index]; - - if slot.tag != layer_id.tag { - return; - } - - match &slot.data { - SlotData::Occupied(_layer) => { - // TODO evict the layer - } - SlotData::Vacant(_) => unimplemented!(), - } - - slot.data = SlotData::Vacant(self.next_empty_slot_idx); - self.next_empty_slot_idx = Some(layer_id.index); - - assert!(self.num_occupied > 0); - self.num_occupied -= 1; - - slot.tag = slot.tag.wrapping_add(1); - } -} diff --git a/pageserver/src/layered_repository/image_layer.rs b/pageserver/src/layered_repository/image_layer.rs index 5b8ec46452..ab51c36cae 100644 --- a/pageserver/src/layered_repository/image_layer.rs +++ b/pageserver/src/layered_repository/image_layer.rs @@ -1,55 +1,54 @@ -//! An ImageLayer represents an image or a snapshot of a segment at one particular LSN. -//! It is stored in a file on disk. +//! An ImageLayer represents an image or a snapshot of a key-range at +//! one particular LSN. It contains an image of all key-value pairs +//! in its key-range. Any key that falls into the image layer's range +//! but does not exist in the layer, does not exist. //! -//! On disk, the image files are stored in timelines/ directory. -//! Currently, there are no subdirectories, and each image layer file is named like this: +//! An image layer is stored in a file on disk. The file is stored in +//! timelines/ directory. Currently, there are no +//! subdirectories, and each image layer file is named like this: //! -//! Note that segno is -//! _____ +//! -__ //! //! For example: //! -//! 1663_13990_2609_0_5_000000000169C348 +//! 000000067F000032BE0000400000000070B6-000000067F000032BE0000400000000080B6__00000000346BC568 //! //! An image file is constructed using the 'bookfile' crate. //! //! Only metadata is loaded into memory by the load function. //! When images are needed, they are read directly from disk. //! -//! For blocky relishes, the images are stored in BLOCKY_IMAGES_CHAPTER. -//! All the images are required to be BLOCK_SIZE, which allows for random access. -//! -//! For non-blocky relishes, the image can be found in NONBLOCKY_IMAGE_CHAPTER. -//! use crate::config::PageServerConf; use crate::layered_repository::filename::{ImageFileName, PathOrConf}; use crate::layered_repository::storage_layer::{ - Layer, PageReconstructData, PageReconstructResult, SegmentBlk, SegmentTag, + BlobRef, Layer, ValueReconstructResult, ValueReconstructState, }; -use crate::layered_repository::RELISH_SEG_SIZE; +use crate::repository::{Key, Value}; use crate::virtual_file::VirtualFile; +use crate::IMAGE_FILE_MAGIC; use crate::{ZTenantId, ZTimelineId}; -use anyhow::{anyhow, bail, ensure, Context, Result}; +use anyhow::{bail, ensure, Context, Result}; use bytes::Bytes; use log::*; use serde::{Deserialize, Serialize}; -use std::convert::TryInto; +use std::collections::HashMap; use std::fs; use std::io::{BufWriter, Write}; +use std::ops::Range; use std::path::{Path, PathBuf}; -use std::sync::{RwLock, RwLockReadGuard}; +use std::sync::{RwLock, RwLockReadGuard, TryLockError}; use bookfile::{Book, BookWriter, ChapterWriter}; use zenith_utils::bin_ser::BeSer; use zenith_utils::lsn::Lsn; -// Magic constant to identify a Zenith segment image file -pub const IMAGE_FILE_MAGIC: u32 = 0x5A616E01 + 1; +/// Mapping from (key, lsn) -> page/WAL record +/// byte ranges in VALUES_CHAPTER +static INDEX_CHAPTER: u64 = 1; /// Contains each block in block # order -const BLOCKY_IMAGES_CHAPTER: u64 = 1; -const NONBLOCKY_IMAGE_CHAPTER: u64 = 2; +const VALUES_CHAPTER: u64 = 2; /// Contains the [`Summary`] struct const SUMMARY_CHAPTER: u64 = 3; @@ -58,7 +57,7 @@ const SUMMARY_CHAPTER: u64 = 3; struct Summary { tenantid: ZTenantId, timelineid: ZTimelineId, - seg: SegmentTag, + key_range: Range, lsn: Lsn, } @@ -68,19 +67,17 @@ impl From<&ImageLayer> for Summary { Self { tenantid: layer.tenantid, timelineid: layer.timelineid, - seg: layer.seg, + key_range: layer.key_range.clone(), lsn: layer.lsn, } } } -const BLOCK_SIZE: usize = 8192; - /// /// ImageLayer is the in-memory data structure associated with an on-disk image /// file. We keep an ImageLayer in memory for each file, in the LayerMap. If a -/// layer is in "loaded" state, we have a copy of the file in memory, in 'inner'. +/// layer is in "loaded" state, we have a copy of the index in memory, in 'inner'. /// Otherwise the struct is just a placeholder for a file that exists on disk, /// and it needs to be loaded before using it in queries. /// @@ -88,7 +85,7 @@ pub struct ImageLayer { path_or_conf: PathOrConf, pub tenantid: ZTenantId, pub timelineid: ZTimelineId, - pub seg: SegmentTag, + pub key_range: Range, // This entry contains an image of all pages as of this LSN pub lsn: Lsn, @@ -96,18 +93,16 @@ pub struct ImageLayer { inner: RwLock, } -#[derive(Clone)] -enum ImageType { - Blocky { num_blocks: SegmentBlk }, - NonBlocky, -} - pub struct ImageLayerInner { - /// If None, the 'image_type' has not been loaded into memory yet. + /// If false, the 'index' has not been loaded into memory yet. + loaded: bool, + + /// The underlying (virtual) file handle. None if the layer hasn't been loaded + /// yet. book: Option>, - /// Derived from filename and bookfile chapter metadata - image_type: ImageType, + /// offset of each value + index: HashMap, } impl Layer for ImageLayer { @@ -123,98 +118,82 @@ impl Layer for ImageLayer { self.timelineid } - fn get_seg_tag(&self) -> SegmentTag { - self.seg + fn get_key_range(&self) -> Range { + self.key_range.clone() } - fn is_dropped(&self) -> bool { - false - } - - fn get_start_lsn(&self) -> Lsn { - self.lsn - } - - fn get_end_lsn(&self) -> Lsn { + fn get_lsn_range(&self) -> Range { // End-bound is exclusive - self.lsn + 1 + self.lsn..(self.lsn + 1) } /// Look up given page in the file - fn get_page_reconstruct_data( + fn get_value_reconstruct_data( &self, - blknum: SegmentBlk, - lsn: Lsn, - reconstruct_data: &mut PageReconstructData, - ) -> anyhow::Result { - ensure!((0..RELISH_SEG_SIZE).contains(&blknum)); - ensure!(lsn >= self.lsn); - - match reconstruct_data.page_img { - Some((cached_lsn, _)) if self.lsn <= cached_lsn => { - return Ok(PageReconstructResult::Complete) - } - _ => {} - } + key: Key, + lsn_range: Range, + reconstruct_state: &mut ValueReconstructState, + ) -> anyhow::Result { + assert!(self.key_range.contains(&key)); + assert!(lsn_range.end >= self.lsn); let inner = self.load()?; - let buf = match &inner.image_type { - ImageType::Blocky { num_blocks } => { - // Check if the request is beyond EOF - if blknum >= *num_blocks { - return Ok(PageReconstructResult::Missing(lsn)); - } + if let Some(blob_ref) = inner.index.get(&key) { + let chapter = inner + .book + .as_ref() + .unwrap() + .chapter_reader(VALUES_CHAPTER)?; - let mut buf = vec![0u8; BLOCK_SIZE]; - let offset = BLOCK_SIZE as u64 * blknum as u64; - - let chapter = inner - .book - .as_ref() - .unwrap() - .chapter_reader(BLOCKY_IMAGES_CHAPTER)?; - - chapter.read_exact_at(&mut buf, offset).with_context(|| { + let mut blob = vec![0; blob_ref.size()]; + chapter + .read_exact_at(&mut blob, blob_ref.pos()) + .with_context(|| { format!( - "failed to read page from data file {} at offset {}", + "failed to read {} bytes from data file {} at offset {}", + blob_ref.size(), self.filename().display(), - offset + blob_ref.pos() ) })?; + let value = Bytes::from(blob); - buf - } - ImageType::NonBlocky => { - ensure!(blknum == 0); - inner - .book - .as_ref() - .unwrap() - .read_chapter(NONBLOCKY_IMAGE_CHAPTER)? - .into_vec() - } - }; - - reconstruct_data.page_img = Some((self.lsn, Bytes::from(buf))); - Ok(PageReconstructResult::Complete) - } - - /// Get size of the segment - fn get_seg_size(&self, _lsn: Lsn) -> Result { - let inner = self.load()?; - match inner.image_type { - ImageType::Blocky { num_blocks } => Ok(num_blocks), - ImageType::NonBlocky => Err(anyhow!("get_seg_size called for non-blocky segment")), + reconstruct_state.img = Some((self.lsn, value)); + Ok(ValueReconstructResult::Complete) + } else { + Ok(ValueReconstructResult::Missing) } } - /// Does this segment exist at given LSN? - fn get_seg_exists(&self, _lsn: Lsn) -> Result { - Ok(true) + fn iter(&self) -> Box>> { + todo!(); } fn unload(&self) -> Result<()> { + // Unload the index. + // + // TODO: we should access the index directly from pages on the disk, + // using the buffer cache. This load/unload mechanism is really ad hoc. + + // FIXME: In debug mode, loading and unloading the index slows + // things down so much that you get timeout errors. At least + // with the test_parallel_copy test. So as an even more ad hoc + // stopgap fix for that, only unload every on average 10 + // checkpoint cycles. + use rand::RngCore; + if rand::thread_rng().next_u32() > (u32::MAX / 10) { + return Ok(()); + } + + let mut inner = match self.inner.try_write() { + Ok(inner) => inner, + Err(TryLockError::WouldBlock) => return Ok(()), + Err(TryLockError::Poisoned(_)) => panic!("ImageLayer lock was poisoned"), + }; + inner.index = HashMap::default(); + inner.loaded = false; + Ok(()) } @@ -235,22 +214,22 @@ impl Layer for ImageLayer { /// debugging function to print out the contents of the layer fn dump(&self) -> Result<()> { println!( - "----- image layer for ten {} tli {} seg {} at {} ----", - self.tenantid, self.timelineid, self.seg, self.lsn + "----- image layer for ten {} tli {} key {}-{} at {} ----", + self.tenantid, self.timelineid, self.key_range.start, self.key_range.end, self.lsn ); let inner = self.load()?; - match inner.image_type { - ImageType::Blocky { num_blocks } => println!("({}) blocks ", num_blocks), - ImageType::NonBlocky => { - let chapter = inner - .book - .as_ref() - .unwrap() - .read_chapter(NONBLOCKY_IMAGE_CHAPTER)?; - println!("non-blocky ({} bytes)", chapter.len()); - } + let mut index_vec: Vec<(&Key, &BlobRef)> = inner.index.iter().collect(); + index_vec.sort_by_key(|x| x.1.pos()); + + for (key, blob_ref) in index_vec { + println!( + "key: {} size {} offset {}", + key, + blob_ref.size(), + blob_ref.pos() + ); } Ok(()) @@ -280,7 +259,7 @@ impl ImageLayer { loop { // Quick exit if already loaded let inner = self.inner.read().unwrap(); - if inner.book.is_some() { + if inner.loaded { return Ok(inner); } @@ -306,14 +285,16 @@ impl ImageLayer { fn load_inner(&self, inner: &mut ImageLayerInner) -> Result<()> { let path = self.path(); - let file = VirtualFile::open(&path) - .with_context(|| format!("Failed to open virtual file '{}'", path.display()))?; - let book = Book::new(file).with_context(|| { - format!( - "Failed to open virtual file '{}' as a bookfile", - path.display() - ) - })?; + + // Open the file if it's not open already. + if inner.book.is_none() { + let file = VirtualFile::open(&path) + .with_context(|| format!("Failed to open file '{}'", path.display()))?; + inner.book = Some(Book::new(file).with_context(|| { + format!("Failed to open file '{}' as a bookfile", path.display()) + })?); + } + let book = inner.book.as_ref().unwrap(); match &self.path_or_conf { PathOrConf::Conf(_) => { @@ -340,23 +321,13 @@ impl ImageLayer { } } - let image_type = if self.seg.rel.is_blocky() { - let chapter = book.chapter_reader(BLOCKY_IMAGES_CHAPTER)?; - let images_len = chapter.len(); - ensure!(images_len % BLOCK_SIZE as u64 == 0); - let num_blocks: SegmentBlk = (images_len / BLOCK_SIZE as u64).try_into()?; - ImageType::Blocky { num_blocks } - } else { - let _chapter = book.chapter_reader(NONBLOCKY_IMAGE_CHAPTER)?; - ImageType::NonBlocky - }; + let chapter = book.read_chapter(INDEX_CHAPTER)?; + let index = HashMap::des(&chapter)?; - debug!("loaded from {}", &path.display()); + info!("loaded from {}", &path.display()); - *inner = ImageLayerInner { - book: Some(book), - image_type, - }; + inner.index = index; + inner.loaded = true; Ok(()) } @@ -372,11 +343,12 @@ impl ImageLayer { path_or_conf: PathOrConf::Conf(conf), timelineid, tenantid, - seg: filename.seg, + key_range: filename.key_range.clone(), lsn: filename.lsn, inner: RwLock::new(ImageLayerInner { book: None, - image_type: ImageType::Blocky { num_blocks: 0 }, + index: HashMap::new(), + loaded: false, }), } } @@ -395,18 +367,19 @@ impl ImageLayer { path_or_conf: PathOrConf::Path(path.to_path_buf()), timelineid: summary.timelineid, tenantid: summary.tenantid, - seg: summary.seg, + key_range: summary.key_range, lsn: summary.lsn, inner: RwLock::new(ImageLayerInner { book: None, - image_type: ImageType::Blocky { num_blocks: 0 }, + index: HashMap::new(), + loaded: false, }), }) } fn layer_name(&self) -> ImageFileName { ImageFileName { - seg: self.seg, + key_range: self.key_range.clone(), lsn: self.lsn, } } @@ -435,15 +408,18 @@ impl ImageLayer { /// pub struct ImageLayerWriter { conf: &'static PageServerConf, + path: PathBuf, timelineid: ZTimelineId, tenantid: ZTenantId, - seg: SegmentTag, + key_range: Range, lsn: Lsn, - num_blocks: SegmentBlk, + values_writer: Option>>, + end_offset: u64, - page_image_writer: ChapterWriter>, - num_blocks_written: SegmentBlk, + index: HashMap, + + finished: bool, } impl ImageLayerWriter { @@ -451,9 +427,8 @@ impl ImageLayerWriter { conf: &'static PageServerConf, timelineid: ZTimelineId, tenantid: ZTenantId, - seg: SegmentTag, + key_range: &Range, lsn: Lsn, - num_blocks: SegmentBlk, ) -> anyhow::Result { // Create the file // @@ -463,70 +438,75 @@ impl ImageLayerWriter { &PathOrConf::Conf(conf), timelineid, tenantid, - &ImageFileName { seg, lsn }, + &ImageFileName { + key_range: key_range.clone(), + lsn, + }, ); + info!("new image layer {}", path.display()); let file = VirtualFile::create(&path)?; let buf_writer = BufWriter::new(file); let book = BookWriter::new(buf_writer, IMAGE_FILE_MAGIC)?; // Open the page-images chapter for writing. The calls to - // `put_page_image` will use this to write the contents. - let chapter = if seg.rel.is_blocky() { - book.new_chapter(BLOCKY_IMAGES_CHAPTER) - } else { - ensure!(num_blocks == 1); - book.new_chapter(NONBLOCKY_IMAGE_CHAPTER) - }; + // `put_image` will use this to write the contents. + let chapter = book.new_chapter(VALUES_CHAPTER); let writer = ImageLayerWriter { conf, + path, timelineid, tenantid, - seg, + key_range: key_range.clone(), lsn, - num_blocks, - page_image_writer: chapter, - num_blocks_written: 0, + values_writer: Some(chapter), + index: HashMap::new(), + end_offset: 0, + finished: false, }; Ok(writer) } /// - /// Write next page image to the file. + /// Write next value to the file. /// /// The page versions must be appended in blknum order. /// - pub fn put_page_image(&mut self, block_bytes: &[u8]) -> anyhow::Result<()> { - ensure!(self.num_blocks_written < self.num_blocks); - if self.seg.rel.is_blocky() { - ensure!(block_bytes.len() == BLOCK_SIZE); + pub fn put_image(&mut self, key: Key, img: &[u8]) -> Result<()> { + ensure!(self.key_range.contains(&key)); + let off = self.end_offset; + + if let Some(writer) = &mut self.values_writer { + let len = img.len(); + writer.write_all(img)?; + self.end_offset += len as u64; + + let old = self.index.insert(key, BlobRef::new(off, len, true)); + assert!(old.is_none()); + } else { + panic!() } - self.page_image_writer.write_all(block_bytes)?; - self.num_blocks_written += 1; + Ok(()) } - pub fn finish(self) -> anyhow::Result { - // Check that the `put_page_image' was called for every block. - ensure!(self.num_blocks_written == self.num_blocks); + pub fn finish(&mut self) -> anyhow::Result { + // Close the values chapter + let book = self.values_writer.take().unwrap().close()?; - // Close the page-images chapter - let book = self.page_image_writer.close()?; + // Write out the index + let mut chapter = book.new_chapter(INDEX_CHAPTER); + let buf = HashMap::ser(&self.index)?; + chapter.write_all(&buf)?; + let book = chapter.close()?; // Write out the summary chapter - let image_type = if self.seg.rel.is_blocky() { - ImageType::Blocky { - num_blocks: self.num_blocks, - } - } else { - ImageType::NonBlocky - }; let mut chapter = book.new_chapter(SUMMARY_CHAPTER); let summary = Summary { tenantid: self.tenantid, timelineid: self.timelineid, - seg: self.seg, + key_range: self.key_range.clone(), lsn: self.lsn, }; Summary::ser_into(&summary, &mut chapter)?; @@ -542,15 +522,31 @@ impl ImageLayerWriter { path_or_conf: PathOrConf::Conf(self.conf), timelineid: self.timelineid, tenantid: self.tenantid, - seg: self.seg, + key_range: self.key_range.clone(), lsn: self.lsn, inner: RwLock::new(ImageLayerInner { book: None, - image_type, + loaded: false, + index: HashMap::new(), }), }; trace!("created image layer {}", layer.path().display()); + self.finished = true; + Ok(layer) } } + +impl Drop for ImageLayerWriter { + fn drop(&mut self) { + if let Some(page_image_writer) = self.values_writer.take() { + if let Ok(book) = page_image_writer.close() { + let _ = book.close(); + } + } + if !self.finished { + let _ = fs::remove_file(&self.path); + } + } +} diff --git a/pageserver/src/layered_repository/inmemory_layer.rs b/pageserver/src/layered_repository/inmemory_layer.rs index fed1fb6469..b5d98a4ca3 100644 --- a/pageserver/src/layered_repository/inmemory_layer.rs +++ b/pageserver/src/layered_repository/inmemory_layer.rs @@ -1,30 +1,29 @@ -//! An in-memory layer stores recently received PageVersions. -//! The page versions are held in a BTreeMap. To avoid OOM errors, the map size is limited -//! and layers can be spilled to disk into ephemeral files. +//! An in-memory layer stores recently received key-value pairs. //! -//! And there's another BTreeMap to track the size of the relation. +//! The "in-memory" part of the name is a bit misleading: the actual page versions are +//! held in an ephemeral file, not in memory. The metadata for each page version, i.e. +//! its position in the file, is kept in memory, though. //! use crate::config::PageServerConf; use crate::layered_repository::delta_layer::{DeltaLayer, DeltaLayerWriter}; use crate::layered_repository::ephemeral_file::EphemeralFile; -use crate::layered_repository::filename::DeltaFileName; -use crate::layered_repository::image_layer::{ImageLayer, ImageLayerWriter}; use crate::layered_repository::storage_layer::{ - Layer, PageReconstructData, PageReconstructResult, PageVersion, SegmentBlk, SegmentTag, - RELISH_SEG_SIZE, + BlobRef, Layer, ValueReconstructResult, ValueReconstructState, }; -use crate::layered_repository::LayeredTimeline; -use crate::layered_repository::ZERO_PAGE; -use crate::repository::ZenithWalRecord; +use crate::repository::{Key, Value}; +use crate::walrecord; use crate::{ZTenantId, ZTimelineId}; use anyhow::{bail, ensure, Result}; -use bytes::Bytes; use log::*; use std::collections::HashMap; -use std::io::Seek; +// avoid binding to Write (conflicts with std::io::Write) +// while being able to use std::fmt::Write's methods +use std::fmt::Write as _; +use std::io::Write; +use std::ops::Range; use std::os::unix::fs::FileExt; use std::path::PathBuf; -use std::sync::{Arc, RwLock}; +use std::sync::RwLock; use zenith_utils::bin_ser::BeSer; use zenith_utils::lsn::Lsn; use zenith_utils::vec_map::VecMap; @@ -33,7 +32,6 @@ pub struct InMemoryLayer { conf: &'static PageServerConf, tenantid: ZTenantId, timelineid: ZTimelineId, - seg: SegmentTag, /// /// This layer contains all the changes from 'start_lsn'. The @@ -41,27 +39,9 @@ pub struct InMemoryLayer { /// start_lsn: Lsn, - /// - /// LSN of the oldest page version stored in this layer. - /// - /// This is different from 'start_lsn' in that we enforce that the 'start_lsn' - /// of a layer always matches the 'end_lsn' of its predecessor, even if there - /// are no page versions until at a later LSN. That way you can detect any - /// missing layer files more easily. 'oldest_lsn' is the first page version - /// actually stored in this layer. In the range between 'start_lsn' and - /// 'oldest_lsn', there are no changes to the segment. - /// 'oldest_lsn' is used to adjust 'disk_consistent_lsn' and that is why it should - /// point to the beginning of WAL record. This is the other difference with 'start_lsn' - /// which points to end of WAL record. This is why 'oldest_lsn' can be smaller than 'start_lsn'. - /// - oldest_lsn: Lsn, - /// The above fields never change. The parts that do change are in 'inner', /// and protected by mutex. inner: RwLock, - - /// Predecessor layer might be needed? - incremental: bool, } pub struct InMemoryLayerInner { @@ -69,98 +49,25 @@ pub struct InMemoryLayerInner { /// Writes are only allowed when this is None end_lsn: Option, - /// If this relation was dropped, remember when that happened. - /// The drop LSN is recorded in [`end_lsn`]. - dropped: bool, + /// + /// All versions of all pages in the layer are kept here. Indexed + /// by block number and LSN. The value is an offset into the + /// ephemeral file where the page version is stored. + /// + index: HashMap>, - /// The PageVersion structs are stored in a serialized format in this file. - /// Each serialized PageVersion is preceded by a 'u32' length field. - /// 'page_versions' map stores offsets into this file. + /// The values are stored in a serialized format in this file. + /// Each serialized Value is preceded by a 'u32' length field. + /// PerSeg::page_versions map stores offsets into this file. file: EphemeralFile, - /// Metadata about all versions of all pages in the layer is kept - /// here. Indexed by block number and LSN. The value is an offset - /// into the ephemeral file where the page version is stored. - page_versions: HashMap>, - - /// - /// `seg_sizes` tracks the size of the segment at different points in time. - /// - /// For a blocky rel, there is always one entry, at the layer's start_lsn, - /// so that determining the size never depends on the predecessor layer. For - /// a non-blocky rel, 'seg_sizes' is not used and is always empty. - /// - seg_sizes: VecMap, - - /// - /// LSN of the newest page version stored in this layer. - /// - /// The difference between 'end_lsn' and 'latest_lsn' is the same as between - /// 'start_lsn' and 'oldest_lsn'. See comments in 'oldest_lsn'. - /// - latest_lsn: Lsn, + end_offset: u64, } impl InMemoryLayerInner { fn assert_writeable(&self) { assert!(self.end_lsn.is_none()); } - - fn get_seg_size(&self, lsn: Lsn) -> SegmentBlk { - // Scan the BTreeMap backwards, starting from the given entry. - let slice = self.seg_sizes.slice_range(..=lsn); - - // We make sure there is always at least one entry - if let Some((_entry_lsn, entry)) = slice.last() { - *entry - } else { - panic!("could not find seg size in in-memory layer"); - } - } - - /// - /// Read a page version from the ephemeral file. - /// - fn read_pv(&self, off: u64) -> Result { - let mut buf = Vec::new(); - self.read_pv_bytes(off, &mut buf)?; - Ok(PageVersion::des(&buf)?) - } - - /// - /// Read a page version from the ephemeral file, as raw bytes, at - /// the given offset. The bytes are read into 'buf', which is - /// expanded if necessary. Returns the size of the page version. - /// - fn read_pv_bytes(&self, off: u64, buf: &mut Vec) -> Result { - // read length - let mut lenbuf = [0u8; 4]; - self.file.read_exact_at(&mut lenbuf, off)?; - let len = u32::from_ne_bytes(lenbuf) as usize; - - if buf.len() < len { - buf.resize(len, 0); - } - self.file.read_exact_at(&mut buf[0..len], off + 4)?; - Ok(len) - } - - fn write_pv(&mut self, pv: &PageVersion) -> Result { - // remember starting position - let pos = self.file.stream_position()?; - - // make room for the 'length' field by writing zeros as a placeholder. - self.file.seek(std::io::SeekFrom::Start(pos + 4))?; - - pv.ser_into(&mut self.file)?; - - // write the 'length' field. - let len = self.file.stream_position()? - pos - 4; - let lenbuf = u32::to_ne_bytes(len as u32); - self.file.write_all_at(&lenbuf, pos)?; - - Ok(pos) - } } impl Layer for InMemoryLayer { @@ -170,21 +77,12 @@ impl Layer for InMemoryLayer { fn filename(&self) -> PathBuf { let inner = self.inner.read().unwrap(); - let end_lsn = if let Some(drop_lsn) = inner.end_lsn { - drop_lsn - } else { - Lsn(u64::MAX) - }; + let end_lsn = inner.end_lsn.unwrap_or(Lsn(u64::MAX)); - let delta_filename = DeltaFileName { - seg: self.seg, - start_lsn: self.start_lsn, - end_lsn, - dropped: inner.dropped, - } - .to_string(); - - PathBuf::from(format!("inmem-{}", delta_filename)) + PathBuf::from(format!( + "inmem-{:016X}-{:016X}", + self.start_lsn.0, end_lsn.0 + )) } fn get_tenant_id(&self) -> ZTenantId { @@ -195,132 +93,78 @@ impl Layer for InMemoryLayer { self.timelineid } - fn get_seg_tag(&self) -> SegmentTag { - self.seg + fn get_key_range(&self) -> Range { + Key::MIN..Key::MAX } - fn get_start_lsn(&self) -> Lsn { - self.start_lsn - } - - fn get_end_lsn(&self) -> Lsn { + fn get_lsn_range(&self) -> Range { let inner = self.inner.read().unwrap(); - if let Some(end_lsn) = inner.end_lsn { + let end_lsn = if let Some(end_lsn) = inner.end_lsn { end_lsn } else { Lsn(u64::MAX) - } + }; + self.start_lsn..end_lsn } - fn is_dropped(&self) -> bool { - let inner = self.inner.read().unwrap(); - inner.dropped - } - - /// Look up given page in the cache. - fn get_page_reconstruct_data( + /// Look up given value in the layer. + fn get_value_reconstruct_data( &self, - blknum: SegmentBlk, - lsn: Lsn, - reconstruct_data: &mut PageReconstructData, - ) -> anyhow::Result { + key: Key, + lsn_range: Range, + reconstruct_state: &mut ValueReconstructState, + ) -> anyhow::Result { + ensure!(lsn_range.start <= self.start_lsn); let mut need_image = true; - ensure!((0..RELISH_SEG_SIZE).contains(&blknum)); + let inner = self.inner.read().unwrap(); - { - let inner = self.inner.read().unwrap(); - - // Scan the page versions backwards, starting from `lsn`. - if let Some(vec_map) = inner.page_versions.get(&blknum) { - let slice = vec_map.slice_range(..=lsn); - for (entry_lsn, pos) in slice.iter().rev() { - match &reconstruct_data.page_img { - Some((cached_lsn, _)) if entry_lsn <= cached_lsn => { - return Ok(PageReconstructResult::Complete) - } - _ => {} + // Scan the page versions backwards, starting from `lsn`. + if let Some(vec_map) = inner.index.get(&key) { + let slice = vec_map.slice_range(lsn_range); + for (entry_lsn, blob_ref) in slice.iter().rev() { + match &reconstruct_state.img { + Some((cached_lsn, _)) if entry_lsn <= cached_lsn => { + return Ok(ValueReconstructResult::Complete) } + _ => {} + } - let pv = inner.read_pv(*pos)?; - match pv { - PageVersion::Page(img) => { - reconstruct_data.page_img = Some((*entry_lsn, img)); + let mut buf = vec![0u8; blob_ref.size()]; + inner.file.read_exact_at(&mut buf, blob_ref.pos())?; + let value = Value::des(&buf)?; + match value { + Value::Image(img) => { + reconstruct_state.img = Some((*entry_lsn, img)); + return Ok(ValueReconstructResult::Complete); + } + Value::WalRecord(rec) => { + let will_init = rec.will_init(); + reconstruct_state.records.push((*entry_lsn, rec)); + if will_init { + // This WAL record initializes the page, so no need to go further back need_image = false; break; } - PageVersion::Wal(rec) => { - reconstruct_data.records.push((*entry_lsn, rec.clone())); - if rec.will_init() { - // This WAL record initializes the page, so no need to go further back - need_image = false; - break; - } - } } } } - - // If we didn't find any records for this, check if the request is beyond EOF - if need_image - && reconstruct_data.records.is_empty() - && self.seg.rel.is_blocky() - && blknum >= self.get_seg_size(lsn)? - { - return Ok(PageReconstructResult::Missing(self.start_lsn)); - } - - // release lock on 'inner' } + // release lock on 'inner' + // If an older page image is needed to reconstruct the page, let the - // caller know + // caller know. if need_image { - if self.incremental { - Ok(PageReconstructResult::Continue(Lsn(self.start_lsn.0 - 1))) - } else { - Ok(PageReconstructResult::Missing(self.start_lsn)) - } + Ok(ValueReconstructResult::Continue) } else { - Ok(PageReconstructResult::Complete) + Ok(ValueReconstructResult::Complete) } } - /// Get size of the relation at given LSN - fn get_seg_size(&self, lsn: Lsn) -> anyhow::Result { - ensure!(lsn >= self.start_lsn); - ensure!( - self.seg.rel.is_blocky(), - "get_seg_size() called on a non-blocky rel" - ); - - let inner = self.inner.read().unwrap(); - Ok(inner.get_seg_size(lsn)) - } - - /// Does this segment exist at given LSN? - fn get_seg_exists(&self, lsn: Lsn) -> anyhow::Result { - let inner = self.inner.read().unwrap(); - - // If the segment created after requested LSN, - // it doesn't exist in the layer. But we shouldn't - // have requested it in the first place. - ensure!(lsn >= self.start_lsn); - - // Is the requested LSN after the segment was dropped? - if inner.dropped { - if let Some(end_lsn) = inner.end_lsn { - if lsn >= end_lsn { - return Ok(false); - } - } else { - bail!("dropped in-memory layer with no end LSN"); - } - } - - // Otherwise, it exists - Ok(true) + fn iter(&self) -> Box>> { + todo!(); } /// Cannot unload anything in an in-memory layer, since there's no backing @@ -337,7 +181,8 @@ impl Layer for InMemoryLayer { } fn is_incremental(&self) -> bool { - self.incremental + // in-memory layer is always considered incremental. + true } fn is_in_memory(&self) -> bool { @@ -355,29 +200,36 @@ impl Layer for InMemoryLayer { .unwrap_or_default(); println!( - "----- in-memory layer for tli {} seg {} {}-{} {} ----", - self.timelineid, self.seg, self.start_lsn, end_str, inner.dropped, + "----- in-memory layer for tli {} LSNs {}-{} ----", + self.timelineid, self.start_lsn, end_str, ); - for (k, v) in inner.seg_sizes.as_slice() { - println!("seg_sizes {}: {}", k, v); - } - - // List the blocks in order - let mut page_versions: Vec<(&SegmentBlk, &VecMap)> = - inner.page_versions.iter().collect(); - page_versions.sort_by_key(|k| k.0); - - for (blknum, versions) in page_versions { - for (lsn, off) in versions.as_slice() { - let pv = inner.read_pv(*off); - let pv_description = match pv { - Ok(PageVersion::Page(_img)) => "page", - Ok(PageVersion::Wal(_rec)) => "wal", - Err(_err) => "INVALID", - }; - - println!("blk {} at {}: {}\n", blknum, lsn, pv_description); + let mut buf = Vec::new(); + for (key, vec_map) in inner.index.iter() { + for (lsn, blob_ref) in vec_map.as_slice() { + let mut desc = String::new(); + buf.resize(blob_ref.size(), 0); + inner.file.read_exact_at(&mut buf, blob_ref.pos())?; + let val = Value::des(&buf); + match val { + Ok(Value::Image(img)) => { + write!(&mut desc, " img {} bytes", img.len())?; + } + Ok(Value::WalRecord(rec)) => { + let wal_desc = walrecord::describe_wal_record(&rec); + write!( + &mut desc, + " rec {} bytes will_init: {} {}", + buf.len(), + rec.will_init(), + wal_desc + )?; + } + Err(err) => { + write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?; + } + } + println!(" key {} at {}: {}", key, lsn, desc); } } @@ -385,23 +237,7 @@ impl Layer for InMemoryLayer { } } -/// A result of an inmemory layer data being written to disk. -pub struct LayersOnDisk { - pub delta_layers: Vec, - pub image_layers: Vec, -} - impl InMemoryLayer { - /// Return the oldest page version that's stored in this layer - pub fn get_oldest_lsn(&self) -> Lsn { - self.oldest_lsn - } - - pub fn get_latest_lsn(&self) -> Lsn { - let inner = self.inner.read().unwrap(); - inner.latest_lsn - } - /// /// Create a new, empty, in-memory layer /// @@ -409,291 +245,83 @@ impl InMemoryLayer { conf: &'static PageServerConf, timelineid: ZTimelineId, tenantid: ZTenantId, - seg: SegmentTag, start_lsn: Lsn, - oldest_lsn: Lsn, ) -> Result { trace!( - "initializing new empty InMemoryLayer for writing {} on timeline {} at {}", - seg, + "initializing new empty InMemoryLayer for writing on timeline {} at {}", timelineid, start_lsn ); - // The segment is initially empty, so initialize 'seg_sizes' with 0. - let mut seg_sizes = VecMap::default(); - if seg.rel.is_blocky() { - seg_sizes.append(start_lsn, 0).unwrap(); - } - let file = EphemeralFile::create(conf, tenantid, timelineid)?; Ok(InMemoryLayer { conf, timelineid, tenantid, - seg, start_lsn, - oldest_lsn, - incremental: false, inner: RwLock::new(InMemoryLayerInner { end_lsn: None, - dropped: false, + index: HashMap::new(), file, - page_versions: HashMap::new(), - seg_sizes, - latest_lsn: oldest_lsn, + end_offset: 0, }), }) } // Write operations - /// Remember new page version, as a WAL record over previous version - pub fn put_wal_record( - &self, - lsn: Lsn, - blknum: SegmentBlk, - rec: ZenithWalRecord, - ) -> Result { - self.put_page_version(blknum, lsn, PageVersion::Wal(rec)) - } - - /// Remember new page version, as a full page image - pub fn put_page_image(&self, blknum: SegmentBlk, lsn: Lsn, img: Bytes) -> Result { - self.put_page_version(blknum, lsn, PageVersion::Page(img)) - } - /// Common subroutine of the public put_wal_record() and put_page_image() functions. /// Adds the page version to the in-memory tree - pub fn put_page_version( - &self, - blknum: SegmentBlk, - lsn: Lsn, - pv: PageVersion, - ) -> anyhow::Result { - ensure!((0..RELISH_SEG_SIZE).contains(&blknum)); - - trace!( - "put_page_version blk {} of {} at {}/{}", - blknum, - self.seg.rel, - self.timelineid, - lsn - ); + pub fn put_value(&self, key: Key, lsn: Lsn, val: Value) -> Result<()> { + trace!("put_value key {} at {}/{}", key, self.timelineid, lsn); let mut inner = self.inner.write().unwrap(); inner.assert_writeable(); - ensure!(lsn >= inner.latest_lsn); - inner.latest_lsn = lsn; - // Write the page version to the file, and remember its offset in 'page_versions' - { - let off = inner.write_pv(&pv)?; - let vec_map = inner.page_versions.entry(blknum).or_default(); - let old = vec_map.append_or_update_last(lsn, off).unwrap().0; - if old.is_some() { - // We already had an entry for this LSN. That's odd.. - warn!( - "Page version of rel {} blk {} at {} already exists", - self.seg.rel, blknum, lsn - ); - } - } - - // Also update the relation size, if this extended the relation. - if self.seg.rel.is_blocky() { - let newsize = blknum + 1; - - // use inner get_seg_size, since calling self.get_seg_size will try to acquire the lock, - // which we've just acquired above - let oldsize = inner.get_seg_size(lsn); - if newsize > oldsize { - trace!( - "enlarging segment {} from {} to {} blocks at {}", - self.seg, - oldsize, - newsize, - lsn - ); - - // If we are extending the relation by more than one page, initialize the "gap" - // with zeros - // - // XXX: What if the caller initializes the gap with subsequent call with same LSN? - // I don't think that can happen currently, but that is highly dependent on how - // PostgreSQL writes its WAL records and there's no guarantee of it. If it does - // happen, we would hit the "page version already exists" warning above on the - // subsequent call to initialize the gap page. - for gapblknum in oldsize..blknum { - let zeropv = PageVersion::Page(ZERO_PAGE.clone()); - trace!( - "filling gap blk {} with zeros for write of {}", - gapblknum, - blknum - ); - - // Write the page version to the file, and remember its offset in - // 'page_versions' - { - let off = inner.write_pv(&zeropv)?; - let vec_map = inner.page_versions.entry(gapblknum).or_default(); - let old = vec_map.append_or_update_last(lsn, off).unwrap().0; - if old.is_some() { - warn!( - "Page version of seg {} blk {} at {} already exists", - self.seg, gapblknum, lsn - ); - } - } - } - - inner.seg_sizes.append_or_update_last(lsn, newsize).unwrap(); - return Ok(newsize - oldsize); - } - } - - Ok(0) - } - - /// Remember that the relation was truncated at given LSN - pub fn put_truncation(&self, lsn: Lsn, new_size: SegmentBlk) { - assert!( - self.seg.rel.is_blocky(), - "put_truncation() called on a non-blocky rel" - ); - - let mut inner = self.inner.write().unwrap(); - inner.assert_writeable(); - - // check that this we truncate to a smaller size than segment was before the truncation - let old_size = inner.get_seg_size(lsn); - assert!(new_size < old_size); - - let (old, _delta_size) = inner - .seg_sizes - .append_or_update_last(lsn, new_size) - .unwrap(); + let off = inner.end_offset; + let buf = Value::ser(&val)?; + let len = buf.len(); + inner.file.write_all(&buf)?; + inner.end_offset += len as u64; + let vec_map = inner.index.entry(key).or_default(); + let blob_ref = BlobRef::new(off, len, val.will_init()); + let old = vec_map.append_or_update_last(lsn, blob_ref).unwrap().0; if old.is_some() { // We already had an entry for this LSN. That's odd.. - warn!("Inserting truncation, but had an entry for the LSN already"); - } - } - - /// Remember that the segment was dropped at given LSN - pub fn drop_segment(&self, lsn: Lsn) { - let mut inner = self.inner.write().unwrap(); - - assert!(inner.end_lsn.is_none()); - assert!(!inner.dropped); - inner.dropped = true; - assert!(self.start_lsn < lsn); - inner.end_lsn = Some(lsn); - - trace!("dropped segment {} at {}", self.seg, lsn); - } - - /// - /// Initialize a new InMemoryLayer for, by copying the state at the given - /// point in time from given existing layer. - /// - pub fn create_successor_layer( - conf: &'static PageServerConf, - src: Arc, - timelineid: ZTimelineId, - tenantid: ZTenantId, - start_lsn: Lsn, - oldest_lsn: Lsn, - ) -> Result { - let seg = src.get_seg_tag(); - - assert!(oldest_lsn.is_aligned()); - - trace!( - "initializing new InMemoryLayer for writing {} on timeline {} at {}", - seg, - timelineid, - start_lsn, - ); - - // Copy the segment size at the start LSN from the predecessor layer. - let mut seg_sizes = VecMap::default(); - if seg.rel.is_blocky() { - let size = src.get_seg_size(start_lsn)?; - seg_sizes.append(start_lsn, size).unwrap(); + warn!("Key {} at {} already exists", key, lsn); } - let file = EphemeralFile::create(conf, tenantid, timelineid)?; - - Ok(InMemoryLayer { - conf, - timelineid, - tenantid, - seg, - start_lsn, - oldest_lsn, - incremental: true, - inner: RwLock::new(InMemoryLayerInner { - end_lsn: None, - dropped: false, - file, - page_versions: HashMap::new(), - seg_sizes, - latest_lsn: oldest_lsn, - }), - }) + Ok(()) } - pub fn is_writeable(&self) -> bool { - let inner = self.inner.read().unwrap(); - inner.end_lsn.is_none() + pub fn put_tombstone(&self, _key_range: Range, _lsn: Lsn) -> Result<()> { + // TODO: Currently, we just leak the storage for any deleted keys + + Ok(()) } /// Make the layer non-writeable. Only call once. /// Records the end_lsn for non-dropped layers. - /// `end_lsn` is inclusive + /// `end_lsn` is exclusive pub fn freeze(&self, end_lsn: Lsn) { let mut inner = self.inner.write().unwrap(); - if inner.end_lsn.is_some() { - assert!(inner.dropped); - } else { - assert!(!inner.dropped); - assert!(self.start_lsn < end_lsn + 1); - inner.end_lsn = Some(Lsn(end_lsn.0 + 1)); + assert!(self.start_lsn < end_lsn); + inner.end_lsn = Some(end_lsn); - if let Some((lsn, _)) = inner.seg_sizes.as_slice().last() { - assert!(lsn <= &end_lsn, "{:?} {:?}", lsn, end_lsn); - } - - for (_blk, vec_map) in inner.page_versions.iter() { - for (lsn, _pos) in vec_map.as_slice() { - assert!(*lsn <= end_lsn); - } + for vec_map in inner.index.values() { + for (lsn, _pos) in vec_map.as_slice() { + assert!(*lsn < end_lsn); } } } - /// Write the this frozen in-memory layer to disk. + /// Write this frozen in-memory layer to disk. /// - /// Returns new layers that replace this one. - /// If not dropped and reconstruct_pages is true, returns a new image layer containing the page versions - /// at the `end_lsn`. Can also return a DeltaLayer that includes all the - /// WAL records between start and end LSN. (The delta layer is not needed - /// when a new relish is created with a single LSN, so that the start and - /// end LSN are the same.) - pub fn write_to_disk( - &self, - timeline: &LayeredTimeline, - reconstruct_pages: bool, - ) -> Result { - trace!( - "write_to_disk {} get_end_lsn is {}", - self.filename().display(), - self.get_end_lsn() - ); - + /// Returns a new delta layer with all the same data as this in-memory layer + pub fn write_to_disk(&self) -> Result { // Grab the lock in read-mode. We hold it over the I/O, but because this // layer is not writeable anymore, no one should be trying to acquire the // write lock on it, so we shouldn't block anyone. There's one exception @@ -705,105 +333,32 @@ impl InMemoryLayer { // rare though, so we just accept the potential latency hit for now. let inner = self.inner.read().unwrap(); - // Since `end_lsn` is exclusive, subtract 1 to calculate the last LSN - // that is included. - let end_lsn_exclusive = inner.end_lsn.unwrap(); - let end_lsn_inclusive = Lsn(end_lsn_exclusive.0 - 1); + let mut delta_layer_writer = DeltaLayerWriter::new( + self.conf, + self.timelineid, + self.tenantid, + Key::MIN, + self.start_lsn..inner.end_lsn.unwrap(), + )?; - // Figure out if we should create a delta layer, image layer, or both. - let image_lsn: Option; - let delta_end_lsn: Option; - if self.is_dropped() || !reconstruct_pages { - // The segment was dropped. Create just a delta layer containing all the - // changes up to and including the drop. - delta_end_lsn = Some(end_lsn_exclusive); - image_lsn = None; - } else if self.start_lsn == end_lsn_inclusive { - // The layer contains exactly one LSN. It's enough to write an image - // layer at that LSN. - delta_end_lsn = None; - image_lsn = Some(end_lsn_inclusive); - } else { - // Create a delta layer with all the changes up to the end LSN, - // and an image layer at the end LSN. - // - // Note that we the delta layer does *not* include the page versions - // at the end LSN. They are included in the image layer, and there's - // no need to store them twice. - delta_end_lsn = Some(end_lsn_inclusive); - image_lsn = Some(end_lsn_inclusive); - } - - let mut delta_layers = Vec::new(); - let mut image_layers = Vec::new(); - - if let Some(delta_end_lsn) = delta_end_lsn { - let mut delta_layer_writer = DeltaLayerWriter::new( - self.conf, - self.timelineid, - self.tenantid, - self.seg, - self.start_lsn, - delta_end_lsn, - self.is_dropped(), - )?; - - // Write all page versions, in block + LSN order - let mut buf: Vec = Vec::new(); - - let pv_iter = inner.page_versions.iter(); - let mut pages: Vec<(&SegmentBlk, &VecMap)> = pv_iter.collect(); - pages.sort_by_key(|(blknum, _vec_map)| *blknum); - for (blknum, vec_map) in pages { - for (lsn, pos) in vec_map.as_slice() { - if *lsn < delta_end_lsn { - let len = inner.read_pv_bytes(*pos, &mut buf)?; - delta_layer_writer.put_page_version(*blknum, *lsn, &buf[..len])?; - } + let mut do_steps = || -> Result<()> { + for (key, vec_map) in inner.index.iter() { + // Write all page versions + for (lsn, blob_ref) in vec_map.as_slice() { + let mut buf = vec![0u8; blob_ref.size()]; + inner.file.read_exact_at(&mut buf, blob_ref.pos())?; + let val = Value::des(&buf)?; + delta_layer_writer.put_value(*key, *lsn, val)?; } } - - // Create seg_sizes - let seg_sizes = if delta_end_lsn == end_lsn_exclusive { - inner.seg_sizes.clone() - } else { - inner.seg_sizes.split_at(&end_lsn_exclusive).0 - }; - - let delta_layer = delta_layer_writer.finish(seg_sizes)?; - delta_layers.push(delta_layer); + Ok(()) + }; + if let Err(err) = do_steps() { + delta_layer_writer.abort(); + return Err(err); } - drop(inner); - - // Write a new base image layer at the cutoff point - if let Some(image_lsn) = image_lsn { - let size = if self.seg.rel.is_blocky() { - self.get_seg_size(image_lsn)? - } else { - 1 - }; - let mut image_layer_writer = ImageLayerWriter::new( - self.conf, - self.timelineid, - self.tenantid, - self.seg, - image_lsn, - size, - )?; - - for blknum in 0..size { - let img = timeline.materialize_page(self.seg, blknum, image_lsn, &*self)?; - - image_layer_writer.put_page_image(&img)?; - } - let image_layer = image_layer_writer.finish()?; - image_layers.push(image_layer); - } - - Ok(LayersOnDisk { - delta_layers, - image_layers, - }) + let delta_layer = delta_layer_writer.finish(Key::MAX)?; + Ok(delta_layer) } } diff --git a/pageserver/src/layered_repository/interval_tree.rs b/pageserver/src/layered_repository/interval_tree.rs deleted file mode 100644 index 978ecd837e..0000000000 --- a/pageserver/src/layered_repository/interval_tree.rs +++ /dev/null @@ -1,468 +0,0 @@ -/// -/// IntervalTree is data structure for holding intervals. It is generic -/// to make unit testing possible, but the only real user of it is the layer map, -/// -/// It's inspired by the "segment tree" or a "statistic tree" as described in -/// https://en.wikipedia.org/wiki/Segment_tree. However, we use a B-tree to hold -/// the points instead of a binary tree. This is called an "interval tree" instead -/// of "segment tree" because the term "segment" is already using Zenith to mean -/// something else. To add to the confusion, there is another data structure known -/// as "interval tree" out there (see https://en.wikipedia.org/wiki/Interval_tree), -/// for storing intervals, but this isn't that. -/// -/// The basic idea is to have a B-tree of "interesting Points". At each Point, -/// there is a list of intervals that contain the point. The Points are formed -/// from the start bounds of each interval; there is a Point for each distinct -/// start bound. -/// -/// Operations: -/// -/// To find intervals that contain a given point, you search the b-tree to find -/// the nearest Point <= search key. Then you just return the list of intervals. -/// -/// To insert an interval, find the Point with start key equal to the inserted item. -/// If the Point doesn't exist yet, create it, by copying all the items from the -/// previous Point that cover the new Point. Then walk right, inserting the new -/// interval to all the Points that are contained by the new interval (including the -/// newly created Point). -/// -/// To remove an interval, you scan the tree for all the Points that are contained by -/// the removed interval, and remove it from the list in each Point. -/// -/// Requirements and assumptions: -/// -/// - Can store overlapping items -/// - But there are not many overlapping items -/// - The interval bounds don't change after it is added to the tree -/// - Intervals are uniquely identified by pointer equality. You must not be insert the -/// same interval object twice, and `remove` uses pointer equality to remove the right -/// interval. It is OK to have two intervals with the same bounds, however. -/// -use std::collections::BTreeMap; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; - -pub struct IntervalTree -where - I: IntervalItem, -{ - points: BTreeMap>, -} - -struct Point { - /// All intervals that contain this point, in no particular order. - /// - /// We assume that there aren't a lot of overlappingg intervals, so that this vector - /// never grows very large. If that assumption doesn't hold, we could keep this ordered - /// by the end bound, to speed up `search`. But as long as there are only a few elements, - /// a linear search is OK. - elements: Vec>, -} - -/// Abstraction for an interval that can be stored in the tree -/// -/// The start bound is inclusive and the end bound is exclusive. End must be greater -/// than start. -pub trait IntervalItem { - type Key: Ord + Copy + Debug + Sized; - - fn start_key(&self) -> Self::Key; - fn end_key(&self) -> Self::Key; - - fn bounds(&self) -> Range { - self.start_key()..self.end_key() - } -} - -impl IntervalTree -where - I: IntervalItem, -{ - /// Return an element that contains 'key', or precedes it. - /// - /// If there are multiple candidates, returns the one with the highest 'end' key. - pub fn search(&self, key: I::Key) -> Option> { - // Find the greatest point that precedes or is equal to the search key. If there is - // none, returns None. - let (_, p) = self.points.range(..=key).next_back()?; - - // Find the element with the highest end key at this point - let highest_item = p - .elements - .iter() - .reduce(|a, b| { - // starting with Rust 1.53, could use `std::cmp::min_by_key` here - if a.end_key() > b.end_key() { - a - } else { - b - } - }) - .unwrap(); - Some(Arc::clone(highest_item)) - } - - /// Iterate over all items with start bound >= 'key' - pub fn iter_newer(&self, key: I::Key) -> IntervalIter { - IntervalIter { - point_iter: self.points.range(key..), - elem_iter: None, - } - } - - /// Iterate over all items - pub fn iter(&self) -> IntervalIter { - IntervalIter { - point_iter: self.points.range(..), - elem_iter: None, - } - } - - pub fn insert(&mut self, item: Arc) { - let start_key = item.start_key(); - let end_key = item.end_key(); - assert!(start_key < end_key); - let bounds = start_key..end_key; - - // Find the starting point and walk forward from there - let mut found_start_point = false; - let iter = self.points.range_mut(bounds); - for (point_key, point) in iter { - if *point_key == start_key { - found_start_point = true; - // It is an error to insert the same item to the tree twice. - assert!( - !point.elements.iter().any(|x| Arc::ptr_eq(x, &item)), - "interval is already in the tree" - ); - } - point.elements.push(Arc::clone(&item)); - } - if !found_start_point { - // Create a new Point for the starting point - - // Look at the previous point, and copy over elements that overlap with this - // new point - let mut new_elements: Vec> = Vec::new(); - if let Some((_, prev_point)) = self.points.range(..start_key).next_back() { - let overlapping_prev_elements = prev_point - .elements - .iter() - .filter(|x| x.bounds().contains(&start_key)) - .cloned(); - - new_elements.extend(overlapping_prev_elements); - } - new_elements.push(item); - - let new_point = Point { - elements: new_elements, - }; - self.points.insert(start_key, new_point); - } - } - - pub fn remove(&mut self, item: &Arc) { - // range search points - let start_key = item.start_key(); - let end_key = item.end_key(); - let bounds = start_key..end_key; - - let mut points_to_remove: Vec = Vec::new(); - let mut found_start_point = false; - for (point_key, point) in self.points.range_mut(bounds) { - if *point_key == start_key { - found_start_point = true; - } - let len_before = point.elements.len(); - point.elements.retain(|other| !Arc::ptr_eq(other, item)); - let len_after = point.elements.len(); - assert_eq!(len_after + 1, len_before); - if len_after == 0 { - points_to_remove.push(*point_key); - } - } - assert!(found_start_point); - - for k in points_to_remove { - self.points.remove(&k).unwrap(); - } - } -} - -pub struct IntervalIter<'a, I: ?Sized> -where - I: IntervalItem, -{ - point_iter: std::collections::btree_map::Range<'a, I::Key, Point>, - elem_iter: Option<(I::Key, std::slice::Iter<'a, Arc>)>, -} - -impl<'a, I> Iterator for IntervalIter<'a, I> -where - I: IntervalItem + ?Sized, -{ - type Item = Arc; - - fn next(&mut self) -> Option { - // Iterate over all elements in all the points in 'point_iter'. To avoid - // returning the same element twice, we only return each element at its - // starting point. - loop { - // Return next remaining element from the current point - if let Some((point_key, elem_iter)) = &mut self.elem_iter { - for elem in elem_iter { - if elem.start_key() == *point_key { - return Some(Arc::clone(elem)); - } - } - } - // No more elements at this point. Move to next point. - if let Some((point_key, point)) = self.point_iter.next() { - self.elem_iter = Some((*point_key, point.elements.iter())); - continue; - } else { - // No more points, all done - return None; - } - } - } -} - -impl Default for IntervalTree -where - I: IntervalItem, -{ - fn default() -> Self { - IntervalTree { - points: BTreeMap::new(), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use std::fmt; - - #[derive(Debug)] - struct MockItem { - start_key: u32, - end_key: u32, - val: String, - } - impl IntervalItem for MockItem { - type Key = u32; - - fn start_key(&self) -> u32 { - self.start_key - } - fn end_key(&self) -> u32 { - self.end_key - } - } - impl MockItem { - fn new(start_key: u32, end_key: u32) -> Self { - MockItem { - start_key, - end_key, - val: format!("{}-{}", start_key, end_key), - } - } - fn new_str(start_key: u32, end_key: u32, val: &str) -> Self { - MockItem { - start_key, - end_key, - val: format!("{}-{}: {}", start_key, end_key, val), - } - } - } - impl fmt::Display for MockItem { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.val) - } - } - #[rustfmt::skip] - fn assert_search( - tree: &IntervalTree, - key: u32, - expected: &[&str], - ) -> Option> { - if let Some(v) = tree.search(key) { - let vstr = v.to_string(); - - assert!(!expected.is_empty(), "search with {} returned {}, expected None", key, v); - assert!( - expected.contains(&vstr.as_str()), - "search with {} returned {}, expected one of: {:?}", - key, v, expected, - ); - - Some(v) - } else { - assert!( - expected.is_empty(), - "search with {} returned None, expected one of {:?}", - key, expected - ); - None - } - } - - fn assert_contents(tree: &IntervalTree, expected: &[&str]) { - let mut contents: Vec = tree.iter().map(|e| e.to_string()).collect(); - contents.sort(); - assert_eq!(contents, expected); - } - - fn dump_tree(tree: &IntervalTree) { - for (point_key, point) in tree.points.iter() { - print!("{}:", point_key); - for e in point.elements.iter() { - print!(" {}", e); - } - println!(); - } - } - - #[test] - fn test_interval_tree_simple() { - let mut tree: IntervalTree = IntervalTree::default(); - - // Simple, non-overlapping ranges. - tree.insert(Arc::new(MockItem::new(10, 11))); - tree.insert(Arc::new(MockItem::new(11, 12))); - tree.insert(Arc::new(MockItem::new(12, 13))); - tree.insert(Arc::new(MockItem::new(18, 19))); - tree.insert(Arc::new(MockItem::new(17, 18))); - tree.insert(Arc::new(MockItem::new(15, 16))); - - assert_search(&tree, 9, &[]); - assert_search(&tree, 10, &["10-11"]); - assert_search(&tree, 11, &["11-12"]); - assert_search(&tree, 12, &["12-13"]); - assert_search(&tree, 13, &["12-13"]); - assert_search(&tree, 14, &["12-13"]); - assert_search(&tree, 15, &["15-16"]); - assert_search(&tree, 16, &["15-16"]); - assert_search(&tree, 17, &["17-18"]); - assert_search(&tree, 18, &["18-19"]); - assert_search(&tree, 19, &["18-19"]); - assert_search(&tree, 20, &["18-19"]); - - // remove a few entries and search around them again - tree.remove(&assert_search(&tree, 10, &["10-11"]).unwrap()); // first entry - tree.remove(&assert_search(&tree, 12, &["12-13"]).unwrap()); // entry in the middle - tree.remove(&assert_search(&tree, 18, &["18-19"]).unwrap()); // last entry - assert_search(&tree, 9, &[]); - assert_search(&tree, 10, &[]); - assert_search(&tree, 11, &["11-12"]); - assert_search(&tree, 12, &["11-12"]); - assert_search(&tree, 14, &["11-12"]); - assert_search(&tree, 15, &["15-16"]); - assert_search(&tree, 17, &["17-18"]); - assert_search(&tree, 18, &["17-18"]); - } - - #[test] - fn test_interval_tree_overlap() { - let mut tree: IntervalTree = IntervalTree::default(); - - // Overlapping items - tree.insert(Arc::new(MockItem::new(22, 24))); - tree.insert(Arc::new(MockItem::new(23, 25))); - let x24_26 = Arc::new(MockItem::new(24, 26)); - tree.insert(Arc::clone(&x24_26)); - let x26_28 = Arc::new(MockItem::new(26, 28)); - tree.insert(Arc::clone(&x26_28)); - tree.insert(Arc::new(MockItem::new(25, 27))); - - assert_search(&tree, 22, &["22-24"]); - assert_search(&tree, 23, &["22-24", "23-25"]); - assert_search(&tree, 24, &["23-25", "24-26"]); - assert_search(&tree, 25, &["24-26", "25-27"]); - assert_search(&tree, 26, &["25-27", "26-28"]); - assert_search(&tree, 27, &["26-28"]); - assert_search(&tree, 28, &["26-28"]); - assert_search(&tree, 29, &["26-28"]); - - tree.remove(&x24_26); - tree.remove(&x26_28); - assert_search(&tree, 23, &["22-24", "23-25"]); - assert_search(&tree, 24, &["23-25"]); - assert_search(&tree, 25, &["25-27"]); - assert_search(&tree, 26, &["25-27"]); - assert_search(&tree, 27, &["25-27"]); - assert_search(&tree, 28, &["25-27"]); - assert_search(&tree, 29, &["25-27"]); - } - - #[test] - fn test_interval_tree_nested() { - let mut tree: IntervalTree = IntervalTree::default(); - - // Items containing other items - tree.insert(Arc::new(MockItem::new(31, 39))); - tree.insert(Arc::new(MockItem::new(32, 34))); - tree.insert(Arc::new(MockItem::new(33, 35))); - tree.insert(Arc::new(MockItem::new(30, 40))); - - assert_search(&tree, 30, &["30-40"]); - assert_search(&tree, 31, &["30-40", "31-39"]); - assert_search(&tree, 32, &["30-40", "32-34", "31-39"]); - assert_search(&tree, 33, &["30-40", "32-34", "33-35", "31-39"]); - assert_search(&tree, 34, &["30-40", "33-35", "31-39"]); - assert_search(&tree, 35, &["30-40", "31-39"]); - assert_search(&tree, 36, &["30-40", "31-39"]); - assert_search(&tree, 37, &["30-40", "31-39"]); - assert_search(&tree, 38, &["30-40", "31-39"]); - assert_search(&tree, 39, &["30-40"]); - assert_search(&tree, 40, &["30-40"]); - assert_search(&tree, 41, &["30-40"]); - } - - #[test] - fn test_interval_tree_duplicates() { - let mut tree: IntervalTree = IntervalTree::default(); - - // Duplicate keys - let item_a = Arc::new(MockItem::new_str(55, 56, "a")); - tree.insert(Arc::clone(&item_a)); - let item_b = Arc::new(MockItem::new_str(55, 56, "b")); - tree.insert(Arc::clone(&item_b)); - let item_c = Arc::new(MockItem::new_str(55, 56, "c")); - tree.insert(Arc::clone(&item_c)); - let item_d = Arc::new(MockItem::new_str(54, 56, "d")); - tree.insert(Arc::clone(&item_d)); - let item_e = Arc::new(MockItem::new_str(55, 57, "e")); - tree.insert(Arc::clone(&item_e)); - - dump_tree(&tree); - - assert_search( - &tree, - 55, - &["55-56: a", "55-56: b", "55-56: c", "54-56: d", "55-57: e"], - ); - tree.remove(&item_b); - dump_tree(&tree); - - assert_contents(&tree, &["54-56: d", "55-56: a", "55-56: c", "55-57: e"]); - - tree.remove(&item_d); - dump_tree(&tree); - assert_contents(&tree, &["55-56: a", "55-56: c", "55-57: e"]); - } - - #[test] - #[should_panic] - fn test_interval_tree_insert_twice() { - let mut tree: IntervalTree = IntervalTree::default(); - - // Inserting the same item twice is not cool - let item = Arc::new(MockItem::new(1, 2)); - tree.insert(Arc::clone(&item)); - tree.insert(Arc::clone(&item)); // fails assertion - } -} diff --git a/pageserver/src/layered_repository/layer_map.rs b/pageserver/src/layered_repository/layer_map.rs index fe82fd491c..c4929a6173 100644 --- a/pageserver/src/layered_repository/layer_map.rs +++ b/pageserver/src/layered_repository/layer_map.rs @@ -1,32 +1,29 @@ //! -//! The layer map tracks what layers exist for all the relishes in a timeline. +//! The layer map tracks what layers exist in a timeline. //! //! When the timeline is first accessed, the server lists of all layer files //! in the timelines/ directory, and populates this map with -//! ImageLayer and DeltaLayer structs corresponding to each file. When new WAL -//! is received, we create InMemoryLayers to hold the incoming records. Now and -//! then, in the checkpoint() function, the in-memory layers are frozen, forming -//! new image and delta layers and corresponding files are written to disk. +//! ImageLayer and DeltaLayer structs corresponding to each file. When the first +//! new WAL record is received, we create an InMemoryLayer to hold the incoming +//! records. Now and then, in the checkpoint() function, the in-memory layer is +//! are frozen, and it is split up into new image and delta layers and the +//! corresponding files are written to disk. //! -use crate::layered_repository::interval_tree::{IntervalItem, IntervalIter, IntervalTree}; -use crate::layered_repository::storage_layer::{Layer, SegmentTag}; +use crate::layered_repository::storage_layer::Layer; +use crate::layered_repository::storage_layer::{range_eq, range_overlaps}; use crate::layered_repository::InMemoryLayer; -use crate::relish::*; +use crate::repository::Key; use anyhow::Result; use lazy_static::lazy_static; -use std::cmp::Ordering; -use std::collections::{BinaryHeap, HashMap}; +use std::collections::VecDeque; +use std::ops::Range; use std::sync::Arc; +use tracing::*; use zenith_metrics::{register_int_gauge, IntGauge}; use zenith_utils::lsn::Lsn; -use super::global_layer_map::{LayerId, GLOBAL_LAYER_MAP}; - lazy_static! { - static ref NUM_INMEMORY_LAYERS: IntGauge = - register_int_gauge!("pageserver_inmemory_layers", "Number of layers in memory") - .expect("failed to define a metric"); static ref NUM_ONDISK_LAYERS: IntGauge = register_int_gauge!("pageserver_ondisk_layers", "Number of layers on-disk") .expect("failed to define a metric"); @@ -37,98 +34,147 @@ lazy_static! { /// #[derive(Default)] pub struct LayerMap { - /// All the layers keyed by segment tag - segs: HashMap, + // + // 'open_layer' holds the current InMemoryLayer that is accepting new + // records. If it is None, 'next_open_layer_at' will be set instead, indicating + // where the start LSN of the next InMemoryLayer that is to be created. + // + pub open_layer: Option>, + pub next_open_layer_at: Option, - /// All in-memory layers, ordered by 'oldest_lsn' and generation - /// of each layer. This allows easy access to the in-memory layer that - /// contains the oldest WAL record. - open_layers: BinaryHeap, + /// + /// The frozen layer, if any, contains WAL older than the current 'open_layer' + /// or 'next_open_layer_at', but newer than any historic layer. The frozen + /// layer is during checkpointing, when an InMemoryLayer is being written out + /// to disk. + /// + pub frozen_layers: VecDeque>, - /// Generation number, used to distinguish newly inserted entries in the - /// binary heap from older entries during checkpoint. - current_generation: u64, + /// All the historic layers are kept here + + /// TODO: This is a placeholder implementation of a data structure + /// to hold information about all the layer files on disk and in + /// S3. Currently, it's just a vector and all operations perform a + /// linear scan over it. That obviously becomes slow as the + /// number of layers grows. I'm imagining that an R-tree or some + /// other 2D data structure would be the long-term solution here. + historic_layers: Vec>, +} + +/// Return value of LayerMap::search +pub struct SearchResult { + pub layer: Arc, + pub lsn_floor: Lsn, } impl LayerMap { /// - /// Look up a layer using the given segment tag and LSN. This differs from a - /// plain key-value lookup in that if there is any layer that covers the - /// given LSN, or precedes the given LSN, it is returned. In other words, - /// you don't need to know the exact start LSN of the layer. + /// Find the latest layer that covers the given 'key', with lsn < + /// 'end_lsn'. /// - pub fn get(&self, tag: &SegmentTag, lsn: Lsn) -> Option> { - let segentry = self.segs.get(tag)?; - - segentry.get(lsn) - } - + /// Returns the layer, if any, and an 'lsn_floor' value that + /// indicates which portion of the layer the caller should + /// check. 'lsn_floor' is normally the start-LSN of the layer, but + /// can be greater if there is an overlapping layer that might + /// contain the version, even if it's missing from the returned + /// layer. /// - /// Get the open layer for given segment for writing. Or None if no open - /// layer exists. - /// - pub fn get_open(&self, tag: &SegmentTag) -> Option> { - let segentry = self.segs.get(tag)?; + pub fn search(&self, key: Key, end_lsn: Lsn) -> Result> { + // linear search + // Find the latest image layer that covers the given key + let mut latest_img: Option> = None; + let mut latest_img_lsn: Option = None; + for l in self.historic_layers.iter() { + if l.is_incremental() { + continue; + } + if !l.get_key_range().contains(&key) { + continue; + } + let img_lsn = l.get_lsn_range().start; - segentry - .open_layer_id - .and_then(|layer_id| GLOBAL_LAYER_MAP.read().unwrap().get(&layer_id)) - } + if img_lsn >= end_lsn { + // too new + continue; + } + if Lsn(img_lsn.0 + 1) == end_lsn { + // found exact match + return Ok(Some(SearchResult { + layer: Arc::clone(l), + lsn_floor: img_lsn, + })); + } + if img_lsn > latest_img_lsn.unwrap_or(Lsn(0)) { + latest_img = Some(Arc::clone(l)); + latest_img_lsn = Some(img_lsn); + } + } - /// - /// Insert an open in-memory layer - /// - pub fn insert_open(&mut self, layer: Arc) { - let segentry = self.segs.entry(layer.get_seg_tag()).or_default(); - - let layer_id = segentry.update_open(Arc::clone(&layer)); - - let oldest_lsn = layer.get_oldest_lsn(); - - // After a crash and restart, 'oldest_lsn' of the oldest in-memory - // layer becomes the WAL streaming starting point, so it better not point - // in the middle of a WAL record. - assert!(oldest_lsn.is_aligned()); - - // Also add it to the binary heap - let open_layer_entry = OpenLayerEntry { - oldest_lsn: layer.get_oldest_lsn(), - layer_id, - generation: self.current_generation, - }; - self.open_layers.push(open_layer_entry); - - NUM_INMEMORY_LAYERS.inc(); - } - - /// Remove an open in-memory layer - pub fn remove_open(&mut self, layer_id: LayerId) { - // Note: we don't try to remove the entry from the binary heap. - // It will be removed lazily by peek_oldest_open() when it's made it to - // the top of the heap. - - let layer_opt = { - let mut global_map = GLOBAL_LAYER_MAP.write().unwrap(); - let layer_opt = global_map.get(&layer_id); - global_map.remove(&layer_id); - // TODO it's bad that a ref can still exist after being evicted from cache - layer_opt - }; - - if let Some(layer) = layer_opt { - let mut segentry = self.segs.get_mut(&layer.get_seg_tag()).unwrap(); - - if segentry.open_layer_id == Some(layer_id) { - // Also remove it from the SegEntry of this segment - segentry.open_layer_id = None; - } else { - // We could have already updated segentry.open for - // dropped (non-writeable) layer. This is fine. - assert!(!layer.is_writeable()); - assert!(layer.is_dropped()); + // Search the delta layers + let mut latest_delta: Option> = None; + for l in self.historic_layers.iter() { + if !l.is_incremental() { + continue; + } + if !l.get_key_range().contains(&key) { + continue; } - NUM_INMEMORY_LAYERS.dec(); + if l.get_lsn_range().start >= end_lsn { + // too new + continue; + } + + if l.get_lsn_range().end >= end_lsn { + // this layer contains the requested point in the key/lsn space. + // No need to search any further + trace!( + "found layer {} for request on {} at {}", + l.filename().display(), + key, + end_lsn + ); + latest_delta.replace(Arc::clone(l)); + break; + } + // this layer's end LSN is smaller than the requested point. If there's + // nothing newer, this is what we need to return. Remember this. + if let Some(ref old_candidate) = latest_delta { + if l.get_lsn_range().end > old_candidate.get_lsn_range().end { + latest_delta.replace(Arc::clone(l)); + } + } else { + latest_delta.replace(Arc::clone(l)); + } + } + if let Some(l) = latest_delta { + trace!( + "found (old) layer {} for request on {} at {}", + l.filename().display(), + key, + end_lsn + ); + let lsn_floor = std::cmp::max( + Lsn(latest_img_lsn.unwrap_or(Lsn(0)).0 + 1), + l.get_lsn_range().start, + ); + Ok(Some(SearchResult { + lsn_floor, + layer: l, + })) + } else if let Some(l) = latest_img { + trace!( + "found img layer and no deltas for request on {} at {}", + key, + end_lsn + ); + Ok(Some(SearchResult { + lsn_floor: latest_img_lsn.unwrap(), + layer: l, + })) + } else { + trace!("no layer found for request on {} at {}", key, end_lsn); + Ok(None) } } @@ -136,9 +182,7 @@ impl LayerMap { /// Insert an on-disk layer /// pub fn insert_historic(&mut self, layer: Arc) { - let segentry = self.segs.entry(layer.get_seg_tag()).or_default(); - segentry.insert_historic(layer); - + self.historic_layers.push(layer); NUM_ONDISK_LAYERS.inc(); } @@ -147,61 +191,62 @@ impl LayerMap { /// /// This should be called when the corresponding file on disk has been deleted. /// + #[allow(dead_code)] pub fn remove_historic(&mut self, layer: Arc) { - let tag = layer.get_seg_tag(); + let len_before = self.historic_layers.len(); - if let Some(segentry) = self.segs.get_mut(&tag) { - segentry.historic.remove(&layer); - } + // FIXME: ptr_eq might fail to return true for 'dyn' + // references. Clippy complains about this. In practice it + // seems to work, the assertion below would be triggered + // otherwise but this ought to be fixed. + #[allow(clippy::vtable_address_comparisons)] + self.historic_layers + .retain(|other| !Arc::ptr_eq(other, &layer)); + + assert_eq!(self.historic_layers.len(), len_before - 1); NUM_ONDISK_LAYERS.dec(); } - // List relations along with a flag that marks if they exist at the given lsn. - // spcnode 0 and dbnode 0 have special meanings and mean all tabespaces/databases. - // Pass Tag if we're only interested in some relations. - pub fn list_relishes(&self, tag: Option, lsn: Lsn) -> Result> { - let mut rels: HashMap = HashMap::new(); - - for (seg, segentry) in self.segs.iter() { - match seg.rel { - RelishTag::Relation(reltag) => { - if let Some(request_rel) = tag { - if (request_rel.spcnode == 0 || reltag.spcnode == request_rel.spcnode) - && (request_rel.dbnode == 0 || reltag.dbnode == request_rel.dbnode) - { - if let Some(exists) = segentry.exists_at_lsn(lsn)? { - rels.insert(seg.rel, exists); - } - } - } - } - _ => { - if tag == None { - if let Some(exists) = segentry.exists_at_lsn(lsn)? { - rels.insert(seg.rel, exists); - } - } - } - } - } - Ok(rels) - } - /// Is there a newer image layer for given segment? /// /// This is used for garbage collection, to determine if an old layer can /// be deleted. /// We ignore segments newer than disk_consistent_lsn because they will be removed at restart + /// We also only look at historic layers + //#[allow(dead_code)] pub fn newer_image_layer_exists( &self, - seg: SegmentTag, + key_range: &Range, lsn: Lsn, disk_consistent_lsn: Lsn, - ) -> bool { - if let Some(segentry) = self.segs.get(&seg) { - segentry.newer_image_layer_exists(lsn, disk_consistent_lsn) - } else { - false + ) -> Result { + let mut range_remain = key_range.clone(); + + loop { + let mut made_progress = false; + for l in self.historic_layers.iter() { + if l.is_incremental() { + continue; + } + let img_lsn = l.get_lsn_range().start; + if !l.is_incremental() + && l.get_key_range().contains(&range_remain.start) + && img_lsn > lsn + && img_lsn < disk_consistent_lsn + { + made_progress = true; + let img_key_end = l.get_key_range().end; + + if img_key_end >= range_remain.end { + return Ok(true); + } + range_remain.start = img_key_end; + } + } + + if !made_progress { + return Ok(false); + } } } @@ -211,284 +256,148 @@ impl LayerMap { /// used for garbage collection, to determine if some alive layer /// exists at the lsn. If so, we shouldn't delete a newer dropped layer /// to avoid incorrectly making it visible. - pub fn layer_exists_at_lsn(&self, seg: SegmentTag, lsn: Lsn) -> Result { - Ok(if let Some(segentry) = self.segs.get(&seg) { - segentry.exists_at_lsn(lsn)?.unwrap_or(false) - } else { - false - }) + /* + pub fn layer_exists_at_lsn(&self, seg: SegmentTag, lsn: Lsn) -> Result { + Ok(if let Some(segentry) = self.historic_layers.get(&seg) { + segentry.exists_at_lsn(seg, lsn)?.unwrap_or(false) + } else { + false + }) + } + */ + + pub fn iter_historic_layers(&self) -> std::slice::Iter> { + self.historic_layers.iter() } - /// Return the oldest in-memory layer, along with its generation number. - pub fn peek_oldest_open(&mut self) -> Option<(LayerId, Arc, u64)> { - let global_map = GLOBAL_LAYER_MAP.read().unwrap(); + /// Find the last image layer that covers 'key', ignoring any image layers + /// newer than 'lsn'. + fn find_latest_image(&self, key: Key, lsn: Lsn) -> Option> { + let mut candidate_lsn = Lsn(0); + let mut candidate = None; + for l in self.historic_layers.iter() { + if l.is_incremental() { + continue; + } - while let Some(oldest_entry) = self.open_layers.peek() { - if let Some(layer) = global_map.get(&oldest_entry.layer_id) { - return Some((oldest_entry.layer_id, layer, oldest_entry.generation)); - } else { - self.open_layers.pop(); + if !l.get_key_range().contains(&key) { + continue; + } + + let this_lsn = l.get_lsn_range().start; + if this_lsn > lsn { + continue; + } + if this_lsn < candidate_lsn { + // our previous candidate was better + continue; + } + candidate_lsn = this_lsn; + candidate = Some(Arc::clone(l)); + } + + candidate + } + + /// + /// Divide the whole given range of keys into sub-ranges based on the latest + /// image layer that covers each range. (This is used when creating new + /// image layers) + /// + // FIXME: clippy complains that the result type is very complex. She's probably + // right... + #[allow(clippy::type_complexity)] + pub fn image_coverage( + &self, + key_range: &Range, + lsn: Lsn, + ) -> Result, Option>)>> { + let mut points: Vec; + + points = vec![key_range.start]; + for l in self.historic_layers.iter() { + if l.get_lsn_range().start > lsn { + continue; + } + let range = l.get_key_range(); + if key_range.contains(&range.start) { + points.push(l.get_key_range().start); + } + if key_range.contains(&range.end) { + points.push(l.get_key_range().end); } } - None - } + points.push(key_range.end); - /// Increment the generation number used to stamp open in-memory layers. Layers - /// added with `insert_open` after this call will be associated with the new - /// generation. Returns the new generation number. - pub fn increment_generation(&mut self) -> u64 { - self.current_generation += 1; - self.current_generation - } + points.sort(); + points.dedup(); - pub fn iter_historic_layers(&self) -> HistoricLayerIter { - HistoricLayerIter { - seg_iter: self.segs.iter(), - iter: None, + // Ok, we now have a list of "interesting" points in the key space + + // For each range between the points, find the latest image + let mut start = *points.first().unwrap(); + let mut ranges = Vec::new(); + for end in points[1..].iter() { + let img = self.find_latest_image(start, lsn); + + ranges.push((start..*end, img)); + + start = *end; } + Ok(ranges) + } + + /// Count how many L1 delta layers there are that overlap with the + /// given key and LSN range. + pub fn count_deltas(&self, key_range: &Range, lsn_range: &Range) -> Result { + let mut result = 0; + for l in self.historic_layers.iter() { + if !l.is_incremental() { + continue; + } + if !range_overlaps(&l.get_lsn_range(), lsn_range) { + continue; + } + if !range_overlaps(&l.get_key_range(), key_range) { + continue; + } + + // We ignore level0 delta layers. Unless the whole keyspace fits + // into one partition + if !range_eq(key_range, &(Key::MIN..Key::MAX)) + && range_eq(&l.get_key_range(), &(Key::MIN..Key::MAX)) + { + continue; + } + + result += 1; + } + Ok(result) + } + + /// Return all L0 delta layers + pub fn get_level0_deltas(&self) -> Result>> { + let mut deltas = Vec::new(); + for l in self.historic_layers.iter() { + if !l.is_incremental() { + continue; + } + if l.get_key_range() != (Key::MIN..Key::MAX) { + continue; + } + deltas.push(Arc::clone(l)); + } + Ok(deltas) } /// debugging function to print out the contents of the layer map #[allow(unused)] pub fn dump(&self) -> Result<()> { println!("Begin dump LayerMap"); - for (seg, segentry) in self.segs.iter() { - if let Some(open) = &segentry.open_layer_id { - if let Some(layer) = GLOBAL_LAYER_MAP.read().unwrap().get(open) { - layer.dump()?; - } else { - println!("layer not found in global map"); - } - } - - for layer in segentry.historic.iter() { - layer.dump()?; - } + for layer in self.historic_layers.iter() { + layer.dump()?; } println!("End dump LayerMap"); Ok(()) } } - -impl IntervalItem for dyn Layer { - type Key = Lsn; - - fn start_key(&self) -> Lsn { - self.get_start_lsn() - } - fn end_key(&self) -> Lsn { - self.get_end_lsn() - } -} - -/// -/// Per-segment entry in the LayerMap::segs hash map. Holds all the layers -/// associated with the segment. -/// -/// The last layer that is open for writes is always an InMemoryLayer, -/// and is kept in a separate field, because there can be only one for -/// each segment. The older layers, stored on disk, are kept in an -/// IntervalTree. -#[derive(Default)] -struct SegEntry { - open_layer_id: Option, - historic: IntervalTree, -} - -impl SegEntry { - /// Does the segment exist at given LSN? - /// Return None if object is not found in this SegEntry. - fn exists_at_lsn(&self, lsn: Lsn) -> Result> { - if let Some(layer) = self.get(lsn) { - Ok(Some(layer.get_seg_exists(lsn)?)) - } else { - Ok(None) - } - } - - pub fn get(&self, lsn: Lsn) -> Option> { - if let Some(open_layer_id) = &self.open_layer_id { - let open_layer = GLOBAL_LAYER_MAP.read().unwrap().get(open_layer_id)?; - if open_layer.get_start_lsn() <= lsn { - return Some(open_layer); - } - } - - self.historic.search(lsn) - } - - pub fn newer_image_layer_exists(&self, lsn: Lsn, disk_consistent_lsn: Lsn) -> bool { - // We only check on-disk layers, because - // in-memory layers are not durable - - // The end-LSN is exclusive, while disk_consistent_lsn is - // inclusive. For example, if disk_consistent_lsn is 100, it is - // OK for a delta layer to have end LSN 101, but if the end LSN - // is 102, then it might not have been fully flushed to disk - // before crash. - self.historic - .iter_newer(lsn) - .any(|layer| !layer.is_incremental() && layer.get_end_lsn() <= disk_consistent_lsn + 1) - } - - // Set new open layer for a SegEntry. - // It's ok to rewrite previous open layer, - // but only if it is not writeable anymore. - pub fn update_open(&mut self, layer: Arc) -> LayerId { - if let Some(prev_open_layer_id) = &self.open_layer_id { - if let Some(prev_open_layer) = GLOBAL_LAYER_MAP.read().unwrap().get(prev_open_layer_id) - { - assert!(!prev_open_layer.is_writeable()); - } - } - let open_layer_id = GLOBAL_LAYER_MAP.write().unwrap().insert(layer); - self.open_layer_id = Some(open_layer_id); - open_layer_id - } - - pub fn insert_historic(&mut self, layer: Arc) { - self.historic.insert(layer); - } -} - -/// Entry held in LayerMap::open_layers, with boilerplate comparison routines -/// to implement a min-heap ordered by 'oldest_lsn' and 'generation' -/// -/// The generation number associated with each entry can be used to distinguish -/// recently-added entries (i.e after last call to increment_generation()) from older -/// entries with the same 'oldest_lsn'. -struct OpenLayerEntry { - oldest_lsn: Lsn, // copy of layer.get_oldest_lsn() - generation: u64, - layer_id: LayerId, -} -impl Ord for OpenLayerEntry { - fn cmp(&self, other: &Self) -> Ordering { - // BinaryHeap is a max-heap, and we want a min-heap. Reverse the ordering here - // to get that. Entries with identical oldest_lsn are ordered by generation - other - .oldest_lsn - .cmp(&self.oldest_lsn) - .then_with(|| other.generation.cmp(&self.generation)) - } -} -impl PartialOrd for OpenLayerEntry { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} -impl PartialEq for OpenLayerEntry { - fn eq(&self, other: &Self) -> bool { - self.cmp(other) == Ordering::Equal - } -} -impl Eq for OpenLayerEntry {} - -/// Iterator returned by LayerMap::iter_historic_layers() -pub struct HistoricLayerIter<'a> { - seg_iter: std::collections::hash_map::Iter<'a, SegmentTag, SegEntry>, - iter: Option>, -} - -impl<'a> Iterator for HistoricLayerIter<'a> { - type Item = Arc; - - fn next(&mut self) -> std::option::Option<::Item> { - loop { - if let Some(x) = &mut self.iter { - if let Some(x) = x.next() { - return Some(Arc::clone(&x)); - } - } - if let Some((_tag, segentry)) = self.seg_iter.next() { - self.iter = Some(segentry.historic.iter()); - continue; - } else { - return None; - } - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::config::PageServerConf; - use std::str::FromStr; - use zenith_utils::zid::{ZTenantId, ZTimelineId}; - - /// Arbitrary relation tag, for testing. - const TESTREL_A: RelishTag = RelishTag::Relation(RelTag { - spcnode: 0, - dbnode: 111, - relnode: 1000, - forknum: 0, - }); - - lazy_static! { - static ref DUMMY_TIMELINEID: ZTimelineId = - ZTimelineId::from_str("00000000000000000000000000000000").unwrap(); - static ref DUMMY_TENANTID: ZTenantId = - ZTenantId::from_str("00000000000000000000000000000000").unwrap(); - } - - /// Construct a dummy InMemoryLayer for testing - fn dummy_inmem_layer( - conf: &'static PageServerConf, - segno: u32, - start_lsn: Lsn, - oldest_lsn: Lsn, - ) -> Arc { - Arc::new( - InMemoryLayer::create( - conf, - *DUMMY_TIMELINEID, - *DUMMY_TENANTID, - SegmentTag { - rel: TESTREL_A, - segno, - }, - start_lsn, - oldest_lsn, - ) - .unwrap(), - ) - } - - #[test] - fn test_open_layers() -> Result<()> { - let conf = PageServerConf::dummy_conf(PageServerConf::test_repo_dir("dummy_inmem_layer")); - let conf = Box::leak(Box::new(conf)); - std::fs::create_dir_all(conf.timeline_path(&DUMMY_TIMELINEID, &DUMMY_TENANTID))?; - - let mut layers = LayerMap::default(); - - let gen1 = layers.increment_generation(); - layers.insert_open(dummy_inmem_layer(conf, 0, Lsn(0x100), Lsn(0x100))); - layers.insert_open(dummy_inmem_layer(conf, 1, Lsn(0x100), Lsn(0x200))); - layers.insert_open(dummy_inmem_layer(conf, 2, Lsn(0x100), Lsn(0x120))); - layers.insert_open(dummy_inmem_layer(conf, 3, Lsn(0x100), Lsn(0x110))); - - let gen2 = layers.increment_generation(); - layers.insert_open(dummy_inmem_layer(conf, 4, Lsn(0x100), Lsn(0x110))); - layers.insert_open(dummy_inmem_layer(conf, 5, Lsn(0x100), Lsn(0x100))); - - // A helper function (closure) to pop the next oldest open entry from the layer map, - // and assert that it is what we'd expect - let mut assert_pop_layer = |expected_segno: u32, expected_generation: u64| { - let (layer_id, l, generation) = layers.peek_oldest_open().unwrap(); - assert!(l.get_seg_tag().segno == expected_segno); - assert!(generation == expected_generation); - layers.remove_open(layer_id); - }; - - assert_pop_layer(0, gen1); // 0x100 - assert_pop_layer(5, gen2); // 0x100 - assert_pop_layer(3, gen1); // 0x110 - assert_pop_layer(4, gen2); // 0x110 - assert_pop_layer(2, gen1); // 0x120 - assert_pop_layer(1, gen1); // 0x200 - - Ok(()) - } -} diff --git a/pageserver/src/layered_repository/metadata.rs b/pageserver/src/layered_repository/metadata.rs index 17e0485093..7daf899ba2 100644 --- a/pageserver/src/layered_repository/metadata.rs +++ b/pageserver/src/layered_repository/metadata.rs @@ -6,9 +6,10 @@ //! //! The module contains all structs and related helper methods related to timeline metadata. -use std::{convert::TryInto, path::PathBuf}; +use std::path::PathBuf; use anyhow::ensure; +use serde::{Deserialize, Serialize}; use zenith_utils::{ bin_ser::BeSer, lsn::Lsn, @@ -16,11 +17,13 @@ use zenith_utils::{ }; use crate::config::PageServerConf; +use crate::STORAGE_FORMAT_VERSION; -// Taken from PG_CONTROL_MAX_SAFE_SIZE -const METADATA_MAX_SAFE_SIZE: usize = 512; -const METADATA_CHECKSUM_SIZE: usize = std::mem::size_of::(); -const METADATA_MAX_DATA_SIZE: usize = METADATA_MAX_SAFE_SIZE - METADATA_CHECKSUM_SIZE; +/// We assume that a write of up to METADATA_MAX_SIZE bytes is atomic. +/// +/// This is the same assumption that PostgreSQL makes with the control file, +/// see PG_CONTROL_MAX_SAFE_SIZE +const METADATA_MAX_SIZE: usize = 512; /// The name of the metadata file pageserver creates per timeline. pub const METADATA_FILE_NAME: &str = "metadata"; @@ -30,6 +33,20 @@ pub const METADATA_FILE_NAME: &str = "metadata"; /// The fields correspond to the values we hold in memory, in LayeredTimeline. #[derive(Debug, Clone, PartialEq, Eq)] pub struct TimelineMetadata { + hdr: TimelineMetadataHeader, + body: TimelineMetadataBody, +} + +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +struct TimelineMetadataHeader { + checksum: u32, // CRC of serialized metadata body + size: u16, // size of serialized metadata + format_version: u16, // storage format version (used for compatibility checks) +} +const METADATA_HDR_SIZE: usize = std::mem::size_of::(); + +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +struct TimelineMetadataBody { disk_consistent_lsn: Lsn, // This is only set if we know it. We track it in memory when the page // server is running, but we only track the value corresponding to @@ -69,130 +86,90 @@ impl TimelineMetadata { initdb_lsn: Lsn, ) -> Self { Self { - disk_consistent_lsn, - prev_record_lsn, - ancestor_timeline, - ancestor_lsn, - latest_gc_cutoff_lsn, - initdb_lsn, + hdr: TimelineMetadataHeader { + checksum: 0, + size: 0, + format_version: STORAGE_FORMAT_VERSION, + }, + body: TimelineMetadataBody { + disk_consistent_lsn, + prev_record_lsn, + ancestor_timeline, + ancestor_lsn, + latest_gc_cutoff_lsn, + initdb_lsn, + }, } } pub fn from_bytes(metadata_bytes: &[u8]) -> anyhow::Result { ensure!( - metadata_bytes.len() == METADATA_MAX_SAFE_SIZE, + metadata_bytes.len() == METADATA_MAX_SIZE, "metadata bytes size is wrong" ); - - let data = &metadata_bytes[..METADATA_MAX_DATA_SIZE]; - let calculated_checksum = crc32c::crc32c(data); - - let checksum_bytes: &[u8; METADATA_CHECKSUM_SIZE] = - metadata_bytes[METADATA_MAX_DATA_SIZE..].try_into()?; - let expected_checksum = u32::from_le_bytes(*checksum_bytes); + let hdr = TimelineMetadataHeader::des(&metadata_bytes[0..METADATA_HDR_SIZE])?; ensure!( - calculated_checksum == expected_checksum, + hdr.format_version == STORAGE_FORMAT_VERSION, + "format version mismatch" + ); + let metadata_size = hdr.size as usize; + ensure!( + metadata_size <= METADATA_MAX_SIZE, + "corrupted metadata file" + ); + let calculated_checksum = crc32c::crc32c(&metadata_bytes[METADATA_HDR_SIZE..metadata_size]); + ensure!( + hdr.checksum == calculated_checksum, "metadata checksum mismatch" ); + let body = TimelineMetadataBody::des(&metadata_bytes[METADATA_HDR_SIZE..metadata_size])?; + ensure!( + body.disk_consistent_lsn.is_aligned(), + "disk_consistent_lsn is not aligned" + ); - let data = TimelineMetadata::from(serialize::DeTimelineMetadata::des_prefix(data)?); - ensure!(data.disk_consistent_lsn.is_aligned()); - - Ok(data) + Ok(TimelineMetadata { hdr, body }) } pub fn to_bytes(&self) -> anyhow::Result> { - let serializeable_metadata = serialize::SeTimelineMetadata::from(self); - let mut metadata_bytes = serialize::SeTimelineMetadata::ser(&serializeable_metadata)?; - ensure!(metadata_bytes.len() <= METADATA_MAX_DATA_SIZE); - metadata_bytes.resize(METADATA_MAX_SAFE_SIZE, 0u8); - - let checksum = crc32c::crc32c(&metadata_bytes[..METADATA_MAX_DATA_SIZE]); - metadata_bytes[METADATA_MAX_DATA_SIZE..].copy_from_slice(&u32::to_le_bytes(checksum)); + let body_bytes = self.body.ser()?; + let metadata_size = METADATA_HDR_SIZE + body_bytes.len(); + let hdr = TimelineMetadataHeader { + size: metadata_size as u16, + format_version: STORAGE_FORMAT_VERSION, + checksum: crc32c::crc32c(&body_bytes), + }; + let hdr_bytes = hdr.ser()?; + let mut metadata_bytes = vec![0u8; METADATA_MAX_SIZE]; + metadata_bytes[0..METADATA_HDR_SIZE].copy_from_slice(&hdr_bytes); + metadata_bytes[METADATA_HDR_SIZE..metadata_size].copy_from_slice(&body_bytes); Ok(metadata_bytes) } /// [`Lsn`] that corresponds to the corresponding timeline directory /// contents, stored locally in the pageserver workdir. pub fn disk_consistent_lsn(&self) -> Lsn { - self.disk_consistent_lsn + self.body.disk_consistent_lsn } pub fn prev_record_lsn(&self) -> Option { - self.prev_record_lsn + self.body.prev_record_lsn } pub fn ancestor_timeline(&self) -> Option { - self.ancestor_timeline + self.body.ancestor_timeline } pub fn ancestor_lsn(&self) -> Lsn { - self.ancestor_lsn + self.body.ancestor_lsn } pub fn latest_gc_cutoff_lsn(&self) -> Lsn { - self.latest_gc_cutoff_lsn + self.body.latest_gc_cutoff_lsn } pub fn initdb_lsn(&self) -> Lsn { - self.initdb_lsn - } -} - -/// This module is for direct conversion of metadata to bytes and back. -/// For a certain metadata, besides the conversion a few verification steps has to -/// be done, so all serde derives are hidden from the user, to avoid accidental -/// verification-less metadata creation. -mod serialize { - use serde::{Deserialize, Serialize}; - use zenith_utils::{lsn::Lsn, zid::ZTimelineId}; - - use super::TimelineMetadata; - - #[derive(Serialize)] - pub(super) struct SeTimelineMetadata<'a> { - disk_consistent_lsn: &'a Lsn, - prev_record_lsn: &'a Option, - ancestor_timeline: &'a Option, - ancestor_lsn: &'a Lsn, - latest_gc_cutoff_lsn: &'a Lsn, - initdb_lsn: &'a Lsn, - } - - impl<'a> From<&'a TimelineMetadata> for SeTimelineMetadata<'a> { - fn from(other: &'a TimelineMetadata) -> Self { - Self { - disk_consistent_lsn: &other.disk_consistent_lsn, - prev_record_lsn: &other.prev_record_lsn, - ancestor_timeline: &other.ancestor_timeline, - ancestor_lsn: &other.ancestor_lsn, - latest_gc_cutoff_lsn: &other.latest_gc_cutoff_lsn, - initdb_lsn: &other.initdb_lsn, - } - } - } - - #[derive(Deserialize)] - pub(super) struct DeTimelineMetadata { - disk_consistent_lsn: Lsn, - prev_record_lsn: Option, - ancestor_timeline: Option, - ancestor_lsn: Lsn, - latest_gc_cutoff_lsn: Lsn, - initdb_lsn: Lsn, - } - - impl From for TimelineMetadata { - fn from(other: DeTimelineMetadata) -> Self { - Self { - disk_consistent_lsn: other.disk_consistent_lsn, - prev_record_lsn: other.prev_record_lsn, - ancestor_timeline: other.ancestor_timeline, - ancestor_lsn: other.ancestor_lsn, - latest_gc_cutoff_lsn: other.latest_gc_cutoff_lsn, - initdb_lsn: other.initdb_lsn, - } - } + self.body.initdb_lsn } } @@ -204,14 +181,14 @@ mod tests { #[test] fn metadata_serializes_correctly() { - let original_metadata = TimelineMetadata { - disk_consistent_lsn: Lsn(0x200), - prev_record_lsn: Some(Lsn(0x100)), - ancestor_timeline: Some(TIMELINE_ID), - ancestor_lsn: Lsn(0), - latest_gc_cutoff_lsn: Lsn(0), - initdb_lsn: Lsn(0), - }; + let original_metadata = TimelineMetadata::new( + Lsn(0x200), + Some(Lsn(0x100)), + Some(TIMELINE_ID), + Lsn(0), + Lsn(0), + Lsn(0), + ); let metadata_bytes = original_metadata .to_bytes() @@ -221,7 +198,7 @@ mod tests { .expect("Should deserialize its own bytes"); assert_eq!( - deserialized_metadata, original_metadata, + deserialized_metadata.body, original_metadata.body, "Metadata that was serialized to bytes and deserialized back should not change" ); } diff --git a/pageserver/src/layered_repository/storage_layer.rs b/pageserver/src/layered_repository/storage_layer.rs index 8976491fc0..de34545980 100644 --- a/pageserver/src/layered_repository/storage_layer.rs +++ b/pageserver/src/layered_repository/storage_layer.rs @@ -2,139 +2,102 @@ //! Common traits and structs for layers //! -use crate::relish::RelishTag; -use crate::repository::{BlockNumber, ZenithWalRecord}; +use crate::repository::{Key, Value}; +use crate::walrecord::ZenithWalRecord; use crate::{ZTenantId, ZTimelineId}; use anyhow::Result; use bytes::Bytes; use serde::{Deserialize, Serialize}; -use std::fmt; +use std::ops::Range; use std::path::PathBuf; use zenith_utils::lsn::Lsn; -// Size of one segment in pages (10 MB) -pub const RELISH_SEG_SIZE: u32 = 10 * 1024 * 1024 / 8192; - -/// -/// Each relish stored in the repository is divided into fixed-sized "segments", -/// with 10 MB of key-space, or 1280 8k pages each. -/// -#[derive(Debug, PartialEq, Eq, PartialOrd, Hash, Ord, Clone, Copy, Serialize, Deserialize)] -pub struct SegmentTag { - pub rel: RelishTag, - pub segno: u32, -} - -/// SegmentBlk represents a block number within a segment, or the size of segment. -/// -/// This is separate from BlockNumber, which is used for block number within the -/// whole relish. Since this is just a type alias, the compiler will let you mix -/// them freely, but we use the type alias as documentation to make it clear -/// which one we're dealing with. -/// -/// (We could turn this into "struct SegmentBlk(u32)" to forbid accidentally -/// assigning a BlockNumber to SegmentBlk or vice versa, but that makes -/// operations more verbose). -pub type SegmentBlk = u32; - -impl fmt::Display for SegmentTag { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}.{}", self.rel, self.segno) +pub fn range_overlaps(a: &Range, b: &Range) -> bool +where + T: PartialOrd, +{ + if a.start < b.start { + a.end > b.start + } else { + b.end > a.start } } -impl SegmentTag { - /// Given a relish and block number, calculate the corresponding segment and - /// block number within the segment. - pub const fn from_blknum(rel: RelishTag, blknum: BlockNumber) -> (SegmentTag, SegmentBlk) { - ( - SegmentTag { - rel, - segno: blknum / RELISH_SEG_SIZE, - }, - blknum % RELISH_SEG_SIZE, - ) - } +pub fn range_eq(a: &Range, b: &Range) -> bool +where + T: PartialEq, +{ + a.start == b.start && a.end == b.end } +/// Struct used to communicate across calls to 'get_value_reconstruct_data'. /// -/// Represents a version of a page at a specific LSN. The LSN is the key of the -/// entry in the 'page_versions' hash, it is not duplicated here. +/// Before first call, you can fill in 'page_img' if you have an older cached +/// version of the page available. That can save work in +/// 'get_value_reconstruct_data', as it can stop searching for page versions +/// when all the WAL records going back to the cached image have been collected. /// -/// A page version can be stored as a full page image, or as WAL record that needs -/// to be applied over the previous page version to reconstruct this version. -#[derive(Debug, Clone, Serialize, Deserialize)] -pub enum PageVersion { - Page(Bytes), - Wal(ZenithWalRecord), -} - -/// -/// Struct used to communicate across calls to 'get_page_reconstruct_data'. -/// -/// Before first call to get_page_reconstruct_data, you can fill in 'page_img' -/// if you have an older cached version of the page available. That can save -/// work in 'get_page_reconstruct_data', as it can stop searching for page -/// versions when all the WAL records going back to the cached image have been -/// collected. -/// -/// When get_page_reconstruct_data returns Complete, 'page_img' is set to an -/// image of the page, or the oldest WAL record in 'records' is a will_init-type +/// When get_value_reconstruct_data returns Complete, 'img' is set to an image +/// of the page, or the oldest WAL record in 'records' is a will_init-type /// record that initializes the page without requiring a previous image. /// /// If 'get_page_reconstruct_data' returns Continue, some 'records' may have /// been collected, but there are more records outside the current layer. Pass -/// the same PageReconstructData struct in the next 'get_page_reconstruct_data' +/// the same ValueReconstructState struct in the next 'get_value_reconstruct_data' /// call, to collect more records. /// -pub struct PageReconstructData { +#[derive(Debug)] +pub struct ValueReconstructState { pub records: Vec<(Lsn, ZenithWalRecord)>, - pub page_img: Option<(Lsn, Bytes)>, + pub img: Option<(Lsn, Bytes)>, } /// Return value from Layer::get_page_reconstruct_data -pub enum PageReconstructResult { +#[derive(Clone, Copy, Debug)] +pub enum ValueReconstructResult { /// Got all the data needed to reconstruct the requested page Complete, /// This layer didn't contain all the required data, the caller should look up /// the predecessor layer at the returned LSN and collect more data from there. - Continue(Lsn), + Continue, + /// This layer didn't contain data needed to reconstruct the page version at /// the returned LSN. This is usually considered an error, but might be OK /// in some circumstances. - Missing(Lsn), + Missing, } +/// A Layer contains all data in a "rectangle" consisting of a range of keys and +/// range of LSNs. /// -/// A Layer corresponds to one RELISH_SEG_SIZE slice of a relish in a range of LSNs. /// There are two kinds of layers, in-memory and on-disk layers. In-memory -/// layers are used to ingest incoming WAL, and provide fast access -/// to the recent page versions. On-disk layers are stored as files on disk, and -/// are immutable. This trait presents the common functionality of -/// in-memory and on-disk layers. +/// layers are used to ingest incoming WAL, and provide fast access to the +/// recent page versions. On-disk layers are stored as files on disk, and are +/// immutable. This trait presents the common functionality of in-memory and +/// on-disk layers. +/// +/// Furthermore, there are two kinds of on-disk layers: delta and image layers. +/// A delta layer contains all modifications within a range of LSNs and keys. +/// An image layer is a snapshot of all the data in a key-range, at a single +/// LSN /// pub trait Layer: Send + Sync { fn get_tenant_id(&self) -> ZTenantId; - /// Identify the timeline this relish belongs to + /// Identify the timeline this layer belongs to fn get_timeline_id(&self) -> ZTimelineId; - /// Identify the relish segment - fn get_seg_tag(&self) -> SegmentTag; + /// Range of segments that this layer covers + fn get_key_range(&self) -> Range; /// Inclusive start bound of the LSN range that this layer holds - fn get_start_lsn(&self) -> Lsn; - /// Exclusive end bound of the LSN range that this layer holds. /// /// - For an open in-memory layer, this is MAX_LSN. /// - For a frozen in-memory layer or a delta layer, this is a valid end bound. /// - An image layer represents snapshot at one LSN, so end_lsn is always the snapshot LSN + 1 - fn get_end_lsn(&self) -> Lsn; - - /// Is the segment represented by this layer dropped by PostgreSQL? - fn is_dropped(&self) -> bool; + fn get_lsn_range(&self) -> Range; /// Filename used to store this layer on disk. (Even in-memory layers /// implement this, to print a handy unique identifier for the layer for @@ -153,18 +116,12 @@ pub trait Layer: Send + Sync { /// is available. If this returns PageReconstructResult::Continue, look up /// the predecessor layer and call again with the same 'reconstruct_data' to /// collect more data. - fn get_page_reconstruct_data( + fn get_value_reconstruct_data( &self, - blknum: SegmentBlk, - lsn: Lsn, - reconstruct_data: &mut PageReconstructData, - ) -> Result; - - /// Return size of the segment at given LSN. (Only for blocky relations.) - fn get_seg_size(&self, lsn: Lsn) -> Result; - - /// Does the segment exist at given LSN? Or was it dropped before it. - fn get_seg_exists(&self, lsn: Lsn) -> Result; + key: Key, + lsn_range: Range, + reconstruct_data: &mut ValueReconstructState, + ) -> Result; /// Does this layer only contain some data for the segment (incremental), /// or does it contain a version of every page? This is important to know @@ -175,6 +132,9 @@ pub trait Layer: Send + Sync { /// Returns true for layers that are represented in memory. fn is_in_memory(&self) -> bool; + /// Iterate through all keys and values stored in the layer + fn iter(&self) -> Box> + '_>; + /// Release memory used by this layer. There is no corresponding 'load' /// function, that's done implicitly when you call one of the get-functions. fn unload(&self) -> Result<()>; @@ -185,3 +145,36 @@ pub trait Layer: Send + Sync { /// Dump summary of the contents of the layer to stdout fn dump(&self) -> Result<()>; } + +// Flag indicating that this version initialize the page +const WILL_INIT: u64 = 1; + +/// +/// Struct representing reference to BLOB in layers. Reference contains BLOB offset and size. +/// For WAL records (delta layer) it also contains `will_init` flag which helps to determine range of records +/// which needs to be applied without reading/deserializing records themselves. +/// +#[derive(Debug, Serialize, Deserialize, Copy, Clone)] +pub struct BlobRef(u64); + +impl BlobRef { + pub fn will_init(&self) -> bool { + (self.0 & WILL_INIT) != 0 + } + + pub fn pos(&self) -> u64 { + self.0 >> 32 + } + + pub fn size(&self) -> usize { + ((self.0 & 0xFFFFFFFF) >> 1) as usize + } + + pub fn new(pos: u64, size: usize, will_init: bool) -> BlobRef { + let mut blob_ref = (pos << 32) | ((size as u64) << 1); + if will_init { + blob_ref |= WILL_INIT; + } + BlobRef(blob_ref) + } +} diff --git a/pageserver/src/lib.rs b/pageserver/src/lib.rs index 060fa54b23..4790ab6652 100644 --- a/pageserver/src/lib.rs +++ b/pageserver/src/lib.rs @@ -2,10 +2,12 @@ pub mod basebackup; pub mod config; pub mod http; pub mod import_datadir; +pub mod keyspace; pub mod layered_repository; pub mod page_cache; pub mod page_service; -pub mod relish; +pub mod pgdatadir_mapping; +pub mod reltag; pub mod remote_storage; pub mod repository; pub mod tenant_mgr; @@ -28,6 +30,20 @@ use zenith_utils::{ use crate::thread_mgr::ThreadKind; +use layered_repository::LayeredRepository; +use pgdatadir_mapping::DatadirTimeline; + +/// Current storage format version +/// +/// This is embedded in the metadata file, and also in the header of all the +/// layer files. If you make any backwards-incompatible changes to the storage +/// format, bump this! +pub const STORAGE_FORMAT_VERSION: u16 = 1; + +// Magic constants used to identify different kinds of files +pub const IMAGE_FILE_MAGIC: u32 = 0x5A60_0000 | STORAGE_FORMAT_VERSION as u32; +pub const DELTA_FILE_MAGIC: u32 = 0x5A61_0000 | STORAGE_FORMAT_VERSION as u32; + lazy_static! { static ref LIVE_CONNECTIONS_COUNT: IntGaugeVec = register_int_gauge_vec!( "pageserver_live_connections_count", @@ -42,14 +58,16 @@ pub const LOG_FILE_NAME: &str = "pageserver.log"; /// Config for the Repository checkpointer #[derive(Debug, Clone, Copy)] pub enum CheckpointConfig { - // Flush in-memory data that is older than this - Distance(u64), // Flush all in-memory data Flush, // Flush all in-memory data and reconstruct all page images Forced, } +pub type RepositoryImpl = LayeredRepository; + +pub type DatadirTimelineImpl = DatadirTimeline; + pub fn shutdown_pageserver() { // Shut down the libpq endpoint thread. This prevents new connections from // being accepted. diff --git a/pageserver/src/page_cache.rs b/pageserver/src/page_cache.rs index ef802ba0e2..299575f792 100644 --- a/pageserver/src/page_cache.rs +++ b/pageserver/src/page_cache.rs @@ -53,7 +53,7 @@ use zenith_utils::{ }; use crate::layered_repository::writeback_ephemeral_file; -use crate::relish::RelTag; +use crate::repository::Key; static PAGE_CACHE: OnceCell = OnceCell::new(); const TEST_PAGE_CACHE_SIZE: usize = 10; @@ -105,8 +105,7 @@ enum CacheKey { struct MaterializedPageHashKey { tenant_id: ZTenantId, timeline_id: ZTimelineId, - rel_tag: RelTag, - blknum: u32, + key: Key, } #[derive(Clone)] @@ -291,16 +290,14 @@ impl PageCache { &self, tenant_id: ZTenantId, timeline_id: ZTimelineId, - rel_tag: RelTag, - blknum: u32, + key: &Key, lsn: Lsn, ) -> Option<(Lsn, PageReadGuard)> { let mut cache_key = CacheKey::MaterializedPage { hash_key: MaterializedPageHashKey { tenant_id, timeline_id, - rel_tag, - blknum, + key: *key, }, lsn, }; @@ -323,8 +320,7 @@ impl PageCache { &self, tenant_id: ZTenantId, timeline_id: ZTimelineId, - rel_tag: RelTag, - blknum: u32, + key: Key, lsn: Lsn, img: &[u8], ) { @@ -332,8 +328,7 @@ impl PageCache { hash_key: MaterializedPageHashKey { tenant_id, timeline_id, - rel_tag, - blknum, + key, }, lsn, }; diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index 4744f0fe52..43e1ec275d 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -32,7 +32,9 @@ use zenith_utils::zid::{ZTenantId, ZTimelineId}; use crate::basebackup; use crate::config::PageServerConf; -use crate::relish::*; +use crate::pgdatadir_mapping::DatadirTimeline; +use crate::reltag::RelTag; +use crate::repository::Repository; use crate::repository::Timeline; use crate::tenant_mgr; use crate::thread_mgr; @@ -398,8 +400,8 @@ impl PageServerHandler { /// In either case, if the page server hasn't received the WAL up to the /// requested LSN yet, we will wait for it to arrive. The return value is /// the LSN that should be used to look up the page versions. - fn wait_or_get_last_lsn( - timeline: &dyn Timeline, + fn wait_or_get_last_lsn( + timeline: &DatadirTimeline, mut lsn: Lsn, latest: bool, latest_gc_cutoff_lsn: &RwLockReadGuard, @@ -426,7 +428,7 @@ impl PageServerHandler { if lsn <= last_record_lsn { lsn = last_record_lsn; } else { - timeline.wait_lsn(lsn)?; + timeline.tline.wait_lsn(lsn)?; // Since we waited for 'lsn' to arrive, that is now the last // record LSN. (Or close enough for our purposes; the // last-record LSN can advance immediately after we return @@ -436,7 +438,7 @@ impl PageServerHandler { if lsn == Lsn(0) { bail!("invalid LSN(0) in request"); } - timeline.wait_lsn(lsn)?; + timeline.tline.wait_lsn(lsn)?; } ensure!( lsn >= **latest_gc_cutoff_lsn, @@ -446,54 +448,47 @@ impl PageServerHandler { Ok(lsn) } - fn handle_get_rel_exists_request( + fn handle_get_rel_exists_request( &self, - timeline: &dyn Timeline, + timeline: &DatadirTimeline, req: &PagestreamExistsRequest, ) -> Result { let _enter = info_span!("get_rel_exists", rel = %req.rel, req_lsn = %req.lsn).entered(); - let tag = RelishTag::Relation(req.rel); - let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); + let latest_gc_cutoff_lsn = timeline.tline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)?; - let exists = timeline.get_rel_exists(tag, lsn)?; + let exists = timeline.get_rel_exists(req.rel, lsn)?; Ok(PagestreamBeMessage::Exists(PagestreamExistsResponse { exists, })) } - fn handle_get_nblocks_request( + fn handle_get_nblocks_request( &self, - timeline: &dyn Timeline, + timeline: &DatadirTimeline, req: &PagestreamNblocksRequest, ) -> Result { let _enter = info_span!("get_nblocks", rel = %req.rel, req_lsn = %req.lsn).entered(); - let tag = RelishTag::Relation(req.rel); - let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); + let latest_gc_cutoff_lsn = timeline.tline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)?; - let n_blocks = timeline.get_relish_size(tag, lsn)?; - - // Return 0 if relation is not found. - // This is what postgres smgr expects. - let n_blocks = n_blocks.unwrap_or(0); + let n_blocks = timeline.get_rel_size(req.rel, lsn)?; Ok(PagestreamBeMessage::Nblocks(PagestreamNblocksResponse { n_blocks, })) } - fn handle_get_page_at_lsn_request( + fn handle_get_page_at_lsn_request( &self, - timeline: &dyn Timeline, + timeline: &DatadirTimeline, req: &PagestreamGetPageRequest, ) -> Result { let _enter = info_span!("get_page", rel = %req.rel, blkno = &req.blkno, req_lsn = %req.lsn) .entered(); - let tag = RelishTag::Relation(req.rel); - let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); + let latest_gc_cutoff_lsn = timeline.tline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)?; /* // Add a 1s delay to some requests. The delayed causes the requests to @@ -503,7 +498,7 @@ impl PageServerHandler { std::thread::sleep(std::time::Duration::from_millis(1000)); } */ - let page = timeline.get_page_at_lsn(tag, req.blkno, lsn)?; + let page = timeline.get_rel_page_at_lsn(req.rel, req.blkno, lsn)?; Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse { page, @@ -523,7 +518,7 @@ impl PageServerHandler { // check that the timeline exists let timeline = tenant_mgr::get_timeline_for_tenant_load(tenantid, timelineid) .context("Cannot load local timeline")?; - let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); + let latest_gc_cutoff_lsn = timeline.tline.get_latest_gc_cutoff_lsn(); if let Some(lsn) = lsn { timeline .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn) @@ -701,67 +696,19 @@ impl postgres_backend::Handler for PageServerHandler { let repo = tenant_mgr::get_repository_for_tenant(tenantid)?; let result = repo.gc_iteration(Some(timelineid), gc_horizon, true)?; pgb.write_message_noflush(&BeMessage::RowDescription(&[ - RowDescriptor::int8_col(b"layer_relfiles_total"), - RowDescriptor::int8_col(b"layer_relfiles_needed_by_cutoff"), - RowDescriptor::int8_col(b"layer_relfiles_needed_by_branches"), - RowDescriptor::int8_col(b"layer_relfiles_not_updated"), - RowDescriptor::int8_col(b"layer_relfiles_needed_as_tombstone"), - RowDescriptor::int8_col(b"layer_relfiles_removed"), - RowDescriptor::int8_col(b"layer_relfiles_dropped"), - RowDescriptor::int8_col(b"layer_nonrelfiles_total"), - RowDescriptor::int8_col(b"layer_nonrelfiles_needed_by_cutoff"), - RowDescriptor::int8_col(b"layer_nonrelfiles_needed_by_branches"), - RowDescriptor::int8_col(b"layer_nonrelfiles_not_updated"), - RowDescriptor::int8_col(b"layer_nonrelfiles_needed_as_tombstone"), - RowDescriptor::int8_col(b"layer_nonrelfiles_removed"), - RowDescriptor::int8_col(b"layer_nonrelfiles_dropped"), + RowDescriptor::int8_col(b"layers_total"), + RowDescriptor::int8_col(b"layers_needed_by_cutoff"), + RowDescriptor::int8_col(b"layers_needed_by_branches"), + RowDescriptor::int8_col(b"layers_not_updated"), + RowDescriptor::int8_col(b"layers_removed"), RowDescriptor::int8_col(b"elapsed"), ]))? .write_message_noflush(&BeMessage::DataRow(&[ - Some(result.ondisk_relfiles_total.to_string().as_bytes()), - Some( - result - .ondisk_relfiles_needed_by_cutoff - .to_string() - .as_bytes(), - ), - Some( - result - .ondisk_relfiles_needed_by_branches - .to_string() - .as_bytes(), - ), - Some(result.ondisk_relfiles_not_updated.to_string().as_bytes()), - Some( - result - .ondisk_relfiles_needed_as_tombstone - .to_string() - .as_bytes(), - ), - Some(result.ondisk_relfiles_removed.to_string().as_bytes()), - Some(result.ondisk_relfiles_dropped.to_string().as_bytes()), - Some(result.ondisk_nonrelfiles_total.to_string().as_bytes()), - Some( - result - .ondisk_nonrelfiles_needed_by_cutoff - .to_string() - .as_bytes(), - ), - Some( - result - .ondisk_nonrelfiles_needed_by_branches - .to_string() - .as_bytes(), - ), - Some(result.ondisk_nonrelfiles_not_updated.to_string().as_bytes()), - Some( - result - .ondisk_nonrelfiles_needed_as_tombstone - .to_string() - .as_bytes(), - ), - Some(result.ondisk_nonrelfiles_removed.to_string().as_bytes()), - Some(result.ondisk_nonrelfiles_dropped.to_string().as_bytes()), + Some(result.layers_total.to_string().as_bytes()), + Some(result.layers_needed_by_cutoff.to_string().as_bytes()), + Some(result.layers_needed_by_branches.to_string().as_bytes()), + Some(result.layers_not_updated.to_string().as_bytes()), + Some(result.layers_removed.to_string().as_bytes()), Some(result.elapsed.as_millis().to_string().as_bytes()), ]))? .write_message(&BeMessage::CommandComplete(b"SELECT 1"))?; @@ -781,7 +728,14 @@ impl postgres_backend::Handler for PageServerHandler { let timeline = tenant_mgr::get_timeline_for_tenant_load(tenantid, timelineid) .context("Cannot load local timeline")?; - timeline.checkpoint(CheckpointConfig::Forced)?; + timeline.tline.checkpoint(CheckpointConfig::Forced)?; + + // Also compact it. + // + // FIXME: This probably shouldn't be part of a "checkpoint" command, but a + // separate operation. Update the tests if you change this. + timeline.tline.compact()?; + pgb.write_message_noflush(&SINGLE_COL_ROWDESC)? .write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?; } else { diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs new file mode 100644 index 0000000000..7b0fc606de --- /dev/null +++ b/pageserver/src/pgdatadir_mapping.rs @@ -0,0 +1,1350 @@ +//! +//! This provides an abstraction to store PostgreSQL relations and other files +//! in the key-value store that implements the Repository interface. +//! +//! (TODO: The line between PUT-functions here and walingest.rs is a bit blurry, as +//! walingest.rs handles a few things like implicit relation creation and extension. +//! Clarify that) +//! +use crate::keyspace::{KeySpace, KeySpaceAccum, TARGET_FILE_SIZE_BYTES}; +use crate::reltag::{RelTag, SlruKind}; +use crate::repository::*; +use crate::repository::{Repository, Timeline}; +use crate::walrecord::ZenithWalRecord; +use anyhow::{bail, ensure, Result}; +use bytes::{Buf, Bytes}; +use postgres_ffi::{pg_constants, Oid, TransactionId}; +use serde::{Deserialize, Serialize}; +use std::collections::{HashMap, HashSet}; +use std::ops::Range; +use std::sync::atomic::{AtomicIsize, Ordering}; +use std::sync::{Arc, RwLockReadGuard}; +use tracing::{debug, error, trace, warn}; +use zenith_utils::bin_ser::BeSer; +use zenith_utils::lsn::AtomicLsn; +use zenith_utils::lsn::Lsn; + +/// Block number within a relation or SLRU. This matches PostgreSQL's BlockNumber type. +pub type BlockNumber = u32; + +pub struct DatadirTimeline +where + R: Repository, +{ + /// The underlying key-value store. Callers should not read or modify the + /// data in the underlying store directly. However, it is exposed to have + /// access to information like last-LSN, ancestor, and operations like + /// compaction. + pub tline: Arc, + + /// When did we last calculate the partitioning? + last_partitioning: AtomicLsn, + + /// Configuration: how often should the partitioning be recalculated. + repartition_threshold: u64, + + /// Current logical size of the "datadir", at the last LSN. + current_logical_size: AtomicIsize, +} + +impl DatadirTimeline { + pub fn new(tline: Arc, repartition_threshold: u64) -> Self { + DatadirTimeline { + tline, + last_partitioning: AtomicLsn::new(0), + current_logical_size: AtomicIsize::new(0), + repartition_threshold, + } + } + + /// (Re-)calculate the logical size of the database at the latest LSN. + /// + /// This can be a slow operation. + pub fn init_logical_size(&self) -> Result<()> { + let last_lsn = self.tline.get_last_record_lsn(); + self.current_logical_size.store( + self.get_current_logical_size_non_incremental(last_lsn)? as isize, + Ordering::SeqCst, + ); + Ok(()) + } + + /// Start ingesting a WAL record, or other atomic modification of + /// the timeline. + /// + /// This provides a transaction-like interface to perform a bunch + /// of modifications atomically, all stamped with one LSN. + /// + /// To ingest a WAL record, call begin_modification(lsn) to get a + /// DatadirModification object. Use the functions in the object to + /// modify the repository state, updating all the pages and metadata + /// that the WAL record affects. When you're done, call commit() to + /// commit the changes. + /// + /// Note that any pending modifications you make through the + /// modification object won't be visible to calls to the 'get' and list + /// functions of the timeline until you finish! And if you update the + /// same page twice, the last update wins. + /// + pub fn begin_modification(&self, lsn: Lsn) -> DatadirModification { + DatadirModification { + tline: self, + lsn, + pending_updates: HashMap::new(), + pending_deletions: Vec::new(), + pending_nblocks: 0, + } + } + + //------------------------------------------------------------------------------ + // Public GET functions + //------------------------------------------------------------------------------ + + /// Look up given page version. + pub fn get_rel_page_at_lsn(&self, tag: RelTag, blknum: BlockNumber, lsn: Lsn) -> Result { + ensure!(tag.relnode != 0, "invalid relnode"); + + let nblocks = self.get_rel_size(tag, lsn)?; + if blknum >= nblocks { + debug!( + "read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page", + tag, blknum, lsn, nblocks + ); + return Ok(ZERO_PAGE.clone()); + } + + let key = rel_block_to_key(tag, blknum); + self.tline.get(key, lsn) + } + + /// Get size of a relation file + pub fn get_rel_size(&self, tag: RelTag, lsn: Lsn) -> Result { + ensure!(tag.relnode != 0, "invalid relnode"); + + if (tag.forknum == pg_constants::FSM_FORKNUM + || tag.forknum == pg_constants::VISIBILITYMAP_FORKNUM) + && !self.get_rel_exists(tag, lsn)? + { + // FIXME: Postgres sometimes calls smgrcreate() to create + // FSM, and smgrnblocks() on it immediately afterwards, + // without extending it. Tolerate that by claiming that + // any non-existent FSM fork has size 0. + return Ok(0); + } + + let key = rel_size_to_key(tag); + let mut buf = self.tline.get(key, lsn)?; + Ok(buf.get_u32_le()) + } + + /// Does relation exist? + pub fn get_rel_exists(&self, tag: RelTag, lsn: Lsn) -> Result { + ensure!(tag.relnode != 0, "invalid relnode"); + + // fetch directory listing + let key = rel_dir_to_key(tag.spcnode, tag.dbnode); + let buf = self.tline.get(key, lsn)?; + let dir = RelDirectory::des(&buf)?; + + let exists = dir.rels.get(&(tag.relnode, tag.forknum)).is_some(); + + Ok(exists) + } + + /// Get a list of all existing relations in given tablespace and database. + pub fn list_rels(&self, spcnode: Oid, dbnode: Oid, lsn: Lsn) -> Result> { + // fetch directory listing + let key = rel_dir_to_key(spcnode, dbnode); + let buf = self.tline.get(key, lsn)?; + let dir = RelDirectory::des(&buf)?; + + let rels: HashSet = + HashSet::from_iter(dir.rels.iter().map(|(relnode, forknum)| RelTag { + spcnode, + dbnode, + relnode: *relnode, + forknum: *forknum, + })); + + Ok(rels) + } + + /// Look up given SLRU page version. + pub fn get_slru_page_at_lsn( + &self, + kind: SlruKind, + segno: u32, + blknum: BlockNumber, + lsn: Lsn, + ) -> Result { + let key = slru_block_to_key(kind, segno, blknum); + self.tline.get(key, lsn) + } + + /// Get size of an SLRU segment + pub fn get_slru_segment_size( + &self, + kind: SlruKind, + segno: u32, + lsn: Lsn, + ) -> Result { + let key = slru_segment_size_to_key(kind, segno); + let mut buf = self.tline.get(key, lsn)?; + Ok(buf.get_u32_le()) + } + + /// Get size of an SLRU segment + pub fn get_slru_segment_exists(&self, kind: SlruKind, segno: u32, lsn: Lsn) -> Result { + // fetch directory listing + let key = slru_dir_to_key(kind); + let buf = self.tline.get(key, lsn)?; + let dir = SlruSegmentDirectory::des(&buf)?; + + let exists = dir.segments.get(&segno).is_some(); + Ok(exists) + } + + /// Get a list of SLRU segments + pub fn list_slru_segments(&self, kind: SlruKind, lsn: Lsn) -> Result> { + // fetch directory entry + let key = slru_dir_to_key(kind); + + let buf = self.tline.get(key, lsn)?; + let dir = SlruSegmentDirectory::des(&buf)?; + + Ok(dir.segments) + } + + pub fn get_relmap_file(&self, spcnode: Oid, dbnode: Oid, lsn: Lsn) -> Result { + let key = relmap_file_key(spcnode, dbnode); + + let buf = self.tline.get(key, lsn)?; + Ok(buf) + } + + pub fn list_dbdirs(&self, lsn: Lsn) -> Result> { + // fetch directory entry + let buf = self.tline.get(DBDIR_KEY, lsn)?; + let dir = DbDirectory::des(&buf)?; + + Ok(dir.dbdirs) + } + + pub fn get_twophase_file(&self, xid: TransactionId, lsn: Lsn) -> Result { + let key = twophase_file_key(xid); + let buf = self.tline.get(key, lsn)?; + Ok(buf) + } + + pub fn list_twophase_files(&self, lsn: Lsn) -> Result> { + // fetch directory entry + let buf = self.tline.get(TWOPHASEDIR_KEY, lsn)?; + let dir = TwoPhaseDirectory::des(&buf)?; + + Ok(dir.xids) + } + + pub fn get_control_file(&self, lsn: Lsn) -> Result { + self.tline.get(CONTROLFILE_KEY, lsn) + } + + pub fn get_checkpoint(&self, lsn: Lsn) -> Result { + self.tline.get(CHECKPOINT_KEY, lsn) + } + + /// Get the LSN of the last ingested WAL record. + /// + /// This is just a convenience wrapper that calls through to the underlying + /// repository. + pub fn get_last_record_lsn(&self) -> Lsn { + self.tline.get_last_record_lsn() + } + + /// Check that it is valid to request operations with that lsn. + /// + /// This is just a convenience wrapper that calls through to the underlying + /// repository. + pub fn check_lsn_is_in_scope( + &self, + lsn: Lsn, + latest_gc_cutoff_lsn: &RwLockReadGuard, + ) -> Result<()> { + self.tline.check_lsn_is_in_scope(lsn, latest_gc_cutoff_lsn) + } + + /// Retrieve current logical size of the timeline + /// + /// NOTE: counted incrementally, includes ancestors, + pub fn get_current_logical_size(&self) -> usize { + let current_logical_size = self.current_logical_size.load(Ordering::Acquire); + match usize::try_from(current_logical_size) { + Ok(sz) => sz, + Err(_) => { + error!( + "current_logical_size is out of range: {}", + current_logical_size + ); + 0 + } + } + } + + /// Does the same as get_current_logical_size but counted on demand. + /// Used to initialize the logical size tracking on startup. + /// + /// Only relation blocks are counted currently. That excludes metadata, + /// SLRUs, twophase files etc. + pub fn get_current_logical_size_non_incremental(&self, lsn: Lsn) -> Result { + // Fetch list of database dirs and iterate them + let buf = self.tline.get(DBDIR_KEY, lsn)?; + let dbdir = DbDirectory::des(&buf)?; + + let mut total_size: usize = 0; + for (spcnode, dbnode) in dbdir.dbdirs.keys() { + for rel in self.list_rels(*spcnode, *dbnode, lsn)? { + let relsize_key = rel_size_to_key(rel); + let mut buf = self.tline.get(relsize_key, lsn)?; + let relsize = buf.get_u32_le(); + + total_size += relsize as usize; + } + } + Ok(total_size * pg_constants::BLCKSZ as usize) + } + + /// + /// Get a KeySpace that covers all the Keys that are in use at the given LSN. + /// Anything that's not listed maybe removed from the underlying storage (from + /// that LSN forwards). + fn collect_keyspace(&self, lsn: Lsn) -> Result { + // Iterate through key ranges, greedily packing them into partitions + let mut result = KeySpaceAccum::new(); + + // The dbdir metadata always exists + result.add_key(DBDIR_KEY); + + // Fetch list of database dirs and iterate them + let buf = self.tline.get(DBDIR_KEY, lsn)?; + let dbdir = DbDirectory::des(&buf)?; + + let mut dbs: Vec<(Oid, Oid)> = dbdir.dbdirs.keys().cloned().collect(); + dbs.sort_unstable(); + for (spcnode, dbnode) in dbs { + result.add_key(relmap_file_key(spcnode, dbnode)); + result.add_key(rel_dir_to_key(spcnode, dbnode)); + + let mut rels: Vec = self + .list_rels(spcnode, dbnode, lsn)? + .iter() + .cloned() + .collect(); + rels.sort_unstable(); + for rel in rels { + let relsize_key = rel_size_to_key(rel); + let mut buf = self.tline.get(relsize_key, lsn)?; + let relsize = buf.get_u32_le(); + + result.add_range(rel_block_to_key(rel, 0)..rel_block_to_key(rel, relsize)); + result.add_key(relsize_key); + } + } + + // Iterate SLRUs next + for kind in [ + SlruKind::Clog, + SlruKind::MultiXactMembers, + SlruKind::MultiXactOffsets, + ] { + let slrudir_key = slru_dir_to_key(kind); + result.add_key(slrudir_key); + let buf = self.tline.get(slrudir_key, lsn)?; + let dir = SlruSegmentDirectory::des(&buf)?; + let mut segments: Vec = dir.segments.iter().cloned().collect(); + segments.sort_unstable(); + for segno in segments { + let segsize_key = slru_segment_size_to_key(kind, segno); + let mut buf = self.tline.get(segsize_key, lsn)?; + let segsize = buf.get_u32_le(); + + result.add_range( + slru_block_to_key(kind, segno, 0)..slru_block_to_key(kind, segno, segsize), + ); + result.add_key(segsize_key); + } + } + + // Then pg_twophase + result.add_key(TWOPHASEDIR_KEY); + let buf = self.tline.get(TWOPHASEDIR_KEY, lsn)?; + let twophase_dir = TwoPhaseDirectory::des(&buf)?; + let mut xids: Vec = twophase_dir.xids.iter().cloned().collect(); + xids.sort_unstable(); + for xid in xids { + result.add_key(twophase_file_key(xid)); + } + + result.add_key(CONTROLFILE_KEY); + result.add_key(CHECKPOINT_KEY); + + Ok(result.to_keyspace()) + } +} + +/// DatadirModification represents an operation to ingest an atomic set of +/// updates to the repository. It is created by the 'begin_record' +/// function. It is called for each WAL record, so that all the modifications +/// by a one WAL record appear atomic. +pub struct DatadirModification<'a, R: Repository> { + /// The timeline this modification applies to. You can access this to + /// read the state, but note that any pending updates are *not* reflected + /// in the state in 'tline' yet. + pub tline: &'a DatadirTimeline, + + lsn: Lsn, + + // The modifications are not applied directly to the underyling key-value store. + // The put-functions add the modifications here, and they are flushed to the + // underlying key-value store by the 'finish' function. + pending_updates: HashMap, + pending_deletions: Vec>, + pending_nblocks: isize, +} + +impl<'a, R: Repository> DatadirModification<'a, R> { + /// Initialize a completely new repository. + /// + /// This inserts the directory metadata entries that are assumed to + /// always exist. + pub fn init_empty(&mut self) -> Result<()> { + let buf = DbDirectory::ser(&DbDirectory { + dbdirs: HashMap::new(), + })?; + self.put(DBDIR_KEY, Value::Image(buf.into())); + + let buf = TwoPhaseDirectory::ser(&TwoPhaseDirectory { + xids: HashSet::new(), + })?; + self.put(TWOPHASEDIR_KEY, Value::Image(buf.into())); + + let buf: Bytes = SlruSegmentDirectory::ser(&SlruSegmentDirectory::default())?.into(); + let empty_dir = Value::Image(buf); + self.put(slru_dir_to_key(SlruKind::Clog), empty_dir.clone()); + self.put( + slru_dir_to_key(SlruKind::MultiXactMembers), + empty_dir.clone(), + ); + self.put(slru_dir_to_key(SlruKind::MultiXactOffsets), empty_dir); + + Ok(()) + } + + /// Put a new page version that can be constructed from a WAL record + /// + /// NOTE: this will *not* implicitly extend the relation, if the page is beyond the + /// current end-of-file. It's up to the caller to check that the relation size + /// matches the blocks inserted! + pub fn put_rel_wal_record( + &mut self, + rel: RelTag, + blknum: BlockNumber, + rec: ZenithWalRecord, + ) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + self.put(rel_block_to_key(rel, blknum), Value::WalRecord(rec)); + Ok(()) + } + + // Same, but for an SLRU. + pub fn put_slru_wal_record( + &mut self, + kind: SlruKind, + segno: u32, + blknum: BlockNumber, + rec: ZenithWalRecord, + ) -> Result<()> { + self.put( + slru_block_to_key(kind, segno, blknum), + Value::WalRecord(rec), + ); + Ok(()) + } + + /// Like put_wal_record, but with ready-made image of the page. + pub fn put_rel_page_image( + &mut self, + rel: RelTag, + blknum: BlockNumber, + img: Bytes, + ) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + self.put(rel_block_to_key(rel, blknum), Value::Image(img)); + Ok(()) + } + + pub fn put_slru_page_image( + &mut self, + kind: SlruKind, + segno: u32, + blknum: BlockNumber, + img: Bytes, + ) -> Result<()> { + self.put(slru_block_to_key(kind, segno, blknum), Value::Image(img)); + Ok(()) + } + + /// Store a relmapper file (pg_filenode.map) in the repository + pub fn put_relmap_file(&mut self, spcnode: Oid, dbnode: Oid, img: Bytes) -> Result<()> { + // Add it to the directory (if it doesn't exist already) + let buf = self.get(DBDIR_KEY)?; + let mut dbdir = DbDirectory::des(&buf)?; + + let r = dbdir.dbdirs.insert((spcnode, dbnode), true); + if r == None || r == Some(false) { + // The dbdir entry didn't exist, or it contained a + // 'false'. The 'insert' call already updated it with + // 'true', now write the updated 'dbdirs' map back. + let buf = DbDirectory::ser(&dbdir)?; + self.put(DBDIR_KEY, Value::Image(buf.into())); + } + if r == None { + // Create RelDirectory + let buf = RelDirectory::ser(&RelDirectory { + rels: HashSet::new(), + })?; + self.put( + rel_dir_to_key(spcnode, dbnode), + Value::Image(Bytes::from(buf)), + ); + } + + self.put(relmap_file_key(spcnode, dbnode), Value::Image(img)); + Ok(()) + } + + pub fn put_twophase_file(&mut self, xid: TransactionId, img: Bytes) -> Result<()> { + // Add it to the directory entry + let buf = self.get(TWOPHASEDIR_KEY)?; + let mut dir = TwoPhaseDirectory::des(&buf)?; + if !dir.xids.insert(xid) { + bail!("twophase file for xid {} already exists", xid); + } + self.put( + TWOPHASEDIR_KEY, + Value::Image(Bytes::from(TwoPhaseDirectory::ser(&dir)?)), + ); + + self.put(twophase_file_key(xid), Value::Image(img)); + Ok(()) + } + + pub fn put_control_file(&mut self, img: Bytes) -> Result<()> { + self.put(CONTROLFILE_KEY, Value::Image(img)); + Ok(()) + } + + pub fn put_checkpoint(&mut self, img: Bytes) -> Result<()> { + self.put(CHECKPOINT_KEY, Value::Image(img)); + Ok(()) + } + + pub fn drop_dbdir(&mut self, spcnode: Oid, dbnode: Oid) -> Result<()> { + // Remove entry from dbdir + let buf = self.get(DBDIR_KEY)?; + let mut dir = DbDirectory::des(&buf)?; + if dir.dbdirs.remove(&(spcnode, dbnode)).is_some() { + let buf = DbDirectory::ser(&dir)?; + self.put(DBDIR_KEY, Value::Image(buf.into())); + } else { + warn!( + "dropped dbdir for spcnode {} dbnode {} did not exist in db directory", + spcnode, dbnode + ); + } + + // FIXME: update pending_nblocks + + // Delete all relations and metadata files for the spcnode/dnode + self.delete(dbdir_key_range(spcnode, dbnode)); + Ok(()) + } + + /// Create a relation fork. + /// + /// 'nblocks' is the initial size. + pub fn put_rel_creation(&mut self, rel: RelTag, nblocks: BlockNumber) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + // It's possible that this is the first rel for this db in this + // tablespace. Create the reldir entry for it if so. + let mut dbdir = DbDirectory::des(&self.get(DBDIR_KEY)?)?; + let rel_dir_key = rel_dir_to_key(rel.spcnode, rel.dbnode); + let mut rel_dir = if dbdir.dbdirs.get(&(rel.spcnode, rel.dbnode)).is_none() { + // Didn't exist. Update dbdir + dbdir.dbdirs.insert((rel.spcnode, rel.dbnode), false); + let buf = DbDirectory::ser(&dbdir)?; + self.put(DBDIR_KEY, Value::Image(buf.into())); + + // and create the RelDirectory + RelDirectory::default() + } else { + // reldir already exists, fetch it + RelDirectory::des(&self.get(rel_dir_key)?)? + }; + + // Add the new relation to the rel directory entry, and write it back + if !rel_dir.rels.insert((rel.relnode, rel.forknum)) { + bail!("rel {} already exists", rel); + } + self.put( + rel_dir_key, + Value::Image(Bytes::from(RelDirectory::ser(&rel_dir)?)), + ); + + // Put size + let size_key = rel_size_to_key(rel); + let buf = nblocks.to_le_bytes(); + self.put(size_key, Value::Image(Bytes::from(buf.to_vec()))); + + self.pending_nblocks += nblocks as isize; + + // Even if nblocks > 0, we don't insert any actual blocks here. That's up to the + // caller. + + Ok(()) + } + + /// Truncate relation + pub fn put_rel_truncation(&mut self, rel: RelTag, nblocks: BlockNumber) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + let size_key = rel_size_to_key(rel); + + // Fetch the old size first + let old_size = self.get(size_key)?.get_u32_le(); + + // Update the entry with the new size. + let buf = nblocks.to_le_bytes(); + self.put(size_key, Value::Image(Bytes::from(buf.to_vec()))); + + // Update logical database size. + self.pending_nblocks -= old_size as isize - nblocks as isize; + Ok(()) + } + + /// Extend relation + pub fn put_rel_extend(&mut self, rel: RelTag, nblocks: BlockNumber) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + + // Put size + let size_key = rel_size_to_key(rel); + let old_size = self.get(size_key)?.get_u32_le(); + + let buf = nblocks.to_le_bytes(); + self.put(size_key, Value::Image(Bytes::from(buf.to_vec()))); + + self.pending_nblocks += nblocks as isize - old_size as isize; + Ok(()) + } + + /// Drop a relation. + pub fn put_rel_drop(&mut self, rel: RelTag) -> Result<()> { + ensure!(rel.relnode != 0, "invalid relnode"); + + // Remove it from the directory entry + let dir_key = rel_dir_to_key(rel.spcnode, rel.dbnode); + let buf = self.get(dir_key)?; + let mut dir = RelDirectory::des(&buf)?; + + if dir.rels.remove(&(rel.relnode, rel.forknum)) { + self.put(dir_key, Value::Image(Bytes::from(RelDirectory::ser(&dir)?))); + } else { + warn!("dropped rel {} did not exist in rel directory", rel); + } + + // update logical size + let size_key = rel_size_to_key(rel); + let old_size = self.get(size_key)?.get_u32_le(); + self.pending_nblocks -= old_size as isize; + + // Delete size entry, as well as all blocks + self.delete(rel_key_range(rel)); + + Ok(()) + } + + pub fn put_slru_segment_creation( + &mut self, + kind: SlruKind, + segno: u32, + nblocks: BlockNumber, + ) -> Result<()> { + // Add it to the directory entry + let dir_key = slru_dir_to_key(kind); + let buf = self.get(dir_key)?; + let mut dir = SlruSegmentDirectory::des(&buf)?; + + if !dir.segments.insert(segno) { + bail!("slru segment {:?}/{} already exists", kind, segno); + } + self.put( + dir_key, + Value::Image(Bytes::from(SlruSegmentDirectory::ser(&dir)?)), + ); + + // Put size + let size_key = slru_segment_size_to_key(kind, segno); + let buf = nblocks.to_le_bytes(); + self.put(size_key, Value::Image(Bytes::from(buf.to_vec()))); + + // even if nblocks > 0, we don't insert any actual blocks here + + Ok(()) + } + + /// Extend SLRU segment + pub fn put_slru_extend( + &mut self, + kind: SlruKind, + segno: u32, + nblocks: BlockNumber, + ) -> Result<()> { + // Put size + let size_key = slru_segment_size_to_key(kind, segno); + let buf = nblocks.to_le_bytes(); + self.put(size_key, Value::Image(Bytes::from(buf.to_vec()))); + Ok(()) + } + + /// This method is used for marking truncated SLRU files + pub fn drop_slru_segment(&mut self, kind: SlruKind, segno: u32) -> Result<()> { + // Remove it from the directory entry + let dir_key = slru_dir_to_key(kind); + let buf = self.get(dir_key)?; + let mut dir = SlruSegmentDirectory::des(&buf)?; + + if !dir.segments.remove(&segno) { + warn!("slru segment {:?}/{} does not exist", kind, segno); + } + self.put( + dir_key, + Value::Image(Bytes::from(SlruSegmentDirectory::ser(&dir)?)), + ); + + // Delete size entry, as well as all blocks + self.delete(slru_segment_key_range(kind, segno)); + + Ok(()) + } + + /// Drop a relmapper file (pg_filenode.map) + pub fn drop_relmap_file(&mut self, _spcnode: Oid, _dbnode: Oid) -> Result<()> { + // TODO + Ok(()) + } + + /// This method is used for marking truncated SLRU files + pub fn drop_twophase_file(&mut self, xid: TransactionId) -> Result<()> { + // Remove it from the directory entry + let buf = self.get(TWOPHASEDIR_KEY)?; + let mut dir = TwoPhaseDirectory::des(&buf)?; + + if !dir.xids.remove(&xid) { + warn!("twophase file for xid {} does not exist", xid); + } + self.put( + TWOPHASEDIR_KEY, + Value::Image(Bytes::from(TwoPhaseDirectory::ser(&dir)?)), + ); + + // Delete it + self.delete(twophase_key_range(xid)); + + Ok(()) + } + + /// + /// Finish this atomic update, writing all the updated keys to the + /// underlying timeline. + /// + pub fn commit(self) -> Result<()> { + let writer = self.tline.tline.writer(); + + let last_partitioning = self.tline.last_partitioning.load(); + let pending_nblocks = self.pending_nblocks; + + for (key, value) in self.pending_updates { + writer.put(key, self.lsn, value)?; + } + for key_range in self.pending_deletions { + writer.delete(key_range.clone(), self.lsn)?; + } + + writer.finish_write(self.lsn); + + if last_partitioning == Lsn(0) + || self.lsn.0 - last_partitioning.0 > self.tline.repartition_threshold + { + let keyspace = self.tline.collect_keyspace(self.lsn)?; + let partitioning = keyspace.partition(TARGET_FILE_SIZE_BYTES); + self.tline.tline.hint_partitioning(partitioning, self.lsn)?; + self.tline.last_partitioning.store(self.lsn); + } + + if pending_nblocks != 0 { + self.tline.current_logical_size.fetch_add( + pending_nblocks * pg_constants::BLCKSZ as isize, + Ordering::SeqCst, + ); + } + + Ok(()) + } + + // Internal helper functions to batch the modifications + + fn get(&self, key: Key) -> Result { + // Have we already updated the same key? Read the pending updated + // version in that case. + // + // Note: we don't check pending_deletions. It is an error to request a + // value that has been removed, deletion only avoids leaking storage. + if let Some(value) = self.pending_updates.get(&key) { + if let Value::Image(img) = value { + Ok(img.clone()) + } else { + // Currently, we never need to read back a WAL record that we + // inserted in the same "transaction". All the metadata updates + // work directly with Images, and we never need to read actual + // data pages. We could handle this if we had to, by calling + // the walredo manager, but let's keep it simple for now. + bail!("unexpected pending WAL record"); + } + } else { + let last_lsn = self.tline.get_last_record_lsn(); + self.tline.tline.get(key, last_lsn) + } + } + + fn put(&mut self, key: Key, val: Value) { + self.pending_updates.insert(key, val); + } + + fn delete(&mut self, key_range: Range) { + trace!("DELETE {}-{}", key_range.start, key_range.end); + self.pending_deletions.push(key_range); + } +} + +//--- Metadata structs stored in key-value pairs in the repository. + +#[derive(Debug, Serialize, Deserialize)] +struct DbDirectory { + // (spcnode, dbnode) -> (do relmapper and PG_VERSION files exist) + dbdirs: HashMap<(Oid, Oid), bool>, +} + +#[derive(Debug, Serialize, Deserialize)] +struct TwoPhaseDirectory { + xids: HashSet, +} + +#[derive(Debug, Serialize, Deserialize, Default)] +struct RelDirectory { + // Set of relations that exist. (relfilenode, forknum) + // + // TODO: Store it as a btree or radix tree or something else that spans multiple + // key-value pairs, if you have a lot of relations + rels: HashSet<(Oid, u8)>, +} + +#[derive(Debug, Serialize, Deserialize)] +struct RelSizeEntry { + nblocks: u32, +} + +#[derive(Debug, Serialize, Deserialize, Default)] +struct SlruSegmentDirectory { + // Set of SLRU segments that exist. + segments: HashSet, +} + +static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; pg_constants::BLCKSZ as usize]); + +// Layout of the Key address space +// +// The Key struct, used to address the underlying key-value store, consists of +// 18 bytes, split into six fields. See 'Key' in repository.rs. We need to map +// all the data and metadata keys into those 18 bytes. +// +// Principles for the mapping: +// +// - Things that are often accessed or modified together, should be close to +// each other in the key space. For example, if a relation is extended by one +// block, we create a new key-value pair for the block data, and update the +// relation size entry. Because of that, the RelSize key comes after all the +// RelBlocks of a relation: the RelSize and the last RelBlock are always next +// to each other. +// +// The key space is divided into four major sections, identified by the first +// byte, and the form a hierarchy: +// +// 00 Relation data and metadata +// +// DbDir () -> (dbnode, spcnode) +// Filenodemap +// RelDir -> relnode forknum +// RelBlocks +// RelSize +// +// 01 SLRUs +// +// SlruDir kind +// SlruSegBlocks segno +// SlruSegSize +// +// 02 pg_twophase +// +// 03 misc +// controlfile +// checkpoint +// +// Below is a full list of the keyspace allocation: +// +// DbDir: +// 00 00000000 00000000 00000000 00 00000000 +// +// Filenodemap: +// 00 SPCNODE DBNODE 00000000 00 00000000 +// +// RelDir: +// 00 SPCNODE DBNODE 00000000 00 00000001 (Postgres never uses relfilenode 0) +// +// RelBlock: +// 00 SPCNODE DBNODE RELNODE FORK BLKNUM +// +// RelSize: +// 00 SPCNODE DBNODE RELNODE FORK FFFFFFFF +// +// SlruDir: +// 01 kind 00000000 00000000 00 00000000 +// +// SlruSegBlock: +// 01 kind 00000001 SEGNO 00 BLKNUM +// +// SlruSegSize: +// 01 kind 00000001 SEGNO 00 FFFFFFFF +// +// TwoPhaseDir: +// 02 00000000 00000000 00000000 00 00000000 +// +// TwoPhaseFile: +// 02 00000000 00000000 00000000 00 XID +// +// ControlFile: +// 03 00000000 00000000 00000000 00 00000000 +// +// Checkpoint: +// 03 00000000 00000000 00000000 00 00000001 + +//-- Section 01: relation data and metadata + +const DBDIR_KEY: Key = Key { + field1: 0x00, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: 0, +}; + +fn dbdir_key_range(spcnode: Oid, dbnode: Oid) -> Range { + Key { + field1: 0x00, + field2: spcnode, + field3: dbnode, + field4: 0, + field5: 0, + field6: 0, + }..Key { + field1: 0x00, + field2: spcnode, + field3: dbnode, + field4: 0xffffffff, + field5: 0xff, + field6: 0xffffffff, + } +} + +fn relmap_file_key(spcnode: Oid, dbnode: Oid) -> Key { + Key { + field1: 0x00, + field2: spcnode, + field3: dbnode, + field4: 0, + field5: 0, + field6: 0, + } +} + +fn rel_dir_to_key(spcnode: Oid, dbnode: Oid) -> Key { + Key { + field1: 0x00, + field2: spcnode, + field3: dbnode, + field4: 0, + field5: 0, + field6: 1, + } +} + +fn rel_block_to_key(rel: RelTag, blknum: BlockNumber) -> Key { + Key { + field1: 0x00, + field2: rel.spcnode, + field3: rel.dbnode, + field4: rel.relnode, + field5: rel.forknum, + field6: blknum, + } +} + +fn rel_size_to_key(rel: RelTag) -> Key { + Key { + field1: 0x00, + field2: rel.spcnode, + field3: rel.dbnode, + field4: rel.relnode, + field5: rel.forknum, + field6: 0xffffffff, + } +} + +fn rel_key_range(rel: RelTag) -> Range { + Key { + field1: 0x00, + field2: rel.spcnode, + field3: rel.dbnode, + field4: rel.relnode, + field5: rel.forknum, + field6: 0, + }..Key { + field1: 0x00, + field2: rel.spcnode, + field3: rel.dbnode, + field4: rel.relnode, + field5: rel.forknum + 1, + field6: 0, + } +} + +//-- Section 02: SLRUs + +fn slru_dir_to_key(kind: SlruKind) -> Key { + Key { + field1: 0x01, + field2: match kind { + SlruKind::Clog => 0x00, + SlruKind::MultiXactMembers => 0x01, + SlruKind::MultiXactOffsets => 0x02, + }, + field3: 0, + field4: 0, + field5: 0, + field6: 0, + } +} + +fn slru_block_to_key(kind: SlruKind, segno: u32, blknum: BlockNumber) -> Key { + Key { + field1: 0x01, + field2: match kind { + SlruKind::Clog => 0x00, + SlruKind::MultiXactMembers => 0x01, + SlruKind::MultiXactOffsets => 0x02, + }, + field3: 1, + field4: segno, + field5: 0, + field6: blknum, + } +} + +fn slru_segment_size_to_key(kind: SlruKind, segno: u32) -> Key { + Key { + field1: 0x01, + field2: match kind { + SlruKind::Clog => 0x00, + SlruKind::MultiXactMembers => 0x01, + SlruKind::MultiXactOffsets => 0x02, + }, + field3: 1, + field4: segno, + field5: 0, + field6: 0xffffffff, + } +} + +fn slru_segment_key_range(kind: SlruKind, segno: u32) -> Range { + let field2 = match kind { + SlruKind::Clog => 0x00, + SlruKind::MultiXactMembers => 0x01, + SlruKind::MultiXactOffsets => 0x02, + }; + + Key { + field1: 0x01, + field2, + field3: segno, + field4: 0, + field5: 0, + field6: 0, + }..Key { + field1: 0x01, + field2, + field3: segno, + field4: 0, + field5: 1, + field6: 0, + } +} + +//-- Section 03: pg_twophase + +const TWOPHASEDIR_KEY: Key = Key { + field1: 0x02, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: 0, +}; + +fn twophase_file_key(xid: TransactionId) -> Key { + Key { + field1: 0x02, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: xid, + } +} + +fn twophase_key_range(xid: TransactionId) -> Range { + let (next_xid, overflowed) = xid.overflowing_add(1); + + Key { + field1: 0x02, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: xid, + }..Key { + field1: 0x02, + field2: 0, + field3: 0, + field4: 0, + field5: if overflowed { 1 } else { 0 }, + field6: next_xid, + } +} + +//-- Section 03: Control file +const CONTROLFILE_KEY: Key = Key { + field1: 0x03, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: 0, +}; + +const CHECKPOINT_KEY: Key = Key { + field1: 0x03, + field2: 0, + field3: 0, + field4: 0, + field5: 0, + field6: 1, +}; + +// Reverse mappings for a few Keys. +// These are needed by WAL redo manager. + +pub fn key_to_rel_block(key: Key) -> Result<(RelTag, BlockNumber)> { + Ok(match key.field1 { + 0x00 => ( + RelTag { + spcnode: key.field2, + dbnode: key.field3, + relnode: key.field4, + forknum: key.field5, + }, + key.field6, + ), + _ => bail!("unexpected value kind 0x{:02x}", key.field1), + }) +} + +pub fn key_to_slru_block(key: Key) -> Result<(SlruKind, u32, BlockNumber)> { + Ok(match key.field1 { + 0x01 => { + let kind = match key.field2 { + 0x00 => SlruKind::Clog, + 0x01 => SlruKind::MultiXactMembers, + 0x02 => SlruKind::MultiXactOffsets, + _ => bail!("unrecognized slru kind 0x{:02x}", key.field2), + }; + let segno = key.field4; + let blknum = key.field6; + + (kind, segno, blknum) + } + _ => bail!("unexpected value kind 0x{:02x}", key.field1), + }) +} + +// +//-- Tests that should work the same with any Repository/Timeline implementation. +// + +#[cfg(test)] +pub fn create_test_timeline( + repo: R, + timeline_id: zenith_utils::zid::ZTimelineId, +) -> Result>> { + let tline = repo.create_empty_timeline(timeline_id, Lsn(8))?; + let tline = DatadirTimeline::new(tline, crate::layered_repository::tests::TEST_FILE_SIZE / 10); + let mut m = tline.begin_modification(Lsn(8)); + m.init_empty()?; + m.commit()?; + Ok(Arc::new(tline)) +} + +#[allow(clippy::bool_assert_comparison)] +#[cfg(test)] +mod tests { + //use super::repo_harness::*; + //use super::*; + + /* + fn assert_current_logical_size(timeline: &DatadirTimeline, lsn: Lsn) { + let incremental = timeline.get_current_logical_size(); + let non_incremental = timeline + .get_current_logical_size_non_incremental(lsn) + .unwrap(); + assert_eq!(incremental, non_incremental); + } + */ + + /* + /// + /// Test list_rels() function, with branches and dropped relations + /// + #[test] + fn test_list_rels_drop() -> Result<()> { + let repo = RepoHarness::create("test_list_rels_drop")?.load(); + let tline = create_empty_timeline(repo, TIMELINE_ID)?; + const TESTDB: u32 = 111; + + // Import initial dummy checkpoint record, otherwise the get_timeline() call + // after branching fails below + let mut writer = tline.begin_record(Lsn(0x10)); + writer.put_checkpoint(ZERO_CHECKPOINT.clone())?; + writer.finish()?; + + // Create a relation on the timeline + let mut writer = tline.begin_record(Lsn(0x20)); + writer.put_rel_page_image(TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))?; + writer.finish()?; + + let writer = tline.begin_record(Lsn(0x00)); + writer.finish()?; + + // Check that list_rels() lists it after LSN 2, but no before it + assert!(!tline.list_rels(0, TESTDB, Lsn(0x10))?.contains(&TESTREL_A)); + assert!(tline.list_rels(0, TESTDB, Lsn(0x20))?.contains(&TESTREL_A)); + assert!(tline.list_rels(0, TESTDB, Lsn(0x30))?.contains(&TESTREL_A)); + + // Create a branch, check that the relation is visible there + repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x30))?; + let newtline = match repo.get_timeline(NEW_TIMELINE_ID)?.local_timeline() { + Some(timeline) => timeline, + None => panic!("Should have a local timeline"), + }; + let newtline = DatadirTimelineImpl::new(newtline); + assert!(newtline + .list_rels(0, TESTDB, Lsn(0x30))? + .contains(&TESTREL_A)); + + // Drop it on the branch + let mut new_writer = newtline.begin_record(Lsn(0x40)); + new_writer.drop_relation(TESTREL_A)?; + new_writer.finish()?; + + // Check that it's no longer listed on the branch after the point where it was dropped + assert!(newtline + .list_rels(0, TESTDB, Lsn(0x30))? + .contains(&TESTREL_A)); + assert!(!newtline + .list_rels(0, TESTDB, Lsn(0x40))? + .contains(&TESTREL_A)); + + // Run checkpoint and garbage collection and check that it's still not visible + newtline.tline.checkpoint(CheckpointConfig::Forced)?; + repo.gc_iteration(Some(NEW_TIMELINE_ID), 0, true)?; + + assert!(!newtline + .list_rels(0, TESTDB, Lsn(0x40))? + .contains(&TESTREL_A)); + + Ok(()) + } + */ + + /* + #[test] + fn test_read_beyond_eof() -> Result<()> { + let repo = RepoHarness::create("test_read_beyond_eof")?.load(); + let tline = create_test_timeline(repo, TIMELINE_ID)?; + + make_some_layers(&tline, Lsn(0x20))?; + let mut writer = tline.begin_record(Lsn(0x60)); + walingest.put_rel_page_image( + &mut writer, + TESTREL_A, + 0, + TEST_IMG(&format!("foo blk 0 at {}", Lsn(0x60))), + )?; + writer.finish()?; + + // Test read before rel creation. Should error out. + assert!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x10)).is_err()); + + // Read block beyond end of relation at different points in time. + // These reads should fall into different delta, image, and in-memory layers. + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x20))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x25))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x30))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x35))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x45))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x55))?, ZERO_PAGE); + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60))?, ZERO_PAGE); + + // Test on an in-memory layer with no preceding layer + let mut writer = tline.begin_record(Lsn(0x70)); + walingest.put_rel_page_image( + &mut writer, + TESTREL_B, + 0, + TEST_IMG(&format!("foo blk 0 at {}", Lsn(0x70))), + )?; + writer.finish()?; + + assert_eq!(tline.get_rel_page_at_lsn(TESTREL_B, 1, Lsn(0x70))?, ZERO_PAGE); + + Ok(()) + } + */ +} diff --git a/pageserver/src/relish.rs b/pageserver/src/relish.rs deleted file mode 100644 index 9228829aef..0000000000 --- a/pageserver/src/relish.rs +++ /dev/null @@ -1,226 +0,0 @@ -//! -//! Zenith stores PostgreSQL relations, and some other files, in the -//! repository. The relations (i.e. tables and indexes) take up most -//! of the space in a typical installation, while the other files are -//! small. We call each relation and other file that is stored in the -//! repository a "relish". It comes from "rel"-ish, as in "kind of a -//! rel", because it covers relations as well as other things that are -//! not relations, but are treated similarly for the purposes of the -//! storage layer. -//! -//! This source file contains the definition of the RelishTag struct, -//! which uniquely identifies a relish. -//! -//! Relishes come in two flavors: blocky and non-blocky. Relations and -//! SLRUs are blocky, that is, they are divided into 8k blocks, and -//! the repository tracks their size. Other relishes are non-blocky: -//! the content of the whole relish is stored as one blob. Block -//! number must be passed as 0 for all operations on a non-blocky -//! relish. The one "block" that you store in a non-blocky relish can -//! have arbitrary size, but they are expected to be small, or you -//! will have performance issues. -//! -//! All relishes are versioned by LSN in the repository. -//! - -use serde::{Deserialize, Serialize}; -use std::fmt; - -use postgres_ffi::relfile_utils::forknumber_to_name; -use postgres_ffi::{Oid, TransactionId}; - -/// -/// RelishTag identifies one relish. -/// -#[derive(Debug, Clone, Copy, Hash, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)] -pub enum RelishTag { - // Relations correspond to PostgreSQL relation forks. Each - // PostgreSQL relation fork is considered a separate relish. - Relation(RelTag), - - // SLRUs include pg_clog, pg_multixact/members, and - // pg_multixact/offsets. There are other SLRUs in PostgreSQL, but - // they don't need to be stored permanently (e.g. pg_subtrans), - // or we do not support them in zenith yet (pg_commit_ts). - // - // These are currently never requested directly by the compute - // nodes, although in principle that would be possible. However, - // when a new compute node is created, these are included in the - // tarball that we send to the compute node to initialize the - // PostgreSQL data directory. - // - // Each SLRU segment in PostgreSQL is considered a separate - // relish. For example, pg_clog/0000, pg_clog/0001, and so forth. - // - // SLRU segments are divided into blocks, like relations. - Slru { slru: SlruKind, segno: u32 }, - - // Miscellaneous other files that need to be included in the - // tarball at compute node creation. These are non-blocky, and are - // expected to be small. - - // - // FileNodeMap represents PostgreSQL's 'pg_filenode.map' - // files. They are needed to map catalog table OIDs to filenode - // numbers. Usually the mapping is done by looking up a relation's - // 'relfilenode' field in the 'pg_class' system table, but that - // doesn't work for 'pg_class' itself and a few other such system - // relations. See PostgreSQL relmapper.c for details. - // - // Each database has a map file for its local mapped catalogs, - // and there is a separate map file for shared catalogs. - // - // These files are always 512 bytes long (although we don't check - // or care about that in the page server). - // - FileNodeMap { spcnode: Oid, dbnode: Oid }, - - // - // State files for prepared transactions (e.g pg_twophase/1234) - // - TwoPhase { xid: TransactionId }, - - // The control file, stored in global/pg_control - ControlFile, - - // Special entry that represents PostgreSQL checkpoint. It doesn't - // correspond to to any physical file in PostgreSQL, but we use it - // to track fields needed to restore the checkpoint data in the - // control file, when a compute node is created. - Checkpoint, -} - -impl RelishTag { - pub const fn is_blocky(&self) -> bool { - match self { - // These relishes work with blocks - RelishTag::Relation(_) | RelishTag::Slru { slru: _, segno: _ } => true, - - // and these don't - RelishTag::FileNodeMap { - spcnode: _, - dbnode: _, - } - | RelishTag::TwoPhase { xid: _ } - | RelishTag::ControlFile - | RelishTag::Checkpoint => false, - } - } - - // Physical relishes represent files and use - // RelationSizeEntry to track existing and dropped files. - // They can be both blocky and non-blocky. - pub const fn is_physical(&self) -> bool { - match self { - // These relishes represent physical files - RelishTag::Relation(_) - | RelishTag::Slru { .. } - | RelishTag::FileNodeMap { .. } - | RelishTag::TwoPhase { .. } => true, - - // and these don't - RelishTag::ControlFile | RelishTag::Checkpoint => false, - } - } - - // convenience function to check if this relish is a normal relation. - pub const fn is_relation(&self) -> bool { - matches!(self, RelishTag::Relation(_)) - } -} - -/// -/// Relation data file segment id throughout the Postgres cluster. -/// -/// Every data file in Postgres is uniquely identified by 4 numbers: -/// - relation id / node (`relnode`) -/// - database id (`dbnode`) -/// - tablespace id (`spcnode`), in short this is a unique id of a separate -/// directory to store data files. -/// - forknumber (`forknum`) is used to split different kinds of data of the same relation -/// between some set of files (`relnode`, `relnode_fsm`, `relnode_vm`). -/// -/// In native Postgres code `RelFileNode` structure and individual `ForkNumber` value -/// are used for the same purpose. -/// [See more related comments here](https:///github.com/postgres/postgres/blob/99c5852e20a0987eca1c38ba0c09329d4076b6a0/src/include/storage/relfilenode.h#L57). -/// -#[derive(Debug, PartialEq, Eq, PartialOrd, Hash, Ord, Clone, Copy, Serialize, Deserialize)] -pub struct RelTag { - pub forknum: u8, - pub spcnode: Oid, - pub dbnode: Oid, - pub relnode: Oid, -} - -/// Display RelTag in the same format that's used in most PostgreSQL debug messages: -/// -/// //[_fsm|_vm|_init] -/// -impl fmt::Display for RelTag { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if let Some(forkname) = forknumber_to_name(self.forknum) { - write!( - f, - "{}/{}/{}_{}", - self.spcnode, self.dbnode, self.relnode, forkname - ) - } else { - write!(f, "{}/{}/{}", self.spcnode, self.dbnode, self.relnode) - } - } -} - -/// Display RelTag in the same format that's used in most PostgreSQL debug messages: -/// -/// //[_fsm|_vm|_init] -/// -impl fmt::Display for RelishTag { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - RelishTag::Relation(rel) => rel.fmt(f), - RelishTag::Slru { slru, segno } => { - // e.g. pg_clog/0001 - write!(f, "{}/{:04X}", slru.to_str(), segno) - } - RelishTag::FileNodeMap { spcnode, dbnode } => { - write!(f, "relmapper file for spc {} db {}", spcnode, dbnode) - } - RelishTag::TwoPhase { xid } => { - write!(f, "pg_twophase/{:08X}", xid) - } - RelishTag::ControlFile => { - write!(f, "control file") - } - RelishTag::Checkpoint => { - write!(f, "checkpoint") - } - } - } -} - -/// -/// Non-relation transaction status files (clog (a.k.a. pg_xact) and -/// pg_multixact) in Postgres are handled by SLRU (Simple LRU) buffer, -/// hence the name. -/// -/// These files are global for a postgres instance. -/// -/// These files are divided into segments, which are divided into -/// pages of the same BLCKSZ as used for relation files. -/// -#[derive(Debug, Clone, Copy, Hash, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)] -pub enum SlruKind { - Clog, - MultiXactMembers, - MultiXactOffsets, -} - -impl SlruKind { - pub fn to_str(&self) -> &'static str { - match self { - Self::Clog => "pg_xact", - Self::MultiXactMembers => "pg_multixact/members", - Self::MultiXactOffsets => "pg_multixact/offsets", - } - } -} diff --git a/pageserver/src/reltag.rs b/pageserver/src/reltag.rs new file mode 100644 index 0000000000..46ff468f2f --- /dev/null +++ b/pageserver/src/reltag.rs @@ -0,0 +1,105 @@ +use serde::{Deserialize, Serialize}; +use std::cmp::Ordering; +use std::fmt; + +use postgres_ffi::relfile_utils::forknumber_to_name; +use postgres_ffi::Oid; + +/// +/// Relation data file segment id throughout the Postgres cluster. +/// +/// Every data file in Postgres is uniquely identified by 4 numbers: +/// - relation id / node (`relnode`) +/// - database id (`dbnode`) +/// - tablespace id (`spcnode`), in short this is a unique id of a separate +/// directory to store data files. +/// - forknumber (`forknum`) is used to split different kinds of data of the same relation +/// between some set of files (`relnode`, `relnode_fsm`, `relnode_vm`). +/// +/// In native Postgres code `RelFileNode` structure and individual `ForkNumber` value +/// are used for the same purpose. +/// [See more related comments here](https:///github.com/postgres/postgres/blob/99c5852e20a0987eca1c38ba0c09329d4076b6a0/src/include/storage/relfilenode.h#L57). +/// +// FIXME: should move 'forknum' as last field to keep this consistent with Postgres. +// Then we could replace the custo Ord and PartialOrd implementations below with +// deriving them. +#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy, Serialize, Deserialize)] +pub struct RelTag { + pub forknum: u8, + pub spcnode: Oid, + pub dbnode: Oid, + pub relnode: Oid, +} + +impl PartialOrd for RelTag { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for RelTag { + fn cmp(&self, other: &Self) -> Ordering { + let mut cmp; + + cmp = self.spcnode.cmp(&other.spcnode); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.dbnode.cmp(&other.dbnode); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.relnode.cmp(&other.relnode); + if cmp != Ordering::Equal { + return cmp; + } + cmp = self.forknum.cmp(&other.forknum); + + cmp + } +} + +/// Display RelTag in the same format that's used in most PostgreSQL debug messages: +/// +/// //[_fsm|_vm|_init] +/// +impl fmt::Display for RelTag { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + if let Some(forkname) = forknumber_to_name(self.forknum) { + write!( + f, + "{}/{}/{}_{}", + self.spcnode, self.dbnode, self.relnode, forkname + ) + } else { + write!(f, "{}/{}/{}", self.spcnode, self.dbnode, self.relnode) + } + } +} + +/// +/// Non-relation transaction status files (clog (a.k.a. pg_xact) and +/// pg_multixact) in Postgres are handled by SLRU (Simple LRU) buffer, +/// hence the name. +/// +/// These files are global for a postgres instance. +/// +/// These files are divided into segments, which are divided into +/// pages of the same BLCKSZ as used for relation files. +/// +#[derive(Debug, Clone, Copy, Hash, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)] +pub enum SlruKind { + Clog, + MultiXactMembers, + MultiXactOffsets, +} + +impl SlruKind { + pub fn to_str(&self) -> &'static str { + match self { + Self::Clog => "pg_xact", + Self::MultiXactMembers => "pg_multixact/members", + Self::MultiXactOffsets => "pg_multixact/offsets", + } + } +} diff --git a/pageserver/src/remote_storage/README.md b/pageserver/src/remote_storage/README.md index 3c77275da8..339ddce866 100644 --- a/pageserver/src/remote_storage/README.md +++ b/pageserver/src/remote_storage/README.md @@ -17,7 +17,7 @@ This way, the backups are managed in background, not affecting directly other pa Current implementation * provides remote storage wrappers for AWS S3 and local FS * synchronizes the differences with local timelines and remote states as fast as possible -* uploads new relishes, frozen by pageserver checkpoint thread +* uploads new layer files * downloads and registers timelines, found on the remote storage, but missing locally, if those are requested somehow via pageserver (e.g. http api, gc) * uses compression when deals with files, for better S3 usage * maintains an index of what's stored remotely diff --git a/pageserver/src/remote_storage/local_fs.rs b/pageserver/src/remote_storage/local_fs.rs index 6cce127a7c..bac693c8d0 100644 --- a/pageserver/src/remote_storage/local_fs.rs +++ b/pageserver/src/remote_storage/local_fs.rs @@ -662,7 +662,7 @@ mod fs_tests { } async fn upload_dummy_file( - harness: &RepoHarness, + harness: &RepoHarness<'_>, storage: &LocalFs, name: &str, ) -> anyhow::Result { diff --git a/pageserver/src/remote_storage/storage_sync.rs b/pageserver/src/remote_storage/storage_sync.rs index 9fe2ab2847..ddd47ea981 100644 --- a/pageserver/src/remote_storage/storage_sync.rs +++ b/pageserver/src/remote_storage/storage_sync.rs @@ -27,7 +27,7 @@ //! it may schedule the download on such occasions. //! Then, the index is shared across pageserver under [`RemoteIndex`] guard to ensure proper synchronization. //! -//! The synchronization unit is an archive: a set of timeline files (or relishes) and a special metadata file, all compressed into a blob. +//! The synchronization unit is an archive: a set of layer files and a special metadata file, all compressed into a blob. //! Currently, there's no way to process an archive partially, if the archive processing fails, it has to be started from zero next time again. //! An archive contains set of files of a certain timeline, added during checkpoint(s) and the timeline metadata at that moment. //! The archive contains that metadata's `disk_consistent_lsn` in its name, to be able to restore partial index information from just a remote storage file list. @@ -281,7 +281,7 @@ impl SyncKind { /// Current checkpoint design assumes new files are added only, no deletions or amendment happens. #[derive(Debug, Clone)] pub struct NewCheckpoint { - /// Relish file paths in the pageserver workdir, that were added for the corresponding checkpoint. + /// layer file paths in the pageserver workdir, that were added for the corresponding checkpoint. layers: Vec, metadata: TimelineMetadata, } @@ -854,7 +854,7 @@ mod test_utils { #[track_caller] pub async fn ensure_correct_timeline_upload( - harness: &RepoHarness, + harness: &RepoHarness<'_>, remote_assets: Arc<(LocalFs, RemoteIndex)>, timeline_id: ZTimelineId, new_upload: NewCheckpoint, diff --git a/pageserver/src/remote_storage/storage_sync/compression.rs b/pageserver/src/remote_storage/storage_sync/compression.rs index ca245359bf..c5b041349a 100644 --- a/pageserver/src/remote_storage/storage_sync/compression.rs +++ b/pageserver/src/remote_storage/storage_sync/compression.rs @@ -10,7 +10,7 @@ //! Archiving is almost agnostic to timeline file types, with an exception of the metadata file, that's currently distinguished in the [un]compression code. //! The metadata file is treated separately when [de]compression is involved, to reduce the risk of corrupting the metadata file. //! When compressed, the metadata file is always required and stored as the last file in the archive stream. -//! When uncompressed, the metadata file gets naturally uncompressed last, to ensure that all other relishes are decompressed successfully first. +//! When uncompressed, the metadata file gets naturally uncompressed last, to ensure that all other layer files are decompressed successfully first. //! //! Archive structure: //! +----------------------------------------+ diff --git a/pageserver/src/remote_storage/storage_sync/index.rs b/pageserver/src/remote_storage/storage_sync/index.rs index d7bd1f1657..861b78fa3b 100644 --- a/pageserver/src/remote_storage/storage_sync/index.rs +++ b/pageserver/src/remote_storage/storage_sync/index.rs @@ -277,7 +277,7 @@ impl RemoteTimeline { .map(CheckpointArchive::disk_consistent_lsn) } - /// Lists all relish files in the given remote timeline. Omits the metadata file. + /// Lists all layer files in the given remote timeline. Omits the metadata file. pub fn stored_files(&self, timeline_dir: &Path) -> BTreeSet { self.timeline_files .values() diff --git a/pageserver/src/repository.rs b/pageserver/src/repository.rs index 36273e6d6c..b960e037be 100644 --- a/pageserver/src/repository.rs +++ b/pageserver/src/repository.rs @@ -1,22 +1,173 @@ +use crate::keyspace::KeyPartitioning; use crate::layered_repository::metadata::TimelineMetadata; -use crate::relish::*; use crate::remote_storage::RemoteIndex; -use crate::walrecord::MultiXactMember; +use crate::walrecord::ZenithWalRecord; use crate::CheckpointConfig; -use anyhow::Result; +use anyhow::{bail, Result}; use bytes::Bytes; -use postgres_ffi::{MultiXactId, MultiXactOffset, TransactionId}; use serde::{Deserialize, Serialize}; -use std::collections::HashSet; +use std::fmt; use std::fmt::Display; -use std::ops::{AddAssign, Deref}; +use std::ops::{AddAssign, Range}; use std::sync::{Arc, RwLockReadGuard}; use std::time::Duration; use zenith_utils::lsn::{Lsn, RecordLsn}; use zenith_utils::zid::ZTimelineId; -/// Block number within a relish. This matches PostgreSQL's BlockNumber type. -pub type BlockNumber = u32; +#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq, Ord, PartialOrd, Serialize, Deserialize)] +/// Key used in the Repository kv-store. +/// +/// The Repository treates this as an opaque struct, but see the code in pgdatadir_mapping.rs +/// for what we actually store in these fields. +pub struct Key { + pub field1: u8, + pub field2: u32, + pub field3: u32, + pub field4: u32, + pub field5: u8, + pub field6: u32, +} + +impl Key { + pub fn next(&self) -> Key { + self.add(1) + } + + pub fn add(&self, x: u32) -> Key { + let mut key = *self; + + let r = key.field6.overflowing_add(x); + key.field6 = r.0; + if r.1 { + let r = key.field5.overflowing_add(1); + key.field5 = r.0; + if r.1 { + let r = key.field4.overflowing_add(1); + key.field4 = r.0; + if r.1 { + let r = key.field3.overflowing_add(1); + key.field3 = r.0; + if r.1 { + let r = key.field2.overflowing_add(1); + key.field2 = r.0; + if r.1 { + let r = key.field1.overflowing_add(1); + key.field1 = r.0; + assert!(!r.1); + } + } + } + } + } + key + } + + pub fn from_array(b: [u8; 18]) -> Self { + Key { + field1: b[0], + field2: u32::from_be_bytes(b[1..5].try_into().unwrap()), + field3: u32::from_be_bytes(b[5..9].try_into().unwrap()), + field4: u32::from_be_bytes(b[9..13].try_into().unwrap()), + field5: b[13], + field6: u32::from_be_bytes(b[14..18].try_into().unwrap()), + } + } +} + +pub fn key_range_size(key_range: &Range) -> u32 { + let start = key_range.start; + let end = key_range.end; + + if end.field1 != start.field1 + || end.field2 != start.field2 + || end.field3 != start.field3 + || end.field4 != start.field4 + { + return u32::MAX; + } + + let start = (start.field5 as u64) << 32 | start.field6 as u64; + let end = (end.field5 as u64) << 32 | end.field6 as u64; + + let diff = end - start; + if diff > u32::MAX as u64 { + u32::MAX + } else { + diff as u32 + } +} + +pub fn singleton_range(key: Key) -> Range { + key..key.next() +} + +impl fmt::Display for Key { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "{:02X}{:08X}{:08X}{:08X}{:02X}{:08X}", + self.field1, self.field2, self.field3, self.field4, self.field5, self.field6 + ) + } +} + +impl Key { + pub const MIN: Key = Key { + field1: u8::MIN, + field2: u32::MIN, + field3: u32::MIN, + field4: u32::MIN, + field5: u8::MIN, + field6: u32::MIN, + }; + pub const MAX: Key = Key { + field1: u8::MAX, + field2: u32::MAX, + field3: u32::MAX, + field4: u32::MAX, + field5: u8::MAX, + field6: u32::MAX, + }; + + pub fn from_hex(s: &str) -> Result { + if s.len() != 36 { + bail!("parse error"); + } + Ok(Key { + field1: u8::from_str_radix(&s[0..2], 16)?, + field2: u32::from_str_radix(&s[2..10], 16)?, + field3: u32::from_str_radix(&s[10..18], 16)?, + field4: u32::from_str_radix(&s[18..26], 16)?, + field5: u8::from_str_radix(&s[26..28], 16)?, + field6: u32::from_str_radix(&s[28..36], 16)?, + }) + } +} + +/// A 'value' stored for a one Key. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub enum Value { + /// An Image value contains a full copy of the value + Image(Bytes), + /// A WalRecord value contains a WAL record that needs to be + /// replayed get the full value. Replaying the WAL record + /// might need a previous version of the value (if will_init() + /// returns false), or it may be replayed stand-alone (true). + WalRecord(ZenithWalRecord), +} + +impl Value { + pub fn is_image(&self) -> bool { + matches!(self, Value::Image(_)) + } + + pub fn will_init(&self) -> bool { + match self { + Value::Image(_) => true, + Value::WalRecord(rec) => rec.will_init(), + } + } +} #[derive(Clone, Copy, Debug)] pub enum TimelineSyncStatusUpdate { @@ -37,6 +188,8 @@ impl Display for TimelineSyncStatusUpdate { /// A repository corresponds to one .zenith directory. One repository holds multiple /// timelines, forked off from the same initial call to 'initdb'. pub trait Repository: Send + Sync { + type Timeline: Timeline; + /// Updates timeline based on the `TimelineSyncStatusUpdate`, received from the remote storage synchronization. /// See [`crate::remote_storage`] for more details about the synchronization. fn apply_timeline_remote_sync_status_update( @@ -47,14 +200,14 @@ pub trait Repository: Send + Sync { /// Get Timeline handle for given zenith timeline ID. /// This function is idempotent. It doesnt change internal state in any way. - fn get_timeline(&self, timelineid: ZTimelineId) -> Option; + fn get_timeline(&self, timelineid: ZTimelineId) -> Option>; /// Get Timeline handle for locally available timeline. Load it into memory if it is not loaded. - fn get_timeline_load(&self, timelineid: ZTimelineId) -> Result>; + fn get_timeline_load(&self, timelineid: ZTimelineId) -> Result>; /// Lists timelines the repository contains. /// Up to repository's implementation to omit certain timelines that ar not considered ready for use. - fn list_timelines(&self) -> Vec<(ZTimelineId, RepositoryTimeline)>; + fn list_timelines(&self) -> Vec<(ZTimelineId, RepositoryTimeline)>; /// Create a new, empty timeline. The caller is responsible for loading data into it /// Initdb lsn is provided for timeline impl to be able to perform checks for some operations against it. @@ -62,11 +215,16 @@ pub trait Repository: Send + Sync { &self, timelineid: ZTimelineId, initdb_lsn: Lsn, - ) -> Result>; + ) -> Result>; /// Branch a timeline fn branch_timeline(&self, src: ZTimelineId, dst: ZTimelineId, start_lsn: Lsn) -> Result<()>; + /// Flush all data to disk. + /// + /// this is used at graceful shutdown. + fn checkpoint(&self) -> Result<()>; + /// perform one garbage collection iteration, removing old data files from disk. /// this function is periodically called by gc thread. /// also it can be explicitly requested through page server api 'do_gc' command. @@ -83,9 +241,9 @@ pub trait Repository: Send + Sync { checkpoint_before_gc: bool, ) -> Result; - /// perform one checkpoint iteration, flushing in-memory data on disk. - /// this function is periodically called by checkponter thread. - fn checkpoint_iteration(&self, cconf: CheckpointConfig) -> Result<()>; + /// perform one compaction iteration. + /// this function is periodically called by compactor thread. + fn compaction_iteration(&self) -> Result<()>; /// detaches locally available timeline by stopping all threads and removing all the data. fn detach_timeline(&self, timeline_id: ZTimelineId) -> Result<()>; @@ -95,10 +253,10 @@ pub trait Repository: Send + Sync { } /// A timeline, that belongs to the current repository. -pub enum RepositoryTimeline { +pub enum RepositoryTimeline { /// Timeline, with its files present locally in pageserver's working directory. /// Loaded into pageserver's memory and ready to be used. - Loaded(Arc), + Loaded(Arc), /// All the data is available locally, but not loaded into memory, so loading have to be done before actually using the timeline Unloaded { @@ -118,8 +276,8 @@ pub enum LocalTimelineState { Unloaded, } -impl<'a> From<&'a RepositoryTimeline> for LocalTimelineState { - fn from(local_timeline_entry: &'a RepositoryTimeline) -> Self { +impl<'a, T> From<&'a RepositoryTimeline> for LocalTimelineState { + fn from(local_timeline_entry: &'a RepositoryTimeline) -> Self { match local_timeline_entry { RepositoryTimeline::Loaded(_) => LocalTimelineState::Loaded, RepositoryTimeline::Unloaded { .. } => LocalTimelineState::Unloaded, @@ -132,42 +290,22 @@ impl<'a> From<&'a RepositoryTimeline> for LocalTimelineState { /// #[derive(Default)] pub struct GcResult { - pub ondisk_relfiles_total: u64, - pub ondisk_relfiles_needed_by_cutoff: u64, - pub ondisk_relfiles_needed_by_branches: u64, - pub ondisk_relfiles_not_updated: u64, - pub ondisk_relfiles_needed_as_tombstone: u64, - pub ondisk_relfiles_removed: u64, // # of layer files removed because they have been made obsolete by newer ondisk files. - pub ondisk_relfiles_dropped: u64, // # of layer files removed because the relation was dropped - - pub ondisk_nonrelfiles_total: u64, - pub ondisk_nonrelfiles_needed_by_cutoff: u64, - pub ondisk_nonrelfiles_needed_by_branches: u64, - pub ondisk_nonrelfiles_not_updated: u64, - pub ondisk_nonrelfiles_needed_as_tombstone: u64, - pub ondisk_nonrelfiles_removed: u64, // # of layer files removed because they have been made obsolete by newer ondisk files. - pub ondisk_nonrelfiles_dropped: u64, // # of layer files removed because the relation was dropped + pub layers_total: u64, + pub layers_needed_by_cutoff: u64, + pub layers_needed_by_branches: u64, + pub layers_not_updated: u64, + pub layers_removed: u64, // # of layer files removed because they have been made obsolete by newer ondisk files. pub elapsed: Duration, } impl AddAssign for GcResult { fn add_assign(&mut self, other: Self) { - self.ondisk_relfiles_total += other.ondisk_relfiles_total; - self.ondisk_relfiles_needed_by_cutoff += other.ondisk_relfiles_needed_by_cutoff; - self.ondisk_relfiles_needed_by_branches += other.ondisk_relfiles_needed_by_branches; - self.ondisk_relfiles_not_updated += other.ondisk_relfiles_not_updated; - self.ondisk_relfiles_needed_as_tombstone += other.ondisk_relfiles_needed_as_tombstone; - self.ondisk_relfiles_removed += other.ondisk_relfiles_removed; - self.ondisk_relfiles_dropped += other.ondisk_relfiles_dropped; - - self.ondisk_nonrelfiles_total += other.ondisk_nonrelfiles_total; - self.ondisk_nonrelfiles_needed_by_cutoff += other.ondisk_nonrelfiles_needed_by_cutoff; - self.ondisk_nonrelfiles_needed_by_branches += other.ondisk_nonrelfiles_needed_by_branches; - self.ondisk_nonrelfiles_not_updated += other.ondisk_nonrelfiles_not_updated; - self.ondisk_nonrelfiles_needed_as_tombstone += other.ondisk_nonrelfiles_needed_as_tombstone; - self.ondisk_nonrelfiles_removed += other.ondisk_nonrelfiles_removed; - self.ondisk_nonrelfiles_dropped += other.ondisk_nonrelfiles_dropped; + self.layers_total += other.layers_total; + self.layers_needed_by_cutoff += other.layers_needed_by_cutoff; + self.layers_needed_by_branches += other.layers_needed_by_branches; + self.layers_not_updated += other.layers_not_updated; + self.layers_removed += other.layers_removed; self.elapsed += other.elapsed; } @@ -190,23 +328,14 @@ pub trait Timeline: Send + Sync { fn get_latest_gc_cutoff_lsn(&self) -> RwLockReadGuard; /// Look up given page version. - fn get_page_at_lsn(&self, tag: RelishTag, blknum: BlockNumber, lsn: Lsn) -> Result; - - /// Get size of a relish - fn get_relish_size(&self, tag: RelishTag, lsn: Lsn) -> Result>; - - /// Does relation exist? - fn get_rel_exists(&self, tag: RelishTag, lsn: Lsn) -> Result; - - /// Get a list of all existing relations - /// Pass RelTag to get relation objects or None to get nonrels. - fn list_relishes(&self, tag: Option, lsn: Lsn) -> Result>; - - /// Get a list of all existing relations in given tablespace and database. - fn list_rels(&self, spcnode: u32, dbnode: u32, lsn: Lsn) -> Result>; - - /// Get a list of all existing non-relational objects - fn list_nonrels(&self, lsn: Lsn) -> Result>; + /// + /// NOTE: It is considerd an error to 'get' a key that doesn't exist. The abstraction + /// above this needs to store suitable metadata to track what data exists with + /// what keys, in separate metadata entries. If a non-existent key is requested, + /// the Repository implementation may incorrectly return a value from an ancestore + /// branch, for exampel, or waste a lot of cycles chasing the non-existing key. + /// + fn get(&self, key: Key, lsn: Lsn) -> Result; /// Get the ancestor's timeline id fn get_ancestor_timeline_id(&self) -> Option; @@ -219,7 +348,6 @@ pub trait Timeline: Send + Sync { // // These are called by the WAL receiver to digest WAL records. //------------------------------------------------------------------------------ - /// Atomically get both last and prev. fn get_last_record_rlsn(&self) -> RecordLsn; @@ -231,6 +359,10 @@ pub trait Timeline: Send + Sync { fn get_disk_consistent_lsn(&self) -> Lsn; /// Mutate the timeline with a [`TimelineWriter`]. + /// + /// FIXME: This ought to return &'a TimelineWriter, where TimelineWriter + /// is a generic type in this trait. But that doesn't currently work in + /// Rust: https://rust-lang.github.io/rfcs/1598-generic_associated_types.html fn writer<'a>(&'a self) -> Box; /// @@ -240,6 +372,19 @@ pub trait Timeline: Send + Sync { /// know anything about them here in the repository. fn checkpoint(&self, cconf: CheckpointConfig) -> Result<()>; + /// + /// Tell the implementation how the keyspace should be partitioned. + /// + /// FIXME: This is quite a hack. The code in pgdatadir_mapping.rs knows + /// which keys exist and what is the logical grouping of them. That's why + /// the code there (and in keyspace.rs) decides the partitioning, not the + /// layered_repository.rs implementation. That's a layering violation: + /// the Repository implementation ought to be responsible for the physical + /// layout, but currently it's more convenient to do it in pgdatadir_mapping.rs + /// rather than in layered_repository.rs. + /// + fn hint_partitioning(&self, partitioning: KeyPartitioning, lsn: Lsn) -> Result<()>; + /// /// Check that it is valid to request operations with that lsn. fn check_lsn_is_in_scope( @@ -247,107 +392,39 @@ pub trait Timeline: Send + Sync { lsn: Lsn, latest_gc_cutoff_lsn: &RwLockReadGuard, ) -> Result<()>; - - /// Retrieve current logical size of the timeline - /// - /// NOTE: counted incrementally, includes ancestors, - /// doesnt support TwoPhase relishes yet - fn get_current_logical_size(&self) -> usize; - - /// Does the same as get_current_logical_size but counted on demand. - /// Used in tests to ensure that incremental and non incremental variants match. - fn get_current_logical_size_non_incremental(&self, lsn: Lsn) -> Result; - - /// An escape hatch to allow "casting" a generic Timeline to LayeredTimeline. - fn upgrade_to_layered_timeline(&self) -> &crate::layered_repository::LayeredTimeline; } /// Various functions to mutate the timeline. // TODO Currently, Deref is used to allow easy access to read methods from this trait. // This is probably considered a bad practice in Rust and should be fixed eventually, // but will cause large code changes. -pub trait TimelineWriter: Deref { +pub trait TimelineWriter<'a> { /// Put a new page version that can be constructed from a WAL record /// /// This will implicitly extend the relation, if the page is beyond the /// current end-of-file. - fn put_wal_record( - &self, - lsn: Lsn, - tag: RelishTag, - blknum: BlockNumber, - rec: ZenithWalRecord, - ) -> Result<()>; + fn put(&self, key: Key, lsn: Lsn, value: Value) -> Result<()>; - /// Like put_wal_record, but with ready-made image of the page. - fn put_page_image( - &self, - tag: RelishTag, - blknum: BlockNumber, - lsn: Lsn, - img: Bytes, - ) -> Result<()>; + fn delete(&self, key_range: Range, lsn: Lsn) -> Result<()>; - /// Truncate relation - fn put_truncation(&self, rel: RelishTag, lsn: Lsn, nblocks: BlockNumber) -> Result<()>; - - /// This method is used for marking dropped relations and truncated SLRU files and aborted two phase records - fn drop_relish(&self, tag: RelishTag, lsn: Lsn) -> Result<()>; - - /// Track end of the latest digested WAL record. + /// Track the end of the latest digested WAL record. /// - /// Advance requires aligned LSN as an argument and would wake wait_lsn() callers. - /// Previous last record LSN is stored alongside the latest and can be read. - fn advance_last_record_lsn(&self, lsn: Lsn); -} - -/// Each update to a page is represented by a ZenithWalRecord. It can be a wrapper -/// around a PostgreSQL WAL record, or a custom zenith-specific "record". -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] -pub enum ZenithWalRecord { - /// Native PostgreSQL WAL record - Postgres { will_init: bool, rec: Bytes }, - - /// Clear bits in heap visibility map. ('flags' is bitmap of bits to clear) - ClearVisibilityMapFlags { - new_heap_blkno: Option, - old_heap_blkno: Option, - flags: u8, - }, - /// Mark transaction IDs as committed on a CLOG page - ClogSetCommitted { xids: Vec }, - /// Mark transaction IDs as aborted on a CLOG page - ClogSetAborted { xids: Vec }, - /// Extend multixact offsets SLRU - MultixactOffsetCreate { - mid: MultiXactId, - moff: MultiXactOffset, - }, - /// Extend multixact members SLRU. - MultixactMembersCreate { - moff: MultiXactOffset, - members: Vec, - }, -} - -impl ZenithWalRecord { - /// Does replaying this WAL record initialize the page from scratch, or does - /// it need to be applied over the previous image of the page? - pub fn will_init(&self) -> bool { - match self { - ZenithWalRecord::Postgres { will_init, rec: _ } => *will_init, - - // None of the special zenith record types currently initialize the page - _ => false, - } - } + /// Call this after you have finished writing all the WAL up to 'lsn'. + /// + /// 'lsn' must be aligned. This wakes up any wait_lsn() callers waiting for + /// the 'lsn' or anything older. The previous last record LSN is stored alongside + /// the latest and can be read. + fn finish_write(&self, lsn: Lsn); } #[cfg(test)] pub mod repo_harness { use bytes::BytesMut; + use lazy_static::lazy_static; + use std::sync::{Arc, RwLock, RwLockReadGuard, RwLockWriteGuard}; use std::{fs, path::PathBuf}; + use crate::RepositoryImpl; use crate::{ config::PageServerConf, layered_repository::LayeredRepository, @@ -368,18 +445,39 @@ pub mod repo_harness { pub fn TEST_IMG(s: &str) -> Bytes { let mut buf = BytesMut::new(); buf.extend_from_slice(s.as_bytes()); - buf.resize(8192, 0); + buf.resize(64, 0); buf.freeze() } - pub struct RepoHarness { - pub conf: &'static PageServerConf, - pub tenant_id: ZTenantId, + lazy_static! { + static ref LOCK: RwLock<()> = RwLock::new(()); } - impl RepoHarness { + pub struct RepoHarness<'a> { + pub conf: &'static PageServerConf, + pub tenant_id: ZTenantId, + + pub lock_guard: ( + Option>, + Option>, + ), + } + + impl<'a> RepoHarness<'a> { pub fn create(test_name: &'static str) -> Result { + Self::create_internal(test_name, false) + } + pub fn create_exclusive(test_name: &'static str) -> Result { + Self::create_internal(test_name, true) + } + fn create_internal(test_name: &'static str, exclusive: bool) -> Result { + let lock_guard = if exclusive { + (None, Some(LOCK.write().unwrap())) + } else { + (Some(LOCK.read().unwrap()), None) + }; + let repo_dir = PageServerConf::test_repo_dir(test_name); let _ = fs::remove_dir_all(&repo_dir); fs::create_dir_all(&repo_dir)?; @@ -393,23 +491,27 @@ pub mod repo_harness { fs::create_dir_all(conf.tenant_path(&tenant_id))?; fs::create_dir_all(conf.timelines_path(&tenant_id))?; - Ok(Self { conf, tenant_id }) + Ok(Self { + conf, + tenant_id, + lock_guard, + }) } - pub fn load(&self) -> Box { + pub fn load(&self) -> RepositoryImpl { self.try_load().expect("failed to load test repo") } - pub fn try_load(&self) -> Result> { + pub fn try_load(&self) -> Result { let walredo_mgr = Arc::new(TestRedoManager); - let repo = Box::new(LayeredRepository::new( + let repo = LayeredRepository::new( self.conf, walredo_mgr, self.tenant_id, RemoteIndex::empty(), false, - )); + ); // populate repo with locally available timelines for timeline_dir_entry in fs::read_dir(self.conf.timelines_path(&self.tenant_id)) .expect("should be able to read timelines dir") @@ -438,21 +540,19 @@ pub mod repo_harness { } // Mock WAL redo manager that doesn't do much - struct TestRedoManager; + pub struct TestRedoManager; impl WalRedoManager for TestRedoManager { fn request_redo( &self, - rel: RelishTag, - blknum: BlockNumber, + key: Key, lsn: Lsn, base_img: Option, records: Vec<(Lsn, ZenithWalRecord)>, ) -> Result { let s = format!( - "redo for {} blk {} to get to {}, with {} and {} records", - rel, - blknum, + "redo for {} to get to {}, with {} and {} records", + key, lsn, if base_img.is_some() { "base image" @@ -462,6 +562,7 @@ pub mod repo_harness { records.len() ); println!("{}", s); + Ok(TEST_IMG(&s)) } } @@ -475,411 +576,43 @@ pub mod repo_harness { mod tests { use super::repo_harness::*; use super::*; - use postgres_ffi::{pg_constants, xlog_utils::SIZEOF_CHECKPOINT}; - use std::fs; + //use postgres_ffi::{pg_constants, xlog_utils::SIZEOF_CHECKPOINT}; + //use std::sync::Arc; + use bytes::BytesMut; + use hex_literal::hex; + use lazy_static::lazy_static; - /// Arbitrary relation tag, for testing. - const TESTREL_A_REL_TAG: RelTag = RelTag { - spcnode: 0, - dbnode: 111, - relnode: 1000, - forknum: 0, - }; - const TESTREL_A: RelishTag = RelishTag::Relation(TESTREL_A_REL_TAG); - const TESTREL_B: RelishTag = RelishTag::Relation(RelTag { - spcnode: 0, - dbnode: 111, - relnode: 1001, - forknum: 0, - }); - - fn assert_current_logical_size(timeline: &Arc, lsn: Lsn) { - let incremental = timeline.get_current_logical_size(); - let non_incremental = timeline - .get_current_logical_size_non_incremental(lsn) - .unwrap(); - assert_eq!(incremental, non_incremental); + lazy_static! { + static ref TEST_KEY: Key = Key::from_array(hex!("112222222233333333444444445500000001")); } - static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; 8192]); - static ZERO_CHECKPOINT: Bytes = Bytes::from_static(&[0u8; SIZEOF_CHECKPOINT]); - #[test] - fn test_relsize() -> Result<()> { - let repo = RepoHarness::create("test_relsize")?.load(); - // get_timeline() with non-existent timeline id should fail - //repo.get_timeline("11223344556677881122334455667788"); - - // Create timeline to work on + fn test_basic() -> Result<()> { + let repo = RepoHarness::create("test_basic")?.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; + let writer = tline.writer(); + writer.put(*TEST_KEY, Lsn(0x10), Value::Image(TEST_IMG("foo at 0x10")))?; + writer.finish_write(Lsn(0x10)); + drop(writer); - writer.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?; - writer.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?; - writer.put_page_image(TESTREL_A, 0, Lsn(0x30), TEST_IMG("foo blk 0 at 3"))?; - writer.put_page_image(TESTREL_A, 1, Lsn(0x40), TEST_IMG("foo blk 1 at 4"))?; - writer.put_page_image(TESTREL_A, 2, Lsn(0x50), TEST_IMG("foo blk 2 at 5"))?; + let writer = tline.writer(); + writer.put(*TEST_KEY, Lsn(0x20), Value::Image(TEST_IMG("foo at 0x20")))?; + writer.finish_write(Lsn(0x20)); + drop(writer); - writer.advance_last_record_lsn(Lsn(0x50)); - - assert_current_logical_size(&tline, Lsn(0x50)); - - // The relation was created at LSN 2, not visible at LSN 1 yet. - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10))?, false); - assert!(tline.get_relish_size(TESTREL_A, Lsn(0x10))?.is_none()); - - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x20))?.unwrap(), 1); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x50))?.unwrap(), 3); - - // Check page contents at each LSN - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x20))?, - TEST_IMG("foo blk 0 at 2") - ); - - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x30))?, - TEST_IMG("foo blk 0 at 3") - ); - - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x40))?, - TEST_IMG("foo blk 0 at 3") - ); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x40))?, - TEST_IMG("foo blk 1 at 4") - ); - - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x50))?, - TEST_IMG("foo blk 0 at 3") - ); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x50))?, - TEST_IMG("foo blk 1 at 4") - ); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 2, Lsn(0x50))?, - TEST_IMG("foo blk 2 at 5") - ); - - // Truncate last block - writer.put_truncation(TESTREL_A, Lsn(0x60), 2)?; - writer.advance_last_record_lsn(Lsn(0x60)); - assert_current_logical_size(&tline, Lsn(0x60)); - - // Check reported size and contents after truncation - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x60))?.unwrap(), 2); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x60))?, - TEST_IMG("foo blk 0 at 3") - ); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x60))?, - TEST_IMG("foo blk 1 at 4") - ); - - // should still see the truncated block with older LSN - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x50))?.unwrap(), 3); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 2, Lsn(0x50))?, - TEST_IMG("foo blk 2 at 5") - ); - - // Truncate to zero length - writer.put_truncation(TESTREL_A, Lsn(0x68), 0)?; - writer.advance_last_record_lsn(Lsn(0x68)); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x68))?.unwrap(), 0); - - // Extend from 0 to 2 blocks, leaving a gap - writer.put_page_image(TESTREL_A, 1, Lsn(0x70), TEST_IMG("foo blk 1"))?; - writer.advance_last_record_lsn(Lsn(0x70)); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x70))?.unwrap(), 2); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x70))?, ZERO_PAGE); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x70))?, - TEST_IMG("foo blk 1") - ); - - // Extend a lot more, leaving a big gap that spans across segments - // FIXME: This is currently broken, see https://github.com/zenithdb/zenith/issues/500 - /* - tline.put_page_image(TESTREL_A, 1500, Lsn(0x80), TEST_IMG("foo blk 1500"))?; - tline.advance_last_record_lsn(Lsn(0x80)); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x80))?.unwrap(), 1501); - for blk in 2..1500 { - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, blk, Lsn(0x80))?, - ZERO_PAGE); - } - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 1500, Lsn(0x80))?, - TEST_IMG("foo blk 1500")); - */ + assert_eq!(tline.get(*TEST_KEY, Lsn(0x10))?, TEST_IMG("foo at 0x10")); + assert_eq!(tline.get(*TEST_KEY, Lsn(0x1f))?, TEST_IMG("foo at 0x10")); + assert_eq!(tline.get(*TEST_KEY, Lsn(0x20))?, TEST_IMG("foo at 0x20")); Ok(()) } - // Test what happens if we dropped a relation - // and then created it again within the same layer. - #[test] - fn test_drop_extend() -> Result<()> { - let repo = RepoHarness::create("test_drop_extend")?.load(); - - // Create timeline to work on - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - let writer = tline.writer(); - - writer.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?; - writer.advance_last_record_lsn(Lsn(0x20)); - - // Check that rel exists and size is correct - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x20))?.unwrap(), 1); - - // Drop relish - writer.drop_relish(TESTREL_A, Lsn(0x30))?; - writer.advance_last_record_lsn(Lsn(0x30)); - - // Check that rel is not visible anymore - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x30))?, false); - assert!(tline.get_relish_size(TESTREL_A, Lsn(0x30))?.is_none()); - - // Extend it again - writer.put_page_image(TESTREL_A, 0, Lsn(0x40), TEST_IMG("foo blk 0 at 4"))?; - writer.advance_last_record_lsn(Lsn(0x40)); - - // Check that rel exists and size is correct - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x40))?, true); - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x40))?.unwrap(), 1); - - Ok(()) - } - - // Test what happens if we truncated a relation - // so that one of its segments was dropped - // and then extended it again within the same layer. - #[test] - fn test_truncate_extend() -> Result<()> { - let repo = RepoHarness::create("test_truncate_extend")?.load(); - - // Create timeline to work on - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - let writer = tline.writer(); - - //from storage_layer.rs - const RELISH_SEG_SIZE: u32 = 10 * 1024 * 1024 / 8192; - let relsize = RELISH_SEG_SIZE * 2; - - // Create relation with relsize blocks - for blkno in 0..relsize { - let lsn = Lsn(0x20); - let data = format!("foo blk {} at {}", blkno, lsn); - writer.put_page_image(TESTREL_A, blkno, lsn, TEST_IMG(&data))?; - } - - writer.advance_last_record_lsn(Lsn(0x20)); - - // The relation was created at LSN 2, not visible at LSN 1 yet. - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10))?, false); - assert!(tline.get_relish_size(TESTREL_A, Lsn(0x10))?.is_none()); - - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(0x20))?.unwrap(), - relsize - ); - - // Check relation content - for blkno in 0..relsize { - let lsn = Lsn(0x20); - let data = format!("foo blk {} at {}", blkno, lsn); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, blkno, lsn)?, - TEST_IMG(&data) - ); - } - - // Truncate relation so that second segment was dropped - // - only leave one page - writer.put_truncation(TESTREL_A, Lsn(0x60), 1)?; - writer.advance_last_record_lsn(Lsn(0x60)); - - // Check reported size and contents after truncation - assert_eq!(tline.get_relish_size(TESTREL_A, Lsn(0x60))?.unwrap(), 1); - - for blkno in 0..1 { - let lsn = Lsn(0x20); - let data = format!("foo blk {} at {}", blkno, lsn); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, blkno, Lsn(0x60))?, - TEST_IMG(&data) - ); - } - - // should still see all blocks with older LSN - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(0x50))?.unwrap(), - relsize - ); - for blkno in 0..relsize { - let lsn = Lsn(0x20); - let data = format!("foo blk {} at {}", blkno, lsn); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, blkno, Lsn(0x50))?, - TEST_IMG(&data) - ); - } - - // Extend relation again. - // Add enough blocks to create second segment - for blkno in 0..relsize { - let lsn = Lsn(0x80); - let data = format!("foo blk {} at {}", blkno, lsn); - writer.put_page_image(TESTREL_A, blkno, lsn, TEST_IMG(&data))?; - } - writer.advance_last_record_lsn(Lsn(0x80)); - - assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x80))?, true); - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(0x80))?.unwrap(), - relsize - ); - // Check relation content - for blkno in 0..relsize { - let lsn = Lsn(0x80); - let data = format!("foo blk {} at {}", blkno, lsn); - assert_eq!( - tline.get_page_at_lsn(TESTREL_A, blkno, Lsn(0x80))?, - TEST_IMG(&data) - ); - } - - Ok(()) - } - - /// Test get_relsize() and truncation with a file larger than 1 GB, so that it's - /// split into multiple 1 GB segments in Postgres. - #[test] - fn test_large_rel() -> Result<()> { - let repo = RepoHarness::create("test_large_rel")?.load(); - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - let writer = tline.writer(); - - let mut lsn = 0x10; - for blknum in 0..pg_constants::RELSEG_SIZE + 1 { - lsn += 0x10; - let img = TEST_IMG(&format!("foo blk {} at {}", blknum, Lsn(lsn))); - writer.put_page_image(TESTREL_A, blknum as BlockNumber, Lsn(lsn), img)?; - } - writer.advance_last_record_lsn(Lsn(lsn)); - - assert_current_logical_size(&tline, Lsn(lsn)); - - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(lsn))?.unwrap(), - pg_constants::RELSEG_SIZE + 1 - ); - - // Truncate one block - lsn += 0x10; - writer.put_truncation(TESTREL_A, Lsn(lsn), pg_constants::RELSEG_SIZE)?; - writer.advance_last_record_lsn(Lsn(lsn)); - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(lsn))?.unwrap(), - pg_constants::RELSEG_SIZE - ); - assert_current_logical_size(&tline, Lsn(lsn)); - - // Truncate another block - lsn += 0x10; - writer.put_truncation(TESTREL_A, Lsn(lsn), pg_constants::RELSEG_SIZE - 1)?; - writer.advance_last_record_lsn(Lsn(lsn)); - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(lsn))?.unwrap(), - pg_constants::RELSEG_SIZE - 1 - ); - assert_current_logical_size(&tline, Lsn(lsn)); - - // Truncate to 1500, and then truncate all the way down to 0, one block at a time - // This tests the behavior at segment boundaries - let mut size: i32 = 3000; - while size >= 0 { - lsn += 0x10; - writer.put_truncation(TESTREL_A, Lsn(lsn), size as BlockNumber)?; - writer.advance_last_record_lsn(Lsn(lsn)); - assert_eq!( - tline.get_relish_size(TESTREL_A, Lsn(lsn))?.unwrap(), - size as BlockNumber - ); - - size -= 1; - } - assert_current_logical_size(&tline, Lsn(lsn)); - - Ok(()) - } - - /// - /// Test list_rels() function, with branches and dropped relations - /// - #[test] - fn test_list_rels_drop() -> Result<()> { - let repo = RepoHarness::create("test_list_rels_drop")?.load(); - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - let writer = tline.writer(); - const TESTDB: u32 = 111; - - // Import initial dummy checkpoint record, otherwise the get_timeline() call - // after branching fails below - writer.put_page_image(RelishTag::Checkpoint, 0, Lsn(0x10), ZERO_CHECKPOINT.clone())?; - - // Create a relation on the timeline - writer.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?; - - writer.advance_last_record_lsn(Lsn(0x30)); - - // Check that list_rels() lists it after LSN 2, but no before it - assert!(!tline.list_rels(0, TESTDB, Lsn(0x10))?.contains(&TESTREL_A)); - assert!(tline.list_rels(0, TESTDB, Lsn(0x20))?.contains(&TESTREL_A)); - assert!(tline.list_rels(0, TESTDB, Lsn(0x30))?.contains(&TESTREL_A)); - - // Create a branch, check that the relation is visible there - repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x30))?; - let newtline = repo - .get_timeline_load(NEW_TIMELINE_ID) - .expect("Should have a local timeline"); - let new_writer = newtline.writer(); - - assert!(newtline - .list_rels(0, TESTDB, Lsn(0x30))? - .contains(&TESTREL_A)); - - // Drop it on the branch - new_writer.drop_relish(TESTREL_A, Lsn(0x40))?; - new_writer.advance_last_record_lsn(Lsn(0x40)); - - drop(new_writer); - - // Check that it's no longer listed on the branch after the point where it was dropped - assert!(newtline - .list_rels(0, TESTDB, Lsn(0x30))? - .contains(&TESTREL_A)); - assert!(!newtline - .list_rels(0, TESTDB, Lsn(0x40))? - .contains(&TESTREL_A)); - - // Run checkpoint and garbage collection and check that it's still not visible - newtline.checkpoint(CheckpointConfig::Forced)?; - repo.gc_iteration(Some(NEW_TIMELINE_ID), 0, true)?; - - assert!(!newtline - .list_rels(0, TESTDB, Lsn(0x40))? - .contains(&TESTREL_A)); - - Ok(()) + /// Convenience function to create a page image with given string as the only content + pub fn test_value(s: &str) -> Value { + let mut buf = BytesMut::new(); + buf.extend_from_slice(s.as_bytes()); + Value::Image(buf.freeze()) } /// @@ -890,21 +623,24 @@ mod tests { let repo = RepoHarness::create("test_branch")?.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; let writer = tline.writer(); + use std::str::from_utf8; - // Import initial dummy checkpoint record, otherwise the get_timeline() call - // after branching fails below - writer.put_page_image(RelishTag::Checkpoint, 0, Lsn(0x10), ZERO_CHECKPOINT.clone())?; + #[allow(non_snake_case)] + let TEST_KEY_A: Key = Key::from_hex("112222222233333333444444445500000001").unwrap(); + #[allow(non_snake_case)] + let TEST_KEY_B: Key = Key::from_hex("112222222233333333444444445500000002").unwrap(); - // Create a relation on the timeline - writer.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?; - writer.put_page_image(TESTREL_A, 0, Lsn(0x30), TEST_IMG("foo blk 0 at 3"))?; - writer.put_page_image(TESTREL_A, 0, Lsn(0x40), TEST_IMG("foo blk 0 at 4"))?; + // Insert a value on the timeline + writer.put(TEST_KEY_A, Lsn(0x20), test_value("foo at 0x20"))?; + writer.put(TEST_KEY_B, Lsn(0x20), test_value("foobar at 0x20"))?; + writer.finish_write(Lsn(0x20)); - // Create another relation - writer.put_page_image(TESTREL_B, 0, Lsn(0x20), TEST_IMG("foobar blk 0 at 2"))?; + writer.put(TEST_KEY_A, Lsn(0x30), test_value("foo at 0x30"))?; + writer.finish_write(Lsn(0x30)); + writer.put(TEST_KEY_A, Lsn(0x40), test_value("foo at 0x40"))?; + writer.finish_write(Lsn(0x40)); - writer.advance_last_record_lsn(Lsn(0x40)); - assert_current_logical_size(&tline, Lsn(0x40)); + //assert_current_logical_size(&tline, Lsn(0x40)); // Branch the history, modify relation differently on the new timeline repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x30))?; @@ -912,71 +648,65 @@ mod tests { .get_timeline_load(NEW_TIMELINE_ID) .expect("Should have a local timeline"); let new_writer = newtline.writer(); - - new_writer.put_page_image(TESTREL_A, 0, Lsn(0x40), TEST_IMG("bar blk 0 at 4"))?; - new_writer.advance_last_record_lsn(Lsn(0x40)); + new_writer.put(TEST_KEY_A, Lsn(0x40), test_value("bar at 0x40"))?; + new_writer.finish_write(Lsn(0x40)); // Check page contents on both branches assert_eq!( - tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x40))?, - TEST_IMG("foo blk 0 at 4") + from_utf8(&tline.get(TEST_KEY_A, Lsn(0x40))?)?, + "foo at 0x40" ); - assert_eq!( - newtline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x40))?, - TEST_IMG("bar blk 0 at 4") + from_utf8(&newtline.get(TEST_KEY_A, Lsn(0x40))?)?, + "bar at 0x40" ); - assert_eq!( - newtline.get_page_at_lsn(TESTREL_B, 0, Lsn(0x40))?, - TEST_IMG("foobar blk 0 at 2") + from_utf8(&newtline.get(TEST_KEY_B, Lsn(0x40))?)?, + "foobar at 0x20" ); - assert_eq!(newtline.get_relish_size(TESTREL_B, Lsn(0x40))?.unwrap(), 1); - - assert_current_logical_size(&tline, Lsn(0x40)); + //assert_current_logical_size(&tline, Lsn(0x40)); Ok(()) } - fn make_some_layers(tline: &Arc, start_lsn: Lsn) -> Result<()> { + fn make_some_layers(tline: &T, start_lsn: Lsn) -> Result<()> { let mut lsn = start_lsn; + #[allow(non_snake_case)] { let writer = tline.writer(); // Create a relation on the timeline - writer.put_page_image( - TESTREL_A, - 0, + writer.put( + *TEST_KEY, lsn, - TEST_IMG(&format!("foo blk 0 at {}", lsn)), + Value::Image(TEST_IMG(&format!("foo at {}", lsn))), )?; + writer.finish_write(lsn); lsn += 0x10; - writer.put_page_image( - TESTREL_A, - 0, + writer.put( + *TEST_KEY, lsn, - TEST_IMG(&format!("foo blk 0 at {}", lsn)), + Value::Image(TEST_IMG(&format!("foo at {}", lsn))), )?; - writer.advance_last_record_lsn(lsn); + writer.finish_write(lsn); + lsn += 0x10; } tline.checkpoint(CheckpointConfig::Forced)?; { let writer = tline.writer(); - lsn += 0x10; - writer.put_page_image( - TESTREL_A, - 0, + writer.put( + *TEST_KEY, lsn, - TEST_IMG(&format!("foo blk 0 at {}", lsn)), + Value::Image(TEST_IMG(&format!("foo at {}", lsn))), )?; + writer.finish_write(lsn); lsn += 0x10; - writer.put_page_image( - TESTREL_A, - 0, + writer.put( + *TEST_KEY, lsn, - TEST_IMG(&format!("foo blk 0 at {}", lsn)), + Value::Image(TEST_IMG(&format!("foo at {}", lsn))), )?; - writer.advance_last_record_lsn(lsn); + writer.finish_write(lsn); } tline.checkpoint(CheckpointConfig::Forced) } @@ -985,11 +715,13 @@ mod tests { fn test_prohibit_branch_creation_on_garbage_collected_data() -> Result<()> { let repo = RepoHarness::create("test_prohibit_branch_creation_on_garbage_collected_data")?.load(); - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - make_some_layers(&tline, Lsn(0x20))?; + make_some_layers(tline.as_ref(), Lsn(0x20))?; // this removes layers before lsn 40 (50 minus 10), so there are two remaining layers, image and delta for 31-50 + // FIXME: this doesn't actually remove any layer currently, given how the checkpointing + // and compaction works. But it does set the 'cutoff' point so that the cross check + // below should fail. repo.gc_iteration(Some(TIMELINE_ID), 0x10, false)?; // try to branch at lsn 25, should fail because we already garbage collected the data @@ -1029,32 +761,35 @@ mod tests { Ok(()) } + /* + // FIXME: This currently fails to error out. Calling GC doesn't currently + // remove the old value, we'd need to work a little harder #[test] - fn test_prohibit_get_page_at_lsn_for_garbage_collected_pages() -> Result<()> { + fn test_prohibit_get_for_garbage_collected_data() -> Result<()> { let repo = - RepoHarness::create("test_prohibit_get_page_at_lsn_for_garbage_collected_pages")? - .load(); + RepoHarness::create("test_prohibit_get_for_garbage_collected_data")? + .load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - make_some_layers(&tline, Lsn(0x20))?; + make_some_layers(tline.as_ref(), Lsn(0x20))?; repo.gc_iteration(Some(TIMELINE_ID), 0x10, false)?; let latest_gc_cutoff_lsn = tline.get_latest_gc_cutoff_lsn(); assert!(*latest_gc_cutoff_lsn > Lsn(0x25)); - match tline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x25)) { + match tline.get(*TEST_KEY, Lsn(0x25)) { Ok(_) => panic!("request for page should have failed"), Err(err) => assert!(err.to_string().contains("not found at")), } Ok(()) } + */ #[test] fn test_retain_data_in_parent_which_is_needed_for_child() -> Result<()> { let repo = RepoHarness::create("test_retain_data_in_parent_which_is_needed_for_child")?.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - - make_some_layers(&tline, Lsn(0x20))?; + make_some_layers(tline.as_ref(), Lsn(0x20))?; repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x40))?; let newtline = repo @@ -1062,92 +797,31 @@ mod tests { .expect("Should have a local timeline"); // this removes layers before lsn 40 (50 minus 10), so there are two remaining layers, image and delta for 31-50 repo.gc_iteration(Some(TIMELINE_ID), 0x10, false)?; - assert!(newtline.get_page_at_lsn(TESTREL_A, 0, Lsn(0x25)).is_ok()); + assert!(newtline.get(*TEST_KEY, Lsn(0x25)).is_ok()); Ok(()) } - #[test] fn test_parent_keeps_data_forever_after_branching() -> Result<()> { - let harness = RepoHarness::create("test_parent_keeps_data_forever_after_branching")?; - let repo = harness.load(); + let repo = RepoHarness::create("test_parent_keeps_data_forever_after_branching")?.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - - make_some_layers(&tline, Lsn(0x20))?; + make_some_layers(tline.as_ref(), Lsn(0x20))?; repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x40))?; let newtline = repo .get_timeline_load(NEW_TIMELINE_ID) .expect("Should have a local timeline"); - make_some_layers(&newtline, Lsn(0x60))?; + make_some_layers(newtline.as_ref(), Lsn(0x60))?; // run gc on parent repo.gc_iteration(Some(TIMELINE_ID), 0x10, false)?; - // check that the layer in parent before the branching point is still there - let tline_dir = harness.conf.timeline_path(&TIMELINE_ID, &harness.tenant_id); - - let expected_image_layer_path = tline_dir.join(format!( - "rel_{}_{}_{}_{}_{}_{:016X}_{:016X}", - TESTREL_A_REL_TAG.spcnode, - TESTREL_A_REL_TAG.dbnode, - TESTREL_A_REL_TAG.relnode, - TESTREL_A_REL_TAG.forknum, - 0, // seg is 0 - 0x20, - 0x30, - )); - assert!(fs::metadata(&expected_image_layer_path).is_ok()); - - Ok(()) - } - - #[test] - fn test_read_beyond_eof() -> Result<()> { - let harness = RepoHarness::create("test_read_beyond_eof")?; - let repo = harness.load(); - let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - - make_some_layers(&tline, Lsn(0x20))?; - { - let writer = tline.writer(); - writer.put_page_image( - TESTREL_A, - 0, - Lsn(0x60), - TEST_IMG(&format!("foo blk 0 at {}", Lsn(0x50))), - )?; - writer.advance_last_record_lsn(Lsn(0x60)); - } - - // Test read before rel creation. Should error out. - assert!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x10)).is_err()); - - // Read block beyond end of relation at different points in time. - // These reads should fall into different delta, image, and in-memory layers. - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x20))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x25))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x30))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x35))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x40))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x45))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x50))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x55))?, ZERO_PAGE); - assert_eq!(tline.get_page_at_lsn(TESTREL_A, 1, Lsn(0x60))?, ZERO_PAGE); - - // Test on an in-memory layer with no preceding layer - { - let writer = tline.writer(); - writer.put_page_image( - TESTREL_B, - 0, - Lsn(0x70), - TEST_IMG(&format!("foo blk 0 at {}", Lsn(0x70))), - )?; - writer.advance_last_record_lsn(Lsn(0x70)); - } - assert_eq!(tline.get_page_at_lsn(TESTREL_B, 1, Lsn(0x70))?, ZERO_PAGE); + // Check that the data is still accessible on the branch. + assert_eq!( + newtline.get(*TEST_KEY, Lsn(0x50))?, + TEST_IMG(&format!("foo at {}", Lsn(0x40))) + ); Ok(()) } @@ -1159,7 +833,7 @@ mod tests { { let repo = harness.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0x8000))?; - make_some_layers(&tline, Lsn(0x8000))?; + make_some_layers(tline.as_ref(), Lsn(0x8000))?; tline.checkpoint(CheckpointConfig::Forced)?; } @@ -1188,7 +862,7 @@ mod tests { let repo = harness.load(); let tline = repo.create_empty_timeline(TIMELINE_ID, Lsn(0))?; - make_some_layers(&tline, Lsn(0x20))?; + make_some_layers(tline.as_ref(), Lsn(0x20))?; tline.checkpoint(CheckpointConfig::Forced)?; repo.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Lsn(0x40))?; @@ -1197,7 +871,7 @@ mod tests { .get_timeline_load(NEW_TIMELINE_ID) .expect("Should have a local timeline"); - make_some_layers(&newtline, Lsn(0x60))?; + make_some_layers(newtline.as_ref(), Lsn(0x60))?; tline.checkpoint(CheckpointConfig::Forced)?; } diff --git a/pageserver/src/tenant_mgr.rs b/pageserver/src/tenant_mgr.rs index e7cc4ecbaf..aeff718803 100644 --- a/pageserver/src/tenant_mgr.rs +++ b/pageserver/src/tenant_mgr.rs @@ -4,13 +4,13 @@ use crate::config::PageServerConf; use crate::layered_repository::LayeredRepository; use crate::remote_storage::RemoteIndex; -use crate::repository::{Repository, Timeline, TimelineSyncStatusUpdate}; +use crate::repository::{Repository, TimelineSyncStatusUpdate}; use crate::thread_mgr; use crate::thread_mgr::ThreadKind; use crate::timelines; use crate::timelines::CreateRepo; use crate::walredo::PostgresRedoManager; -use crate::CheckpointConfig; +use crate::{DatadirTimelineImpl, RepositoryImpl}; use anyhow::{Context, Result}; use lazy_static::lazy_static; use log::*; @@ -28,7 +28,9 @@ lazy_static! { struct Tenant { state: TenantState, - repo: Arc, + repo: Arc, + + timelines: HashMap>, } #[derive(Debug, Serialize, Deserialize, Clone, Copy, PartialEq, Eq)] @@ -67,14 +69,14 @@ pub fn load_local_repo( conf: &'static PageServerConf, tenant_id: ZTenantId, remote_index: &RemoteIndex, -) -> Arc { +) -> Arc { let mut m = access_tenants(); let tenant = m.entry(tenant_id).or_insert_with(|| { // Set up a WAL redo manager, for applying WAL records. let walredo_mgr = PostgresRedoManager::new(conf, tenant_id); // Set up an object repository, for actual data storage. - let repo: Arc = Arc::new(LayeredRepository::new( + let repo: Arc = Arc::new(LayeredRepository::new( conf, Arc::new(walredo_mgr), tenant_id, @@ -84,6 +86,7 @@ pub fn load_local_repo( Tenant { state: TenantState::Idle, repo, + timelines: HashMap::new(), } }); Arc::clone(&tenant.repo) @@ -138,7 +141,7 @@ pub fn shutdown_all_tenants() { thread_mgr::shutdown_threads(Some(ThreadKind::WalReceiver), None, None); thread_mgr::shutdown_threads(Some(ThreadKind::GarbageCollector), None, None); - thread_mgr::shutdown_threads(Some(ThreadKind::Checkpointer), None, None); + thread_mgr::shutdown_threads(Some(ThreadKind::Compactor), None, None); // Ok, no background threads running anymore. Flush any remaining data in // memory to disk. @@ -152,7 +155,7 @@ pub fn shutdown_all_tenants() { debug!("shutdown tenant {}", tenantid); match get_repository_for_tenant(tenantid) { Ok(repo) => { - if let Err(err) = repo.checkpoint_iteration(CheckpointConfig::Flush) { + if let Err(err) = repo.checkpoint() { error!( "Could not checkpoint tenant {} during shutdown: {:?}", tenantid, err @@ -192,6 +195,7 @@ pub fn create_tenant_repository( v.insert(Tenant { state: TenantState::Idle, repo, + timelines: HashMap::new(), }); Ok(Some(tenantid)) } @@ -203,7 +207,7 @@ pub fn get_tenant_state(tenantid: ZTenantId) -> Option { } /// -/// Change the state of a tenant to Active and launch its checkpointer and GC +/// Change the state of a tenant to Active and launch its compactor and GC /// threads. If the tenant was already in Active state or Stopping, does nothing. /// pub fn activate_tenant(conf: &'static PageServerConf, tenant_id: ZTenantId) -> Result<()> { @@ -218,15 +222,15 @@ pub fn activate_tenant(conf: &'static PageServerConf, tenant_id: ZTenantId) -> R // If the tenant is already active, nothing to do. TenantState::Active => {} - // If it's Idle, launch the checkpointer and GC threads + // If it's Idle, launch the compactor and GC threads TenantState::Idle => { thread_mgr::spawn( - ThreadKind::Checkpointer, + ThreadKind::Compactor, Some(tenant_id), None, - "Checkpointer thread", + "Compactor thread", true, - move || crate::tenant_threads::checkpoint_loop(tenant_id, conf), + move || crate::tenant_threads::compact_loop(tenant_id, conf), )?; let gc_spawn_result = thread_mgr::spawn( @@ -244,7 +248,7 @@ pub fn activate_tenant(conf: &'static PageServerConf, tenant_id: ZTenantId) -> R "Failed to start GC thread for tenant {}, stopping its checkpointer thread: {:?}", tenant_id, e ); - thread_mgr::shutdown_threads(Some(ThreadKind::Checkpointer), Some(tenant_id), None); + thread_mgr::shutdown_threads(Some(ThreadKind::Compactor), Some(tenant_id), None); return gc_spawn_result; } @@ -258,7 +262,7 @@ pub fn activate_tenant(conf: &'static PageServerConf, tenant_id: ZTenantId) -> R Ok(()) } -pub fn get_repository_for_tenant(tenantid: ZTenantId) -> Result> { +pub fn get_repository_for_tenant(tenantid: ZTenantId) -> Result> { let m = access_tenants(); let tenant = m .get(&tenantid) @@ -271,10 +275,27 @@ pub fn get_repository_for_tenant(tenantid: ZTenantId) -> Result Result> { - get_repository_for_tenant(tenantid)? +) -> Result> { + let mut m = access_tenants(); + let tenant = m + .get_mut(&tenantid) + .with_context(|| format!("Tenant {} not found", tenantid))?; + + if let Some(page_tline) = tenant.timelines.get(&timelineid) { + return Ok(Arc::clone(page_tline)); + } + // First access to this timeline. Create a DatadirTimeline wrapper for it + let tline = tenant + .repo .get_timeline_load(timelineid) - .with_context(|| format!("Timeline {} not found for tenant {}", timelineid, tenantid)) + .with_context(|| format!("Timeline {} not found for tenant {}", timelineid, tenantid))?; + + let repartition_distance = tenant.repo.conf.checkpoint_distance / 10; + + let page_tline = Arc::new(DatadirTimelineImpl::new(tline, repartition_distance)); + page_tline.init_logical_size()?; + tenant.timelines.insert(timelineid, Arc::clone(&page_tline)); + Ok(page_tline) } #[serde_as] diff --git a/pageserver/src/tenant_threads.rs b/pageserver/src/tenant_threads.rs index c370eb61c8..0d9a94cc5b 100644 --- a/pageserver/src/tenant_threads.rs +++ b/pageserver/src/tenant_threads.rs @@ -1,34 +1,42 @@ //! This module contains functions to serve per-tenant background processes, -//! such as checkpointer and GC +//! such as compaction and GC use crate::config::PageServerConf; +use crate::repository::Repository; use crate::tenant_mgr; use crate::tenant_mgr::TenantState; -use crate::CheckpointConfig; use anyhow::Result; use std::time::Duration; use tracing::*; use zenith_utils::zid::ZTenantId; /// -/// Checkpointer thread's main loop +/// Compaction thread's main loop /// -pub fn checkpoint_loop(tenantid: ZTenantId, conf: &'static PageServerConf) -> Result<()> { +pub fn compact_loop(tenantid: ZTenantId, conf: &'static PageServerConf) -> Result<()> { + if let Err(err) = compact_loop_ext(tenantid, conf) { + error!("compact loop terminated with error: {:?}", err); + Err(err) + } else { + Ok(()) + } +} + +fn compact_loop_ext(tenantid: ZTenantId, conf: &'static PageServerConf) -> Result<()> { loop { if tenant_mgr::get_tenant_state(tenantid) != Some(TenantState::Active) { break; } - std::thread::sleep(conf.checkpoint_period); - trace!("checkpointer thread for tenant {} waking up", tenantid); + std::thread::sleep(conf.compaction_period); + trace!("compaction thread for tenant {} waking up", tenantid); - // checkpoint timelines that have accumulated more than CHECKPOINT_DISTANCE - // bytes of WAL since last checkpoint. + // Compact timelines let repo = tenant_mgr::get_repository_for_tenant(tenantid)?; - repo.checkpoint_iteration(CheckpointConfig::Distance(conf.checkpoint_distance))?; + repo.compaction_iteration()?; } trace!( - "checkpointer thread stopped for tenant {} state is {:?}", + "compaction thread stopped for tenant {} state is {:?}", tenantid, tenant_mgr::get_tenant_state(tenantid) ); diff --git a/pageserver/src/thread_mgr.rs b/pageserver/src/thread_mgr.rs index cafdc5e700..4484bb1db1 100644 --- a/pageserver/src/thread_mgr.rs +++ b/pageserver/src/thread_mgr.rs @@ -94,13 +94,16 @@ pub enum ThreadKind { // Thread that connects to a safekeeper to fetch WAL for one timeline. WalReceiver, - // Thread that handles checkpointing of all timelines for a tenant. - Checkpointer, + // Thread that handles compaction of all timelines for a tenant. + Compactor, // Thread that handles GC of a tenant GarbageCollector, - // Thread for synchronizing pageserver relish data with the remote storage. + // Thread that flushes frozen in-memory layers to disk + LayerFlushThread, + + // Thread for synchronizing pageserver layer files with the remote storage. // Shared by all tenants. StorageSync, } diff --git a/pageserver/src/timelines.rs b/pageserver/src/timelines.rs index 53c4124701..105c3c869f 100644 --- a/pageserver/src/timelines.rs +++ b/pageserver/src/timelines.rs @@ -23,6 +23,7 @@ use crate::{ layered_repository::metadata::TimelineMetadata, remote_storage::RemoteIndex, repository::{LocalTimelineState, Repository}, + DatadirTimeline, RepositoryImpl, }; use crate::{import_datadir, LOG_FILE_NAME}; use crate::{layered_repository::LayeredRepository, walredo::WalRedoManager}; @@ -48,26 +49,26 @@ pub struct LocalTimelineInfo { } impl LocalTimelineInfo { - pub fn from_loaded_timeline( - timeline: &dyn Timeline, + pub fn from_loaded_timeline( + datadir_tline: &DatadirTimeline, include_non_incremental_logical_size: bool, ) -> anyhow::Result { - let last_record_lsn = timeline.get_last_record_lsn(); + let last_record_lsn = datadir_tline.tline.get_last_record_lsn(); let info = LocalTimelineInfo { - ancestor_timeline_id: timeline.get_ancestor_timeline_id(), + ancestor_timeline_id: datadir_tline.tline.get_ancestor_timeline_id(), ancestor_lsn: { - match timeline.get_ancestor_lsn() { + match datadir_tline.tline.get_ancestor_lsn() { Lsn(0) => None, lsn @ Lsn(_) => Some(lsn), } }, - disk_consistent_lsn: timeline.get_disk_consistent_lsn(), + disk_consistent_lsn: datadir_tline.tline.get_disk_consistent_lsn(), last_record_lsn, - prev_record_lsn: Some(timeline.get_prev_record_lsn()), + prev_record_lsn: Some(datadir_tline.tline.get_prev_record_lsn()), timeline_state: LocalTimelineState::Loaded, - current_logical_size: Some(timeline.get_current_logical_size()), + current_logical_size: Some(datadir_tline.get_current_logical_size()), current_logical_size_non_incremental: if include_non_incremental_logical_size { - Some(timeline.get_current_logical_size_non_incremental(last_record_lsn)?) + Some(datadir_tline.get_current_logical_size_non_incremental(last_record_lsn)?) } else { None }, @@ -93,17 +94,19 @@ impl LocalTimelineInfo { } } - pub fn from_repo_timeline( - repo_timeline: RepositoryTimeline, + pub fn from_repo_timeline( + tenant_id: ZTenantId, + timeline_id: ZTimelineId, + repo_timeline: &RepositoryTimeline, include_non_incremental_logical_size: bool, ) -> anyhow::Result { match repo_timeline { - RepositoryTimeline::Loaded(timeline) => { - Self::from_loaded_timeline(timeline.as_ref(), include_non_incremental_logical_size) - } - RepositoryTimeline::Unloaded { metadata } => { - Ok(Self::from_unloaded_timeline(&metadata)) + RepositoryTimeline::Loaded(_) => { + let datadir_tline = + tenant_mgr::get_timeline_for_tenant_load(tenant_id, timeline_id)?; + Self::from_loaded_timeline(&datadir_tline, include_non_incremental_logical_size) } + RepositoryTimeline::Unloaded { metadata } => Ok(Self::from_unloaded_timeline(metadata)), } } } @@ -172,7 +175,7 @@ pub fn create_repo( conf: &'static PageServerConf, tenant_id: ZTenantId, create_repo: CreateRepo, -) -> Result> { +) -> Result> { let (wal_redo_manager, remote_index) = match create_repo { CreateRepo::Real { wal_redo_manager, @@ -260,12 +263,12 @@ fn run_initdb(conf: &'static PageServerConf, initdbpath: &Path) -> Result<()> { // - run initdb to init temporary instance and get bootstrap data // - after initialization complete, remove the temp dir. // -fn bootstrap_timeline( +fn bootstrap_timeline( conf: &'static PageServerConf, tenantid: ZTenantId, tli: ZTimelineId, - repo: &dyn Repository, -) -> Result> { + repo: &R, +) -> Result<()> { let _enter = info_span!("bootstrapping", timeline = %tli, tenant = %tenantid).entered(); let initdb_path = conf.tenant_path(&tenantid).join("tmp"); @@ -281,23 +284,20 @@ fn bootstrap_timeline( // Initdb lsn will be equal to last_record_lsn which will be set after import. // Because we know it upfront avoid having an option or dummy zero value by passing it to create_empty_timeline. let timeline = repo.create_empty_timeline(tli, lsn)?; - import_datadir::import_timeline_from_postgres_datadir( - &pgdata_path, - timeline.writer().as_ref(), - lsn, - )?; - timeline.checkpoint(CheckpointConfig::Forced)?; + let mut page_tline: DatadirTimeline = DatadirTimeline::new(timeline, u64::MAX); + import_datadir::import_timeline_from_postgres_datadir(&pgdata_path, &mut page_tline, lsn)?; + page_tline.tline.checkpoint(CheckpointConfig::Forced)?; println!( "created initial timeline {} timeline.lsn {}", tli, - timeline.get_last_record_lsn() + page_tline.tline.get_last_record_lsn() ); // Remove temp dir. We don't need it anymore fs::remove_dir_all(pgdata_path)?; - Ok(timeline) + Ok(()) } pub(crate) fn get_local_timelines( @@ -313,7 +313,9 @@ pub(crate) fn get_local_timelines( local_timeline_info.push(( timeline_id, LocalTimelineInfo::from_repo_timeline( - repository_timeline, + tenant_id, + timeline_id, + &repository_timeline, include_non_incremental_logical_size, )?, )) @@ -372,13 +374,17 @@ pub(crate) fn create_timeline( } repo.branch_timeline(ancestor_timeline_id, new_timeline_id, start_lsn)?; // load the timeline into memory - let loaded_timeline = repo.get_timeline_load(new_timeline_id)?; - LocalTimelineInfo::from_loaded_timeline(loaded_timeline.as_ref(), false) + let loaded_timeline = + tenant_mgr::get_timeline_for_tenant_load(tenant_id, new_timeline_id)?; + LocalTimelineInfo::from_loaded_timeline(&loaded_timeline, false) .context("cannot fill timeline info")? } None => { - let new_timeline = bootstrap_timeline(conf, tenant_id, new_timeline_id, repo.as_ref())?; - LocalTimelineInfo::from_loaded_timeline(new_timeline.as_ref(), false) + bootstrap_timeline(conf, tenant_id, new_timeline_id, repo.as_ref())?; + // load the timeline into memory + let new_timeline = + tenant_mgr::get_timeline_for_tenant_load(tenant_id, new_timeline_id)?; + LocalTimelineInfo::from_loaded_timeline(&new_timeline, false) .context("cannot fill timeline info")? } }; diff --git a/pageserver/src/walingest.rs b/pageserver/src/walingest.rs index 506890476f..c6c6e89854 100644 --- a/pageserver/src/walingest.rs +++ b/pageserver/src/walingest.rs @@ -23,14 +23,16 @@ use postgres_ffi::nonrelfile_utils::clogpage_precedes; use postgres_ffi::nonrelfile_utils::slru_may_delete_clogsegment; -use std::cmp::min; use anyhow::Result; use bytes::{Buf, Bytes, BytesMut}; use tracing::*; -use crate::relish::*; -use crate::repository::*; +use std::collections::HashMap; + +use crate::pgdatadir_mapping::*; +use crate::reltag::{RelTag, SlruKind}; +use crate::repository::Repository; use crate::walrecord::*; use postgres_ffi::nonrelfile_utils::mx_offset_to_member_segment; use postgres_ffi::xlog_utils::*; @@ -40,22 +42,28 @@ use zenith_utils::lsn::Lsn; static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; 8192]); -pub struct WalIngest { +pub struct WalIngest<'a, R: Repository> { + timeline: &'a DatadirTimeline, + checkpoint: CheckPoint, checkpoint_modified: bool, + + relsize_cache: HashMap, } -impl WalIngest { - pub fn new(timeline: &dyn Timeline, startpoint: Lsn) -> Result { +impl<'a, R: Repository> WalIngest<'a, R> { + pub fn new(timeline: &DatadirTimeline, startpoint: Lsn) -> Result> { // Fetch the latest checkpoint into memory, so that we can compare with it // quickly in `ingest_record` and update it when it changes. - let checkpoint_bytes = timeline.get_page_at_lsn(RelishTag::Checkpoint, 0, startpoint)?; + let checkpoint_bytes = timeline.get_checkpoint(startpoint)?; let checkpoint = CheckPoint::decode(&checkpoint_bytes)?; trace!("CheckPoint.nextXid = {}", checkpoint.nextXid.value); Ok(WalIngest { + timeline, checkpoint, checkpoint_modified: false, + relsize_cache: HashMap::new(), }) } @@ -68,10 +76,12 @@ impl WalIngest { /// pub fn ingest_record( &mut self, - timeline: &dyn TimelineWriter, + timeline: &DatadirTimeline, recdata: Bytes, lsn: Lsn, ) -> Result<()> { + let mut modification = timeline.begin_modification(lsn); + let mut decoded = decode_wal_record(recdata); let mut buf = decoded.record.clone(); buf.advance(decoded.main_data_offset); @@ -86,48 +96,34 @@ impl WalIngest { if decoded.xl_rmid == pg_constants::RM_HEAP_ID || decoded.xl_rmid == pg_constants::RM_HEAP2_ID { - self.ingest_heapam_record(&mut buf, timeline, lsn, &mut decoded)?; + self.ingest_heapam_record(&mut buf, &mut modification, &mut decoded)?; } // Handle other special record types if decoded.xl_rmid == pg_constants::RM_SMGR_ID + && (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) + == pg_constants::XLOG_SMGR_CREATE + { + let create = XlSmgrCreate::decode(&mut buf); + self.ingest_xlog_smgr_create(&mut modification, &create)?; + } else if decoded.xl_rmid == pg_constants::RM_SMGR_ID && (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == pg_constants::XLOG_SMGR_TRUNCATE { let truncate = XlSmgrTruncate::decode(&mut buf); - self.ingest_xlog_smgr_truncate(timeline, lsn, &truncate)?; + self.ingest_xlog_smgr_truncate(&mut modification, &truncate)?; } else if decoded.xl_rmid == pg_constants::RM_DBASE_ID { if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == pg_constants::XLOG_DBASE_CREATE { let createdb = XlCreateDatabase::decode(&mut buf); - self.ingest_xlog_dbase_create(timeline, lsn, &createdb)?; + self.ingest_xlog_dbase_create(&mut modification, &createdb)?; } else if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == pg_constants::XLOG_DBASE_DROP { let dropdb = XlDropDatabase::decode(&mut buf); - - // To drop the database, we need to drop all the relations in it. Like in - // ingest_xlog_dbase_create(), use the previous record's LSN in the list_rels() call - let req_lsn = min(timeline.get_last_record_lsn(), lsn); - for tablespace_id in dropdb.tablespace_ids { - let rels = timeline.list_rels(tablespace_id, dropdb.db_id, req_lsn)?; - for rel in rels { - timeline.drop_relish(rel, lsn)?; - } - trace!( - "Drop FileNodeMap {}, {} at lsn {}", - tablespace_id, - dropdb.db_id, - lsn - ); - timeline.drop_relish( - RelishTag::FileNodeMap { - spcnode: tablespace_id, - dbnode: dropdb.db_id, - }, - lsn, - )?; + trace!("Drop db {}, {}", tablespace_id, dropdb.db_id); + modification.drop_dbdir(tablespace_id, dropdb.db_id)?; } } } else if decoded.xl_rmid == pg_constants::RM_TBLSPC_ID { @@ -138,19 +134,17 @@ impl WalIngest { let pageno = buf.get_u32_le(); let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; - timeline.put_page_image( - RelishTag::Slru { - slru: SlruKind::Clog, - segno, - }, + self.put_slru_page_image( + &mut modification, + SlruKind::Clog, + segno, rpageno, - lsn, ZERO_PAGE.clone(), )?; } else { assert!(info == pg_constants::CLOG_TRUNCATE); let xlrec = XlClogTruncate::decode(&mut buf); - self.ingest_clog_truncate_record(timeline, lsn, &xlrec)?; + self.ingest_clog_truncate_record(&mut modification, &xlrec)?; } } else if decoded.xl_rmid == pg_constants::RM_XACT_ID { let info = decoded.xl_info & pg_constants::XLOG_XACT_OPMASK; @@ -158,8 +152,7 @@ impl WalIngest { let parsed_xact = XlXactParsedRecord::decode(&mut buf, decoded.xl_xid, decoded.xl_info); self.ingest_xact_record( - timeline, - lsn, + &mut modification, &parsed_xact, info == pg_constants::XLOG_XACT_COMMIT, )?; @@ -169,8 +162,7 @@ impl WalIngest { let parsed_xact = XlXactParsedRecord::decode(&mut buf, decoded.xl_xid, decoded.xl_info); self.ingest_xact_record( - timeline, - lsn, + &mut modification, &parsed_xact, info == pg_constants::XLOG_XACT_COMMIT_PREPARED, )?; @@ -179,23 +171,11 @@ impl WalIngest { "Drop twophaseFile for xid {} parsed_xact.xid {} here at {}", decoded.xl_xid, parsed_xact.xid, - lsn + lsn, ); - timeline.drop_relish( - RelishTag::TwoPhase { - xid: parsed_xact.xid, - }, - lsn, - )?; + modification.drop_twophase_file(parsed_xact.xid)?; } else if info == pg_constants::XLOG_XACT_PREPARE { - timeline.put_page_image( - RelishTag::TwoPhase { - xid: decoded.xl_xid, - }, - 0, - lsn, - Bytes::copy_from_slice(&buf[..]), - )?; + modification.put_twophase_file(decoded.xl_xid, Bytes::copy_from_slice(&buf[..]))?; } } else if decoded.xl_rmid == pg_constants::RM_MULTIXACT_ID { let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK; @@ -204,38 +184,34 @@ impl WalIngest { let pageno = buf.get_u32_le(); let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; - timeline.put_page_image( - RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno, - }, + self.put_slru_page_image( + &mut modification, + SlruKind::MultiXactOffsets, + segno, rpageno, - lsn, ZERO_PAGE.clone(), )?; } else if info == pg_constants::XLOG_MULTIXACT_ZERO_MEM_PAGE { let pageno = buf.get_u32_le(); let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; - timeline.put_page_image( - RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno, - }, + self.put_slru_page_image( + &mut modification, + SlruKind::MultiXactMembers, + segno, rpageno, - lsn, ZERO_PAGE.clone(), )?; } else if info == pg_constants::XLOG_MULTIXACT_CREATE_ID { let xlrec = XlMultiXactCreate::decode(&mut buf); - self.ingest_multixact_create_record(timeline, lsn, &xlrec)?; + self.ingest_multixact_create_record(&mut modification, &xlrec)?; } else if info == pg_constants::XLOG_MULTIXACT_TRUNCATE_ID { let xlrec = XlMultiXactTruncate::decode(&mut buf); - self.ingest_multixact_truncate_record(timeline, lsn, &xlrec)?; + self.ingest_multixact_truncate_record(&mut modification, &xlrec)?; } } else if decoded.xl_rmid == pg_constants::RM_RELMAP_ID { let xlrec = XlRelmapUpdate::decode(&mut buf); - self.ingest_relmap_page(timeline, lsn, &xlrec, &decoded)?; + self.ingest_relmap_page(&mut modification, &xlrec, &decoded)?; } else if decoded.xl_rmid == pg_constants::RM_XLOG_ID { let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK; if info == pg_constants::XLOG_NEXTOID { @@ -270,37 +246,37 @@ impl WalIngest { // Iterate through all the blocks that the record modifies, and // "put" a separate copy of the record for each block. for blk in decoded.blocks.iter() { - self.ingest_decoded_block(timeline, lsn, &decoded, blk)?; + self.ingest_decoded_block(&mut modification, lsn, &decoded, blk)?; } // If checkpoint data was updated, store the new version in the repository if self.checkpoint_modified { let new_checkpoint_bytes = self.checkpoint.encode(); - timeline.put_page_image(RelishTag::Checkpoint, 0, lsn, new_checkpoint_bytes)?; + modification.put_checkpoint(new_checkpoint_bytes)?; self.checkpoint_modified = false; } // Now that this record has been fully handled, including updating the // checkpoint data, let the repository know that it is up-to-date to this LSN - timeline.advance_last_record_lsn(lsn); + modification.commit()?; Ok(()) } fn ingest_decoded_block( &mut self, - timeline: &dyn TimelineWriter, + modification: &mut DatadirModification, lsn: Lsn, decoded: &DecodedWALRecord, blk: &DecodedBkpBlock, ) -> Result<()> { - let tag = RelishTag::Relation(RelTag { + let rel = RelTag { spcnode: blk.rnode_spcnode, dbnode: blk.rnode_dbnode, relnode: blk.rnode_relnode, forknum: blk.forknum as u8, - }); + }; // // Instead of storing full-page-image WAL record, @@ -330,13 +306,13 @@ impl WalIngest { image[0..4].copy_from_slice(&((lsn.0 >> 32) as u32).to_le_bytes()); image[4..8].copy_from_slice(&(lsn.0 as u32).to_le_bytes()); assert_eq!(image.len(), pg_constants::BLCKSZ as usize); - timeline.put_page_image(tag, blk.blkno, lsn, image.freeze())?; + self.put_rel_page_image(modification, rel, blk.blkno, image.freeze())?; } else { let rec = ZenithWalRecord::Postgres { will_init: blk.will_init || blk.apply_image, rec: decoded.record.clone(), }; - timeline.put_wal_record(lsn, tag, blk.blkno, rec)?; + self.put_rel_wal_record(modification, rel, blk.blkno, rec)?; } Ok(()) } @@ -344,8 +320,7 @@ impl WalIngest { fn ingest_heapam_record( &mut self, buf: &mut Bytes, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, decoded: &mut DecodedWALRecord, ) -> Result<()> { // Handle VM bit updates that are implicitly part of heap records. @@ -409,54 +384,76 @@ impl WalIngest { // Clear the VM bits if required. if new_heap_blkno.is_some() || old_heap_blkno.is_some() { - let vm_relish = RelishTag::Relation(RelTag { + let vm_rel = RelTag { forknum: pg_constants::VISIBILITYMAP_FORKNUM, spcnode: decoded.blocks[0].rnode_spcnode, dbnode: decoded.blocks[0].rnode_dbnode, relnode: decoded.blocks[0].rnode_relnode, - }); + }; - let new_vm_blk = new_heap_blkno.map(pg_constants::HEAPBLK_TO_MAPBLOCK); - let old_vm_blk = old_heap_blkno.map(pg_constants::HEAPBLK_TO_MAPBLOCK); - if new_vm_blk == old_vm_blk { - // An UPDATE record that needs to clear the bits for both old and the - // new page, both of which reside on the same VM page. - timeline.put_wal_record( - lsn, - vm_relish, - new_vm_blk.unwrap(), - ZenithWalRecord::ClearVisibilityMapFlags { - new_heap_blkno, - old_heap_blkno, - flags: pg_constants::VISIBILITYMAP_VALID_BITS, - }, - )?; - } else { - // Clear VM bits for one heap page, or for two pages that reside on - // different VM pages. - if let Some(new_vm_blk) = new_vm_blk { - timeline.put_wal_record( - lsn, - vm_relish, - new_vm_blk, + let mut new_vm_blk = new_heap_blkno.map(pg_constants::HEAPBLK_TO_MAPBLOCK); + let mut old_vm_blk = old_heap_blkno.map(pg_constants::HEAPBLK_TO_MAPBLOCK); + + // Sometimes, Postgres seems to create heap WAL records with the + // ALL_VISIBLE_CLEARED flag set, even though the bit in the VM page is + // not set. In fact, it's possible that the VM page does not exist at all. + // In that case, we don't want to store a record to clear the VM bit; + // replaying it would fail to find the previous image of the page, because + // it doesn't exist. So check if the VM page(s) exist, and skip the WAL + // record if it doesn't. + let vm_size = self.get_relsize(vm_rel)?; + if let Some(blknum) = new_vm_blk { + if blknum >= vm_size { + new_vm_blk = None; + } + } + if let Some(blknum) = old_vm_blk { + if blknum >= vm_size { + old_vm_blk = None; + } + } + + if new_vm_blk.is_some() || old_vm_blk.is_some() { + if new_vm_blk == old_vm_blk { + // An UPDATE record that needs to clear the bits for both old and the + // new page, both of which reside on the same VM page. + self.put_rel_wal_record( + modification, + vm_rel, + new_vm_blk.unwrap(), ZenithWalRecord::ClearVisibilityMapFlags { new_heap_blkno, - old_heap_blkno: None, - flags: pg_constants::VISIBILITYMAP_VALID_BITS, - }, - )?; - } - if let Some(old_vm_blk) = old_vm_blk { - timeline.put_wal_record( - lsn, - vm_relish, - old_vm_blk, - ZenithWalRecord::ClearVisibilityMapFlags { - new_heap_blkno: None, old_heap_blkno, flags: pg_constants::VISIBILITYMAP_VALID_BITS, }, )?; + } else { + // Clear VM bits for one heap page, or for two pages that reside on + // different VM pages. + if let Some(new_vm_blk) = new_vm_blk { + self.put_rel_wal_record( + modification, + vm_rel, + new_vm_blk, + ZenithWalRecord::ClearVisibilityMapFlags { + new_heap_blkno, + old_heap_blkno: None, + flags: pg_constants::VISIBILITYMAP_VALID_BITS, + }, + )?; + } + if let Some(old_vm_blk) = old_vm_blk { + self.put_rel_wal_record( + modification, + vm_rel, + old_vm_blk, + ZenithWalRecord::ClearVisibilityMapFlags { + new_heap_blkno: None, + old_heap_blkno, + flags: pg_constants::VISIBILITYMAP_VALID_BITS, + }, + )?; + } } } } @@ -467,8 +464,7 @@ impl WalIngest { /// Subroutine of ingest_record(), to handle an XLOG_DBASE_CREATE record. fn ingest_xlog_dbase_create( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, rec: &XlCreateDatabase, ) -> Result<()> { let db_id = rec.db_id; @@ -481,76 +477,79 @@ impl WalIngest { // cannot pass 'lsn' to the Timeline.get_* functions, or they will block waiting for // the last valid LSN to advance up to it. So we use the previous record's LSN in the // get calls instead. - let req_lsn = min(timeline.get_last_record_lsn(), lsn); + let req_lsn = modification.tline.get_last_record_lsn(); - let rels = timeline.list_rels(src_tablespace_id, src_db_id, req_lsn)?; + let rels = modification + .tline + .list_rels(src_tablespace_id, src_db_id, req_lsn)?; - trace!("ingest_xlog_dbase_create: {} rels", rels.len()); + debug!("ingest_xlog_dbase_create: {} rels", rels.len()); + + // Copy relfilemap + let filemap = modification + .tline + .get_relmap_file(src_tablespace_id, src_db_id, req_lsn)?; + modification.put_relmap_file(tablespace_id, db_id, filemap)?; let mut num_rels_copied = 0; let mut num_blocks_copied = 0; - for rel in rels { - if let RelishTag::Relation(src_rel) = rel { - assert_eq!(src_rel.spcnode, src_tablespace_id); - assert_eq!(src_rel.dbnode, src_db_id); + for src_rel in rels { + assert_eq!(src_rel.spcnode, src_tablespace_id); + assert_eq!(src_rel.dbnode, src_db_id); - let nblocks = timeline.get_relish_size(rel, req_lsn)?.unwrap_or(0); - let dst_rel = RelTag { - spcnode: tablespace_id, - dbnode: db_id, - relnode: src_rel.relnode, - forknum: src_rel.forknum, - }; + let nblocks = modification.tline.get_rel_size(src_rel, req_lsn)?; + let dst_rel = RelTag { + spcnode: tablespace_id, + dbnode: db_id, + relnode: src_rel.relnode, + forknum: src_rel.forknum, + }; - // Copy content - for blknum in 0..nblocks { - let content = timeline.get_page_at_lsn(rel, blknum, req_lsn)?; + modification.put_rel_creation(dst_rel, nblocks)?; - debug!("copying block {} from {} to {}", blknum, src_rel, dst_rel); + // Copy content + debug!("copying rel {} to {}, {} blocks", src_rel, dst_rel, nblocks); + for blknum in 0..nblocks { + debug!("copying block {} from {} to {}", blknum, src_rel, dst_rel); - timeline.put_page_image(RelishTag::Relation(dst_rel), blknum, lsn, content)?; - num_blocks_copied += 1; - } - - if nblocks == 0 { - // make sure we have some trace of the relation, even if it's empty - timeline.put_truncation(RelishTag::Relation(dst_rel), lsn, 0)?; - } - - num_rels_copied += 1; + let content = modification + .tline + .get_rel_page_at_lsn(src_rel, blknum, req_lsn)?; + modification.put_rel_page_image(dst_rel, blknum, content)?; + num_blocks_copied += 1; } + + num_rels_copied += 1; } - // Copy relfilemap - // TODO This implementation is very inefficient - - // it scans all non-rels only to find FileNodeMaps - for tag in timeline.list_nonrels(req_lsn)? { - if let RelishTag::FileNodeMap { spcnode, dbnode } = tag { - if spcnode == src_tablespace_id && dbnode == src_db_id { - let img = timeline.get_page_at_lsn(tag, 0, req_lsn)?; - let new_tag = RelishTag::FileNodeMap { - spcnode: tablespace_id, - dbnode: db_id, - }; - timeline.put_page_image(new_tag, 0, lsn, img)?; - break; - } - } - } info!( - "Created database {}/{}, copied {} blocks in {} rels at {}", - tablespace_id, db_id, num_blocks_copied, num_rels_copied, lsn + "Created database {}/{}, copied {} blocks in {} rels", + tablespace_id, db_id, num_blocks_copied, num_rels_copied ); Ok(()) } + fn ingest_xlog_smgr_create( + &mut self, + modification: &mut DatadirModification, + rec: &XlSmgrCreate, + ) -> Result<()> { + let rel = RelTag { + spcnode: rec.rnode.spcnode, + dbnode: rec.rnode.dbnode, + relnode: rec.rnode.relnode, + forknum: rec.forknum, + }; + self.put_rel_creation(modification, rel)?; + Ok(()) + } + /// Subroutine of ingest_record(), to handle an XLOG_SMGR_TRUNCATE record. /// /// This is the same logic as in PostgreSQL's smgr_redo() function. fn ingest_xlog_smgr_truncate( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, rec: &XlSmgrTruncate, ) -> Result<()> { let spcnode = rec.rnode.spcnode; @@ -564,7 +563,7 @@ impl WalIngest { relnode, forknum: pg_constants::MAIN_FORKNUM, }; - timeline.put_truncation(RelishTag::Relation(rel), lsn, rec.blkno)?; + self.put_rel_truncation(modification, rel, rec.blkno)?; } if (rec.flags & pg_constants::SMGR_TRUNCATE_FSM) != 0 { let rel = RelTag { @@ -587,7 +586,7 @@ impl WalIngest { info!("Partial truncation of FSM is not supported"); } let num_fsm_blocks = 0; - timeline.put_truncation(RelishTag::Relation(rel), lsn, num_fsm_blocks)?; + self.put_rel_truncation(modification, rel, num_fsm_blocks)?; } if (rec.flags & pg_constants::SMGR_TRUNCATE_VM) != 0 { let rel = RelTag { @@ -606,7 +605,7 @@ impl WalIngest { info!("Partial truncation of VM is not supported"); } let num_vm_blocks = 0; - timeline.put_truncation(RelishTag::Relation(rel), lsn, num_vm_blocks)?; + self.put_rel_truncation(modification, rel, num_vm_blocks)?; } Ok(()) } @@ -615,8 +614,7 @@ impl WalIngest { /// fn ingest_xact_record( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, parsed: &XlXactParsedRecord, is_commit: bool, ) -> Result<()> { @@ -632,12 +630,9 @@ impl WalIngest { // This subxact goes to different page. Write the record // for all the XIDs on the previous page, and continue // accumulating XIDs on this new page. - timeline.put_wal_record( - lsn, - RelishTag::Slru { - slru: SlruKind::Clog, - segno, - }, + modification.put_slru_wal_record( + SlruKind::Clog, + segno, rpageno, if is_commit { ZenithWalRecord::ClogSetCommitted { xids: page_xids } @@ -652,12 +647,9 @@ impl WalIngest { rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; page_xids.push(*subxact); } - timeline.put_wal_record( - lsn, - RelishTag::Slru { - slru: SlruKind::Clog, - segno, - }, + modification.put_slru_wal_record( + SlruKind::Clog, + segno, rpageno, if is_commit { ZenithWalRecord::ClogSetCommitted { xids: page_xids } @@ -674,7 +666,10 @@ impl WalIngest { dbnode: xnode.dbnode, relnode: xnode.relnode, }; - timeline.drop_relish(RelishTag::Relation(rel), lsn)?; + let last_lsn = self.timeline.get_last_record_lsn(); + if modification.tline.get_rel_exists(rel, last_lsn)? { + self.put_rel_drop(modification, rel)?; + } } } Ok(()) @@ -682,13 +677,12 @@ impl WalIngest { fn ingest_clog_truncate_record( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, xlrec: &XlClogTruncate, ) -> Result<()> { info!( - "RM_CLOG_ID truncate pageno {} oldestXid {} oldestXidDB {} lsn {}", - xlrec.pageno, xlrec.oldest_xid, xlrec.oldest_xid_db, lsn + "RM_CLOG_ID truncate pageno {} oldestXid {} oldestXidDB {}", + xlrec.pageno, xlrec.oldest_xid, xlrec.oldest_xid_db ); // Here we treat oldestXid and oldestXidDB @@ -719,23 +713,20 @@ impl WalIngest { } // Iterate via SLRU CLOG segments and drop segments that we're ready to truncate - // TODO This implementation is very inefficient - - // it scans all non-rels only to find Clog // // We cannot pass 'lsn' to the Timeline.list_nonrels(), or it // will block waiting for the last valid LSN to advance up to // it. So we use the previous record's LSN in the get calls // instead. - let req_lsn = min(timeline.get_last_record_lsn(), lsn); - for obj in timeline.list_nonrels(req_lsn)? { - if let RelishTag::Slru { slru, segno } = obj { - if slru == SlruKind::Clog { - let segpage = segno * pg_constants::SLRU_PAGES_PER_SEGMENT; - if slru_may_delete_clogsegment(segpage, xlrec.pageno) { - timeline.drop_relish(RelishTag::Slru { slru, segno }, lsn)?; - trace!("Drop CLOG segment {:>04X} at lsn {}", segno, lsn); - } - } + let req_lsn = modification.tline.get_last_record_lsn(); + for segno in modification + .tline + .list_slru_segments(SlruKind::Clog, req_lsn)? + { + let segpage = segno * pg_constants::SLRU_PAGES_PER_SEGMENT; + if slru_may_delete_clogsegment(segpage, xlrec.pageno) { + modification.drop_slru_segment(SlruKind::Clog, segno)?; + trace!("Drop CLOG segment {:>04X}", segno); } } @@ -744,8 +735,7 @@ impl WalIngest { fn ingest_multixact_create_record( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, xlrec: &XlMultiXactCreate, ) -> Result<()> { // Create WAL record for updating the multixact-offsets page @@ -753,12 +743,9 @@ impl WalIngest { let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; - timeline.put_wal_record( - lsn, - RelishTag::Slru { - slru: SlruKind::MultiXactOffsets, - segno, - }, + modification.put_slru_wal_record( + SlruKind::MultiXactOffsets, + segno, rpageno, ZenithWalRecord::MultixactOffsetCreate { mid: xlrec.mid, @@ -790,12 +777,9 @@ impl WalIngest { } let n_this_page = this_page_members.len(); - timeline.put_wal_record( - lsn, - RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno: pageno / pg_constants::SLRU_PAGES_PER_SEGMENT, - }, + modification.put_slru_wal_record( + SlruKind::MultiXactMembers, + pageno / pg_constants::SLRU_PAGES_PER_SEGMENT, pageno % pg_constants::SLRU_PAGES_PER_SEGMENT, ZenithWalRecord::MultixactMembersCreate { moff: offset, @@ -830,8 +814,7 @@ impl WalIngest { fn ingest_multixact_truncate_record( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, xlrec: &XlMultiXactTruncate, ) -> Result<()> { self.checkpoint.oldestMulti = xlrec.end_trunc_off; @@ -847,13 +830,7 @@ impl WalIngest { // Delete all the segments except the last one. The last segment can still // contain, possibly partially, valid data. while segment != endsegment { - timeline.drop_relish( - RelishTag::Slru { - slru: SlruKind::MultiXactMembers, - segno: segment as u32, - }, - lsn, - )?; + modification.drop_slru_segment(SlruKind::MultiXactMembers, segment as u32)?; /* move to next segment, handling wraparound correctly */ if segment == maxsegment { @@ -871,22 +848,538 @@ impl WalIngest { fn ingest_relmap_page( &mut self, - timeline: &dyn TimelineWriter, - lsn: Lsn, + modification: &mut DatadirModification, xlrec: &XlRelmapUpdate, decoded: &DecodedWALRecord, ) -> Result<()> { - let tag = RelishTag::FileNodeMap { - spcnode: xlrec.tsid, - dbnode: xlrec.dbid, - }; - let mut buf = decoded.record.clone(); buf.advance(decoded.main_data_offset); // skip xl_relmap_update buf.advance(12); - timeline.put_page_image(tag, 0, lsn, Bytes::copy_from_slice(&buf[..]))?; + modification.put_relmap_file(xlrec.tsid, xlrec.dbid, Bytes::copy_from_slice(&buf[..]))?; + + Ok(()) + } + + fn put_rel_creation( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + ) -> Result<()> { + self.relsize_cache.insert(rel, 0); + modification.put_rel_creation(rel, 0)?; + Ok(()) + } + + fn put_rel_page_image( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + blknum: BlockNumber, + img: Bytes, + ) -> Result<()> { + self.handle_rel_extend(modification, rel, blknum)?; + modification.put_rel_page_image(rel, blknum, img)?; + Ok(()) + } + + fn put_rel_wal_record( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + blknum: BlockNumber, + rec: ZenithWalRecord, + ) -> Result<()> { + self.handle_rel_extend(modification, rel, blknum)?; + modification.put_rel_wal_record(rel, blknum, rec)?; + Ok(()) + } + + fn put_rel_truncation( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + nblocks: BlockNumber, + ) -> Result<()> { + modification.put_rel_truncation(rel, nblocks)?; + self.relsize_cache.insert(rel, nblocks); + Ok(()) + } + + fn put_rel_drop( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + ) -> Result<()> { + modification.put_rel_drop(rel)?; + self.relsize_cache.remove(&rel); + Ok(()) + } + + fn get_relsize(&mut self, rel: RelTag) -> Result { + if let Some(nblocks) = self.relsize_cache.get(&rel) { + Ok(*nblocks) + } else { + let last_lsn = self.timeline.get_last_record_lsn(); + let nblocks = if !self.timeline.get_rel_exists(rel, last_lsn)? { + 0 + } else { + self.timeline.get_rel_size(rel, last_lsn)? + }; + self.relsize_cache.insert(rel, nblocks); + Ok(nblocks) + } + } + + fn handle_rel_extend( + &mut self, + modification: &mut DatadirModification, + rel: RelTag, + blknum: BlockNumber, + ) -> Result<()> { + let new_nblocks = blknum + 1; + let old_nblocks = if let Some(nblocks) = self.relsize_cache.get(&rel) { + *nblocks + } else { + // Check if the relation exists. We implicitly create relations on first + // record. + // TODO: would be nice if to be more explicit about it + let last_lsn = self.timeline.get_last_record_lsn(); + let nblocks = if !self.timeline.get_rel_exists(rel, last_lsn)? { + // create it with 0 size initially, the logic below will extend it + modification.put_rel_creation(rel, 0)?; + 0 + } else { + self.timeline.get_rel_size(rel, last_lsn)? + }; + self.relsize_cache.insert(rel, nblocks); + nblocks + }; + + if new_nblocks > old_nblocks { + //info!("extending {} {} to {}", rel, old_nblocks, new_nblocks); + modification.put_rel_extend(rel, new_nblocks)?; + + // fill the gap with zeros + for gap_blknum in old_nblocks..blknum { + modification.put_rel_page_image(rel, gap_blknum, ZERO_PAGE.clone())?; + } + self.relsize_cache.insert(rel, new_nblocks); + } + Ok(()) + } + + fn put_slru_page_image( + &mut self, + modification: &mut DatadirModification, + kind: SlruKind, + segno: u32, + blknum: BlockNumber, + img: Bytes, + ) -> Result<()> { + self.handle_slru_extend(modification, kind, segno, blknum)?; + modification.put_slru_page_image(kind, segno, blknum, img)?; + Ok(()) + } + + fn handle_slru_extend( + &mut self, + modification: &mut DatadirModification, + kind: SlruKind, + segno: u32, + blknum: BlockNumber, + ) -> Result<()> { + // we don't use a cache for this like we do for relations. SLRUS are explcitly + // extended with ZEROPAGE records, not with commit records, so it happens + // a lot less frequently. + + let new_nblocks = blknum + 1; + // Check if the relation exists. We implicitly create relations on first + // record. + // TODO: would be nice if to be more explicit about it + let last_lsn = self.timeline.get_last_record_lsn(); + let old_nblocks = if !self + .timeline + .get_slru_segment_exists(kind, segno, last_lsn)? + { + // create it with 0 size initially, the logic below will extend it + modification.put_slru_segment_creation(kind, segno, 0)?; + 0 + } else { + self.timeline.get_slru_segment_size(kind, segno, last_lsn)? + }; + + if new_nblocks > old_nblocks { + trace!( + "extending SLRU {:?} seg {} from {} to {} blocks", + kind, + segno, + old_nblocks, + new_nblocks + ); + modification.put_slru_extend(kind, segno, new_nblocks)?; + + // fill the gap with zeros + for gap_blknum in old_nblocks..blknum { + modification.put_slru_page_image(kind, segno, gap_blknum, ZERO_PAGE.clone())?; + } + } + Ok(()) + } +} + +/// +/// Tests that should work the same with any Repository/Timeline implementation. +/// +#[allow(clippy::bool_assert_comparison)] +#[cfg(test)] +mod tests { + use super::*; + use crate::pgdatadir_mapping::create_test_timeline; + use crate::repository::repo_harness::*; + use postgres_ffi::pg_constants; + + /// Arbitrary relation tag, for testing. + const TESTREL_A: RelTag = RelTag { + spcnode: 0, + dbnode: 111, + relnode: 1000, + forknum: 0, + }; + + fn assert_current_logical_size(_timeline: &DatadirTimeline, _lsn: Lsn) { + // TODO + } + + static ZERO_CHECKPOINT: Bytes = Bytes::from_static(&[0u8; SIZEOF_CHECKPOINT]); + + fn init_walingest_test(tline: &DatadirTimeline) -> Result> { + let mut m = tline.begin_modification(Lsn(0x10)); + m.put_checkpoint(ZERO_CHECKPOINT.clone())?; + m.put_relmap_file(0, 111, Bytes::from(""))?; // dummy relmapper file + m.commit()?; + let walingest = WalIngest::new(tline, Lsn(0x10))?; + + Ok(walingest) + } + + #[test] + fn test_relsize() -> Result<()> { + let repo = RepoHarness::create("test_relsize")?.load(); + let tline = create_test_timeline(repo, TIMELINE_ID)?; + let mut walingest = init_walingest_test(&tline)?; + + let mut m = tline.begin_modification(Lsn(0x20)); + walingest.put_rel_creation(&mut m, TESTREL_A)?; + walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))?; + m.commit()?; + let mut m = tline.begin_modification(Lsn(0x30)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 3"))?; + m.commit()?; + let mut m = tline.begin_modification(Lsn(0x40)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1 at 4"))?; + m.commit()?; + let mut m = tline.begin_modification(Lsn(0x50)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 2, TEST_IMG("foo blk 2 at 5"))?; + m.commit()?; + + assert_current_logical_size(&tline, Lsn(0x50)); + + // The relation was created at LSN 2, not visible at LSN 1 yet. + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10))?, false); + assert!(tline.get_rel_size(TESTREL_A, Lsn(0x10)).is_err()); + + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20))?, 1); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50))?, 3); + + // Check page contents at each LSN + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20))?, + TEST_IMG("foo blk 0 at 2") + ); + + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30))?, + TEST_IMG("foo blk 0 at 3") + ); + + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40))?, + TEST_IMG("foo blk 0 at 3") + ); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40))?, + TEST_IMG("foo blk 1 at 4") + ); + + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50))?, + TEST_IMG("foo blk 0 at 3") + ); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50))?, + TEST_IMG("foo blk 1 at 4") + ); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50))?, + TEST_IMG("foo blk 2 at 5") + ); + + // Truncate last block + let mut m = tline.begin_modification(Lsn(0x60)); + walingest.put_rel_truncation(&mut m, TESTREL_A, 2)?; + m.commit()?; + assert_current_logical_size(&tline, Lsn(0x60)); + + // Check reported size and contents after truncation + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60))?, 2); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60))?, + TEST_IMG("foo blk 0 at 3") + ); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60))?, + TEST_IMG("foo blk 1 at 4") + ); + + // should still see the truncated block with older LSN + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50))?, 3); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50))?, + TEST_IMG("foo blk 2 at 5") + ); + + // Truncate to zero length + let mut m = tline.begin_modification(Lsn(0x68)); + walingest.put_rel_truncation(&mut m, TESTREL_A, 0)?; + m.commit()?; + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x68))?, 0); + + // Extend from 0 to 2 blocks, leaving a gap + let mut m = tline.begin_modification(Lsn(0x70)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1"))?; + m.commit()?; + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x70))?, 2); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70))?, + ZERO_PAGE + ); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70))?, + TEST_IMG("foo blk 1") + ); + + // Extend a lot more, leaving a big gap that spans across segments + let mut m = tline.begin_modification(Lsn(0x80)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 1500, TEST_IMG("foo blk 1500"))?; + m.commit()?; + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x80))?, 1501); + for blk in 2..1500 { + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80))?, + ZERO_PAGE + ); + } + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80))?, + TEST_IMG("foo blk 1500") + ); + + Ok(()) + } + + // Test what happens if we dropped a relation + // and then created it again within the same layer. + #[test] + fn test_drop_extend() -> Result<()> { + let repo = RepoHarness::create("test_drop_extend")?.load(); + let tline = create_test_timeline(repo, TIMELINE_ID)?; + let mut walingest = init_walingest_test(&tline)?; + + let mut m = tline.begin_modification(Lsn(0x20)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))?; + m.commit()?; + + // Check that rel exists and size is correct + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20))?, 1); + + // Drop rel + let mut m = tline.begin_modification(Lsn(0x30)); + walingest.put_rel_drop(&mut m, TESTREL_A)?; + m.commit()?; + + // Check that rel is not visible anymore + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x30))?, false); + + // FIXME: should fail + //assert!(tline.get_rel_size(TESTREL_A, Lsn(0x30))?.is_none()); + + // Re-create it + let mut m = tline.begin_modification(Lsn(0x40)); + walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 4"))?; + m.commit()?; + + // Check that rel exists and size is correct + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x40))?, true); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x40))?, 1); + + Ok(()) + } + + // Test what happens if we truncated a relation + // so that one of its segments was dropped + // and then extended it again within the same layer. + #[test] + fn test_truncate_extend() -> Result<()> { + let repo = RepoHarness::create("test_truncate_extend")?.load(); + let tline = create_test_timeline(repo, TIMELINE_ID)?; + let mut walingest = init_walingest_test(&tline)?; + + // Create a 20 MB relation (the size is arbitrary) + let relsize = 20 * 1024 * 1024 / 8192; + let mut m = tline.begin_modification(Lsn(0x20)); + for blkno in 0..relsize { + let data = format!("foo blk {} at {}", blkno, Lsn(0x20)); + walingest.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))?; + } + m.commit()?; + + // The relation was created at LSN 20, not visible at LSN 1 yet. + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10))?, false); + assert!(tline.get_rel_size(TESTREL_A, Lsn(0x10)).is_err()); + + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20))?, true); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20))?, relsize); + + // Check relation content + for blkno in 0..relsize { + let lsn = Lsn(0x20); + let data = format!("foo blk {} at {}", blkno, lsn); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, blkno, lsn)?, + TEST_IMG(&data) + ); + } + + // Truncate relation so that second segment was dropped + // - only leave one page + let mut m = tline.begin_modification(Lsn(0x60)); + walingest.put_rel_truncation(&mut m, TESTREL_A, 1)?; + m.commit()?; + + // Check reported size and contents after truncation + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60))?, 1); + + for blkno in 0..1 { + let lsn = Lsn(0x20); + let data = format!("foo blk {} at {}", blkno, lsn); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60))?, + TEST_IMG(&data) + ); + } + + // should still see all blocks with older LSN + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50))?, relsize); + for blkno in 0..relsize { + let lsn = Lsn(0x20); + let data = format!("foo blk {} at {}", blkno, lsn); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50))?, + TEST_IMG(&data) + ); + } + + // Extend relation again. + // Add enough blocks to create second segment + let lsn = Lsn(0x80); + let mut m = tline.begin_modification(lsn); + for blkno in 0..relsize { + let data = format!("foo blk {} at {}", blkno, lsn); + walingest.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))?; + } + m.commit()?; + + assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x80))?, true); + assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x80))?, relsize); + // Check relation content + for blkno in 0..relsize { + let lsn = Lsn(0x80); + let data = format!("foo blk {} at {}", blkno, lsn); + assert_eq!( + tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80))?, + TEST_IMG(&data) + ); + } + + Ok(()) + } + + /// Test get_relsize() and truncation with a file larger than 1 GB, so that it's + /// split into multiple 1 GB segments in Postgres. + #[test] + fn test_large_rel() -> Result<()> { + let repo = RepoHarness::create("test_large_rel")?.load(); + let tline = create_test_timeline(repo, TIMELINE_ID)?; + let mut walingest = init_walingest_test(&tline)?; + + let mut lsn = 0x10; + for blknum in 0..pg_constants::RELSEG_SIZE + 1 { + lsn += 0x10; + let mut m = tline.begin_modification(Lsn(lsn)); + let img = TEST_IMG(&format!("foo blk {} at {}", blknum, Lsn(lsn))); + walingest.put_rel_page_image(&mut m, TESTREL_A, blknum as BlockNumber, img)?; + m.commit()?; + } + + assert_current_logical_size(&tline, Lsn(lsn)); + + assert_eq!( + tline.get_rel_size(TESTREL_A, Lsn(lsn))?, + pg_constants::RELSEG_SIZE + 1 + ); + + // Truncate one block + lsn += 0x10; + let mut m = tline.begin_modification(Lsn(lsn)); + walingest.put_rel_truncation(&mut m, TESTREL_A, pg_constants::RELSEG_SIZE)?; + m.commit()?; + assert_eq!( + tline.get_rel_size(TESTREL_A, Lsn(lsn))?, + pg_constants::RELSEG_SIZE + ); + assert_current_logical_size(&tline, Lsn(lsn)); + + // Truncate another block + lsn += 0x10; + let mut m = tline.begin_modification(Lsn(lsn)); + walingest.put_rel_truncation(&mut m, TESTREL_A, pg_constants::RELSEG_SIZE - 1)?; + m.commit()?; + assert_eq!( + tline.get_rel_size(TESTREL_A, Lsn(lsn))?, + pg_constants::RELSEG_SIZE - 1 + ); + assert_current_logical_size(&tline, Lsn(lsn)); + + // Truncate to 1500, and then truncate all the way down to 0, one block at a time + // This tests the behavior at segment boundaries + let mut size: i32 = 3000; + while size >= 0 { + lsn += 0x10; + let mut m = tline.begin_modification(Lsn(lsn)); + walingest.put_rel_truncation(&mut m, TESTREL_A, size as BlockNumber)?; + m.commit()?; + assert_eq!( + tline.get_rel_size(TESTREL_A, Lsn(lsn))?, + size as BlockNumber + ); + + size -= 1; + } + assert_current_logical_size(&tline, Lsn(lsn)); Ok(()) } diff --git a/pageserver/src/walreceiver.rs b/pageserver/src/walreceiver.rs index 2c10ad315b..e382475627 100644 --- a/pageserver/src/walreceiver.rs +++ b/pageserver/src/walreceiver.rs @@ -6,6 +6,7 @@ //! We keep one WAL receiver active per timeline. use crate::config::PageServerConf; +use crate::repository::{Repository, Timeline}; use crate::tenant_mgr; use crate::thread_mgr; use crate::thread_mgr::ThreadKind; @@ -182,13 +183,13 @@ fn walreceiver_main( let repo = tenant_mgr::get_repository_for_tenant(tenant_id) .with_context(|| format!("no repository found for tenant {}", tenant_id))?; - let timeline = repo.get_timeline_load(timeline_id).with_context(|| { - format!( - "local timeline {} not found for tenant {}", - timeline_id, tenant_id - ) - })?; - + let timeline = + tenant_mgr::get_timeline_for_tenant_load(tenant_id, timeline_id).with_context(|| { + format!( + "local timeline {} not found for tenant {}", + timeline_id, tenant_id + ) + })?; let remote_index = repo.get_remote_index(); // @@ -251,11 +252,10 @@ fn walreceiver_main( // It is important to deal with the aligned records as lsn in getPage@LSN is // aligned and can be several bytes bigger. Without this alignment we are - // at risk of hittind a deadlock. + // at risk of hitting a deadlock. anyhow::ensure!(lsn.is_aligned()); - let writer = timeline.writer(); - walingest.ingest_record(writer.as_ref(), recdata, lsn)?; + walingest.ingest_record(&timeline, recdata, lsn)?; fail_point!("walreceiver-after-ingest"); @@ -267,6 +267,8 @@ fn walreceiver_main( caught_up = true; } + timeline.tline.check_checkpoint_distance()?; + Some(endlsn) } @@ -310,7 +312,7 @@ fn walreceiver_main( // The last LSN we processed. It is not guaranteed to survive pageserver crash. let write_lsn = u64::from(last_lsn); // `disk_consistent_lsn` is the LSN at which page server guarantees local persistence of all received data - let flush_lsn = u64::from(timeline.get_disk_consistent_lsn()); + let flush_lsn = u64::from(timeline.tline.get_disk_consistent_lsn()); // The last LSN that is synced to remote storage and is guaranteed to survive pageserver crash // Used by safekeepers to remove WAL preceding `remote_consistent_lsn`. let apply_lsn = u64::from(timeline_remote_consistent_lsn); diff --git a/pageserver/src/walrecord.rs b/pageserver/src/walrecord.rs index ca9107cdbf..5947a0c147 100644 --- a/pageserver/src/walrecord.rs +++ b/pageserver/src/walrecord.rs @@ -10,7 +10,47 @@ use postgres_ffi::{MultiXactId, MultiXactOffset, MultiXactStatus, Oid, Transacti use serde::{Deserialize, Serialize}; use tracing::*; -use crate::repository::ZenithWalRecord; +/// Each update to a page is represented by a ZenithWalRecord. It can be a wrapper +/// around a PostgreSQL WAL record, or a custom zenith-specific "record". +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +pub enum ZenithWalRecord { + /// Native PostgreSQL WAL record + Postgres { will_init: bool, rec: Bytes }, + + /// Clear bits in heap visibility map. ('flags' is bitmap of bits to clear) + ClearVisibilityMapFlags { + new_heap_blkno: Option, + old_heap_blkno: Option, + flags: u8, + }, + /// Mark transaction IDs as committed on a CLOG page + ClogSetCommitted { xids: Vec }, + /// Mark transaction IDs as aborted on a CLOG page + ClogSetAborted { xids: Vec }, + /// Extend multixact offsets SLRU + MultixactOffsetCreate { + mid: MultiXactId, + moff: MultiXactOffset, + }, + /// Extend multixact members SLRU. + MultixactMembersCreate { + moff: MultiXactOffset, + members: Vec, + }, +} + +impl ZenithWalRecord { + /// Does replaying this WAL record initialize the page from scratch, or does + /// it need to be applied over the previous image of the page? + pub fn will_init(&self) -> bool { + match self { + ZenithWalRecord::Postgres { will_init, rec: _ } => *will_init, + + // None of the special zenith record types currently initialize the page + _ => false, + } + } +} /// DecodedBkpBlock represents per-page data contained in a WAL record. #[derive(Default)] @@ -87,6 +127,28 @@ impl XlRelmapUpdate { } } +#[repr(C)] +#[derive(Debug)] +pub struct XlSmgrCreate { + pub rnode: RelFileNode, + // FIXME: This is ForkNumber in storage_xlog.h. That's an enum. Does it have + // well-defined size? + pub forknum: u8, +} + +impl XlSmgrCreate { + pub fn decode(buf: &mut Bytes) -> XlSmgrCreate { + XlSmgrCreate { + rnode: RelFileNode { + spcnode: buf.get_u32_le(), /* tablespace */ + dbnode: buf.get_u32_le(), /* database */ + relnode: buf.get_u32_le(), /* relation */ + }, + forknum: buf.get_u32_le() as u8, + } + } +} + #[repr(C)] #[derive(Debug)] pub struct XlSmgrTruncate { diff --git a/pageserver/src/walredo.rs b/pageserver/src/walredo.rs index 704b8f2583..ae22f1eead 100644 --- a/pageserver/src/walredo.rs +++ b/pageserver/src/walredo.rs @@ -42,8 +42,10 @@ use zenith_utils::nonblock::set_nonblock; use zenith_utils::zid::ZTenantId; use crate::config::PageServerConf; -use crate::relish::*; -use crate::repository::ZenithWalRecord; +use crate::pgdatadir_mapping::{key_to_rel_block, key_to_slru_block}; +use crate::reltag::{RelTag, SlruKind}; +use crate::repository::Key; +use crate::walrecord::ZenithWalRecord; use postgres_ffi::nonrelfile_utils::mx_offset_to_flags_bitshift; use postgres_ffi::nonrelfile_utils::mx_offset_to_flags_offset; use postgres_ffi::nonrelfile_utils::mx_offset_to_member_offset; @@ -75,8 +77,7 @@ pub trait WalRedoManager: Send + Sync { /// the reords. fn request_redo( &self, - rel: RelishTag, - blknum: u32, + key: Key, lsn: Lsn, base_img: Option, records: Vec<(Lsn, ZenithWalRecord)>, @@ -92,8 +93,7 @@ pub struct DummyRedoManager {} impl crate::walredo::WalRedoManager for DummyRedoManager { fn request_redo( &self, - _rel: RelishTag, - _blknum: u32, + _key: Key, _lsn: Lsn, _base_img: Option, _records: Vec<(Lsn, ZenithWalRecord)>, @@ -152,28 +152,6 @@ fn can_apply_in_zenith(rec: &ZenithWalRecord) -> bool { } } -fn check_forknum(rel: &RelishTag, expected_forknum: u8) -> bool { - if let RelishTag::Relation(RelTag { - forknum, - spcnode: _, - dbnode: _, - relnode: _, - }) = rel - { - *forknum == expected_forknum - } else { - false - } -} - -fn check_slru_segno(rel: &RelishTag, expected_slru: SlruKind, expected_segno: u32) -> bool { - if let RelishTag::Slru { slru, segno } = rel { - *slru == expected_slru && *segno == expected_segno - } else { - false - } -} - /// An error happened in WAL redo #[derive(Debug, thiserror::Error)] pub enum WalRedoError { @@ -184,6 +162,8 @@ pub enum WalRedoError { InvalidState, #[error("cannot perform WAL redo for this request")] InvalidRequest, + #[error("cannot perform WAL redo for this record")] + InvalidRecord, } /// @@ -198,8 +178,7 @@ impl WalRedoManager for PostgresRedoManager { /// fn request_redo( &self, - rel: RelishTag, - blknum: u32, + key: Key, lsn: Lsn, base_img: Option, records: Vec<(Lsn, ZenithWalRecord)>, @@ -217,11 +196,10 @@ impl WalRedoManager for PostgresRedoManager { if rec_zenith != batch_zenith { let result = if batch_zenith { - self.apply_batch_zenith(rel, blknum, lsn, img, &records[batch_start..i]) + self.apply_batch_zenith(key, lsn, img, &records[batch_start..i]) } else { self.apply_batch_postgres( - rel, - blknum, + key, lsn, img, &records[batch_start..i], @@ -236,11 +214,10 @@ impl WalRedoManager for PostgresRedoManager { } // last batch if batch_zenith { - self.apply_batch_zenith(rel, blknum, lsn, img, &records[batch_start..]) + self.apply_batch_zenith(key, lsn, img, &records[batch_start..]) } else { self.apply_batch_postgres( - rel, - blknum, + key, lsn, img, &records[batch_start..], @@ -268,16 +245,15 @@ impl PostgresRedoManager { /// fn apply_batch_postgres( &self, - rel: RelishTag, - blknum: u32, + key: Key, lsn: Lsn, base_img: Option, records: &[(Lsn, ZenithWalRecord)], wal_redo_timeout: Duration, ) -> Result { - let start_time = Instant::now(); + let (rel, blknum) = key_to_rel_block(key).or(Err(WalRedoError::InvalidRecord))?; - let apply_result: Result; + let start_time = Instant::now(); let mut process_guard = self.process.lock().unwrap(); let lock_time = Instant::now(); @@ -291,16 +267,11 @@ impl PostgresRedoManager { WAL_REDO_WAIT_TIME.observe(lock_time.duration_since(start_time).as_secs_f64()); - let result = if let RelishTag::Relation(rel) = rel { - // Relational WAL records are applied using wal-redo-postgres - let buf_tag = BufferTag { rel, blknum }; - apply_result = process.apply_wal_records(buf_tag, base_img, records, wal_redo_timeout); - - apply_result.map_err(WalRedoError::IoError) - } else { - error!("unexpected non-relation relish: {:?}", rel); - Err(WalRedoError::InvalidRequest) - }; + // Relational WAL records are applied using wal-redo-postgres + let buf_tag = BufferTag { rel, blknum }; + let result = process + .apply_wal_records(buf_tag, base_img, records, wal_redo_timeout) + .map_err(WalRedoError::IoError); let end_time = Instant::now(); let duration = end_time.duration_since(lock_time); @@ -326,8 +297,7 @@ impl PostgresRedoManager { /// fn apply_batch_zenith( &self, - rel: RelishTag, - blknum: u32, + key: Key, lsn: Lsn, base_img: Option, records: &[(Lsn, ZenithWalRecord)], @@ -346,7 +316,7 @@ impl PostgresRedoManager { // Apply all the WAL records in the batch for (record_lsn, record) in records.iter() { - self.apply_record_zenith(rel, blknum, &mut page, *record_lsn, record)?; + self.apply_record_zenith(key, &mut page, *record_lsn, record)?; } // Success! let end_time = Instant::now(); @@ -365,8 +335,7 @@ impl PostgresRedoManager { fn apply_record_zenith( &self, - rel: RelishTag, - blknum: u32, + key: Key, page: &mut BytesMut, _record_lsn: Lsn, record: &ZenithWalRecord, @@ -384,10 +353,11 @@ impl PostgresRedoManager { old_heap_blkno, flags, } => { - // sanity check that this is modifying the correct relish + // sanity check that this is modifying the correct relation + let (rel, blknum) = key_to_rel_block(key).or(Err(WalRedoError::InvalidRecord))?; assert!( - check_forknum(&rel, pg_constants::VISIBILITYMAP_FORKNUM), - "ClearVisibilityMapFlags record on unexpected rel {:?}", + rel.forknum == pg_constants::VISIBILITYMAP_FORKNUM, + "ClearVisibilityMapFlags record on unexpected rel {}", rel ); if let Some(heap_blkno) = *new_heap_blkno { @@ -421,6 +391,14 @@ impl PostgresRedoManager { // Non-relational WAL records are handled here, with custom code that has the // same effects as the corresponding Postgres WAL redo function. ZenithWalRecord::ClogSetCommitted { xids } => { + let (slru_kind, segno, blknum) = + key_to_slru_block(key).or(Err(WalRedoError::InvalidRecord))?; + assert_eq!( + slru_kind, + SlruKind::Clog, + "ClogSetCommitted record with unexpected key {}", + key + ); for &xid in xids { let pageno = xid as u32 / pg_constants::CLOG_XACTS_PER_PAGE; let expected_segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; @@ -428,12 +406,17 @@ impl PostgresRedoManager { // Check that we're modifying the correct CLOG block. assert!( - check_slru_segno(&rel, SlruKind::Clog, expected_segno), - "ClogSetCommitted record for XID {} with unexpected rel {:?}", + segno == expected_segno, + "ClogSetCommitted record for XID {} with unexpected key {}", xid, - rel + key + ); + assert!( + blknum == expected_blknum, + "ClogSetCommitted record for XID {} with unexpected key {}", + xid, + key ); - assert!(blknum == expected_blknum); transaction_id_set_status( xid, @@ -443,6 +426,14 @@ impl PostgresRedoManager { } } ZenithWalRecord::ClogSetAborted { xids } => { + let (slru_kind, segno, blknum) = + key_to_slru_block(key).or(Err(WalRedoError::InvalidRecord))?; + assert_eq!( + slru_kind, + SlruKind::Clog, + "ClogSetAborted record with unexpected key {}", + key + ); for &xid in xids { let pageno = xid as u32 / pg_constants::CLOG_XACTS_PER_PAGE; let expected_segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; @@ -450,17 +441,30 @@ impl PostgresRedoManager { // Check that we're modifying the correct CLOG block. assert!( - check_slru_segno(&rel, SlruKind::Clog, expected_segno), - "ClogSetCommitted record for XID {} with unexpected rel {:?}", + segno == expected_segno, + "ClogSetAborted record for XID {} with unexpected key {}", xid, - rel + key + ); + assert!( + blknum == expected_blknum, + "ClogSetAborted record for XID {} with unexpected key {}", + xid, + key ); - assert!(blknum == expected_blknum); transaction_id_set_status(xid, pg_constants::TRANSACTION_STATUS_ABORTED, page); } } ZenithWalRecord::MultixactOffsetCreate { mid, moff } => { + let (slru_kind, segno, blknum) = + key_to_slru_block(key).or(Err(WalRedoError::InvalidRecord))?; + assert_eq!( + slru_kind, + SlruKind::MultiXactOffsets, + "MultixactOffsetCreate record with unexpected key {}", + key + ); // Compute the block and offset to modify. // See RecordNewMultiXact in PostgreSQL sources. let pageno = mid / pg_constants::MULTIXACT_OFFSETS_PER_PAGE as u32; @@ -471,16 +475,29 @@ impl PostgresRedoManager { let expected_segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let expected_blknum = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; assert!( - check_slru_segno(&rel, SlruKind::MultiXactOffsets, expected_segno), - "MultiXactOffsetsCreate record for multi-xid {} with unexpected rel {:?}", + segno == expected_segno, + "MultiXactOffsetsCreate record for multi-xid {} with unexpected key {}", mid, - rel + key + ); + assert!( + blknum == expected_blknum, + "MultiXactOffsetsCreate record for multi-xid {} with unexpected key {}", + mid, + key ); - assert!(blknum == expected_blknum); LittleEndian::write_u32(&mut page[offset..offset + 4], *moff); } ZenithWalRecord::MultixactMembersCreate { moff, members } => { + let (slru_kind, segno, blknum) = + key_to_slru_block(key).or(Err(WalRedoError::InvalidRecord))?; + assert_eq!( + slru_kind, + SlruKind::MultiXactMembers, + "MultixactMembersCreate record with unexpected key {}", + key + ); for (i, member) in members.iter().enumerate() { let offset = moff + i as u32; @@ -495,12 +512,17 @@ impl PostgresRedoManager { let expected_segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT; let expected_blknum = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT; assert!( - check_slru_segno(&rel, SlruKind::MultiXactMembers, expected_segno), - "MultiXactMembersCreate record at offset {} with unexpected rel {:?}", + segno == expected_segno, + "MultiXactMembersCreate record for offset {} with unexpected key {}", moff, - rel + key + ); + assert!( + blknum == expected_blknum, + "MultiXactMembersCreate record for offset {} with unexpected key {}", + moff, + key ); - assert!(blknum == expected_blknum); let mut flagsval = LittleEndian::read_u32(&page[flagsoff..flagsoff + 4]); flagsval &= !(((1 << pg_constants::MXACT_MEMBER_BITS_PER_XACT) - 1) << bshift); diff --git a/postgres_ffi/src/pg_constants.rs b/postgres_ffi/src/pg_constants.rs index 76f837cefc..7230b841f5 100644 --- a/postgres_ffi/src/pg_constants.rs +++ b/postgres_ffi/src/pg_constants.rs @@ -24,6 +24,9 @@ pub const VISIBILITYMAP_FORKNUM: u8 = 2; pub const INIT_FORKNUM: u8 = 3; // From storage_xlog.h +pub const XLOG_SMGR_CREATE: u8 = 0x10; +pub const XLOG_SMGR_TRUNCATE: u8 = 0x20; + pub const SMGR_TRUNCATE_HEAP: u32 = 0x0001; pub const SMGR_TRUNCATE_VM: u32 = 0x0002; pub const SMGR_TRUNCATE_FSM: u32 = 0x0004; @@ -113,7 +116,6 @@ pub const XACT_XINFO_HAS_TWOPHASE: u32 = 1u32 << 4; // From pg_control.h and rmgrlist.h pub const XLOG_NEXTOID: u8 = 0x30; pub const XLOG_SWITCH: u8 = 0x40; -pub const XLOG_SMGR_TRUNCATE: u8 = 0x20; pub const XLOG_FPI_FOR_HINT: u8 = 0xA0; pub const XLOG_FPI: u8 = 0xB0; pub const DB_SHUTDOWNED: u32 = 1; diff --git a/test_runner/batch_others/test_snapfiles_gc.py b/test_runner/batch_others/test_snapfiles_gc.py deleted file mode 100644 index d00af53864..0000000000 --- a/test_runner/batch_others/test_snapfiles_gc.py +++ /dev/null @@ -1,130 +0,0 @@ -from contextlib import closing -import psycopg2.extras -from fixtures.utils import print_gc_result -from fixtures.zenith_fixtures import ZenithEnv -from fixtures.log_helper import log - - -# -# Test Garbage Collection of old layer files -# -# This test is pretty tightly coupled with the current implementation of layered -# storage, in layered_repository.rs. -# -def test_layerfiles_gc(zenith_simple_env: ZenithEnv): - env = zenith_simple_env - env.zenith_cli.create_branch("test_layerfiles_gc", "empty") - pg = env.postgres.create_start('test_layerfiles_gc') - - with closing(pg.connect()) as conn: - with conn.cursor() as cur: - with closing(env.pageserver.connect()) as psconn: - with psconn.cursor(cursor_factory=psycopg2.extras.DictCursor) as pscur: - - # Get the timeline ID of our branch. We need it for the 'do_gc' command - cur.execute("SHOW zenith.zenith_timeline") - timeline = cur.fetchone()[0] - - # Create a test table - cur.execute("CREATE TABLE foo(x integer)") - cur.execute("INSERT INTO foo VALUES (1)") - - cur.execute("select relfilenode from pg_class where oid = 'foo'::regclass") - row = cur.fetchone() - log.info(f"relfilenode is {row[0]}") - - # Run GC, to clear out any garbage left behind in the catalogs by - # the CREATE TABLE command. We want to have a clean slate with no garbage - # before running the actual tests below, otherwise the counts won't match - # what we expect. - # - # Also run vacuum first to make it less likely that autovacuum or pruning - # kicks in and confuses our numbers. - cur.execute("VACUUM") - - # delete the row, to update the Visibility Map. We don't want the VM - # update to confuse our numbers either. - cur.execute("DELETE FROM foo") - - log.info("Running GC before test") - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - # remember the number of files - layer_relfiles_remain = (row['layer_relfiles_total'] - - row['layer_relfiles_removed']) - assert layer_relfiles_remain > 0 - - # Insert a row and run GC. Checkpoint should freeze the layer - # so that there is only the most recent image layer left for the rel, - # removing the old image and delta layer. - log.info("Inserting one row and running GC") - cur.execute("INSERT INTO foo VALUES (1)") - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - assert row['layer_relfiles_total'] == layer_relfiles_remain + 2 - assert row['layer_relfiles_removed'] == 2 - assert row['layer_relfiles_dropped'] == 0 - - # Insert two more rows and run GC. - # This should create new image and delta layer file with the new contents, and - # then remove the old one image and the just-created delta layer. - log.info("Inserting two more rows and running GC") - cur.execute("INSERT INTO foo VALUES (2)") - cur.execute("INSERT INTO foo VALUES (3)") - - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - assert row['layer_relfiles_total'] == layer_relfiles_remain + 2 - assert row['layer_relfiles_removed'] == 2 - assert row['layer_relfiles_dropped'] == 0 - - # Do it again. Should again create two new layer files and remove old ones. - log.info("Inserting two more rows and running GC") - cur.execute("INSERT INTO foo VALUES (2)") - cur.execute("INSERT INTO foo VALUES (3)") - - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - assert row['layer_relfiles_total'] == layer_relfiles_remain + 2 - assert row['layer_relfiles_removed'] == 2 - assert row['layer_relfiles_dropped'] == 0 - - # Run GC again, with no changes in the database. Should not remove anything. - log.info("Run GC again, with nothing to do") - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - assert row['layer_relfiles_total'] == layer_relfiles_remain - assert row['layer_relfiles_removed'] == 0 - assert row['layer_relfiles_dropped'] == 0 - - # - # Test DROP TABLE checks that relation data and metadata was deleted by GC from object storage - # - log.info("Drop table and run GC again") - cur.execute("DROP TABLE foo") - - pscur.execute(f"do_gc {env.initial_tenant.hex} {timeline} 0") - row = pscur.fetchone() - print_gc_result(row) - - # We still cannot remove the latest layers - # because they serve as tombstones for earlier layers. - assert row['layer_relfiles_dropped'] == 0 - # Each relation fork is counted separately, hence 3. - assert row['layer_relfiles_needed_as_tombstone'] == 3 - - # The catalog updates also create new layer files of the catalogs, which - # are counted as 'removed' - assert row['layer_relfiles_removed'] > 0 - - # TODO Change the test to check actual CG of dropped layers. - # Each relation fork is counted separately, hence 3. - #assert row['layer_relfiles_dropped'] == 3 - - # TODO: perhaps we should count catalog and user relations separately, - # to make this kind of testing more robust diff --git a/test_runner/fixtures/utils.py b/test_runner/fixtures/utils.py index 236c225bfb..58f7294eb5 100644 --- a/test_runner/fixtures/utils.py +++ b/test_runner/fixtures/utils.py @@ -74,8 +74,5 @@ def lsn_from_hex(lsn_hex: str) -> int: def print_gc_result(row): log.info("GC duration {elapsed} ms".format_map(row)) log.info( - " REL total: {layer_relfiles_total}, needed_by_cutoff {layer_relfiles_needed_by_cutoff}, needed_by_branches: {layer_relfiles_needed_by_branches}, not_updated: {layer_relfiles_not_updated}, needed_as_tombstone {layer_relfiles_needed_as_tombstone}, removed: {layer_relfiles_removed}, dropped: {layer_relfiles_dropped}" - .format_map(row)) - log.info( - " NONREL total: {layer_nonrelfiles_total}, needed_by_cutoff {layer_nonrelfiles_needed_by_cutoff}, needed_by_branches: {layer_nonrelfiles_needed_by_branches}, not_updated: {layer_nonrelfiles_not_updated}, needed_as_tombstone {layer_nonrelfiles_needed_as_tombstone}, removed: {layer_nonrelfiles_removed}, dropped: {layer_nonrelfiles_dropped}" + " total: {layers_total}, needed_by_cutoff {layers_needed_by_cutoff}, needed_by_branches: {layers_needed_by_branches}, not_updated: {layers_not_updated}, removed: {layers_removed}" .format_map(row)) diff --git a/vendor/postgres b/vendor/postgres index 093aa160e5..756a01aade 160000 --- a/vendor/postgres +++ b/vendor/postgres @@ -1 +1 @@ -Subproject commit 093aa160e5df19814ff19b995d36dd5ee03c7f8b +Subproject commit 756a01aade765d1d2ac115e7e189865ff697222b