Compare commits

..

10 Commits

Author SHA1 Message Date
Konstantin Knizhnik
458ca82d75 Bump postgres version 2021-09-27 12:08:55 +03:00
Konstantin Knizhnik
0dface838d Bump postgres version 2021-09-24 19:21:16 +03:00
Konstantin Knizhnik
f7878c5157 Bump postgres version 2021-09-24 19:10:18 +03:00
Konstantin Knizhnik
53c870b8e7 Store file cache outside pgdata dir 2021-09-24 19:07:30 +03:00
Konstantin Knizhnik
c23b65914e Sete zenith.file_cache_size parameter 2021-09-24 18:19:41 +03:00
Konstantin Knizhnik
b51d3f6b2b Revert "Save page received from page server in local file cache"
This reverts commit 137472db91.
2021-09-24 18:04:21 +03:00
Konstantin Knizhnik
137472db91 Save page received from page server in local file cache 2021-09-24 15:15:54 +03:00
Konstantin Knizhnik
f817985f2b Do not throw error on attempt to drop unexisted relation 2021-09-23 12:11:06 +03:00
Konstantin Knizhnik
2e94e1428e Remove import_timeline_wal function 2021-09-23 12:11:06 +03:00
Konstantin Knizhnik
9a486ca109 Do not write WAL at pageserver 2021-09-23 12:11:06 +03:00
73 changed files with 1704 additions and 3759 deletions

View File

@@ -254,7 +254,7 @@ jobs:
when: always
command: |
du -sh /tmp/test_output/*
find /tmp/test_output -type f ! -name "pg.log" ! -name "pageserver.log" ! -name "wal_acceptor.log" ! -name "regression.diffs" ! -name "junit.xml" ! -name "*.filediff" ! -name "*.stdout" ! -name "*.stderr" -delete
find /tmp/test_output -type f ! -name "pg.log" ! -name "pageserver.log" ! -name "wal_acceptor.log" ! -name "regression.diffs" ! -name "junit.xml" ! -name "*.filediff" -delete
du -sh /tmp/test_output/*
- store_artifacts:
path: /tmp/test_output

View File

@@ -2,13 +2,12 @@
**/__pycache__
**/.pytest_cache
.git
target
tmp_check
tmp_install
tmp_check_cli
test_output
.vscode
.zenith
integration_tests/.zenith
.mypy_cache
/target
/tmp_check
/tmp_install
/tmp_check_cli
/test_output
/.vscode
/.zenith
/integration_tests/.zenith
/Dockerfile

50
Cargo.lock generated
View File

@@ -177,9 +177,9 @@ dependencies = [
[[package]]
name = "bitflags"
version = "1.3.2"
version = "1.2.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a"
checksum = "cf1de2fe8c75bc145a2f577add951f8134889b4795d47466a54a5c846d691693"
[[package]]
name = "bitvec"
@@ -307,26 +307,6 @@ dependencies = [
"vec_map",
]
[[package]]
name = "const_format"
version = "0.2.21"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4556f63e28a78fa5e6f310cfea5647a25636def49a338ab69e33b34a3382057b"
dependencies = [
"const_format_proc_macros",
]
[[package]]
name = "const_format_proc_macros"
version = "0.2.21"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "552782506c398da94466b364973b563887e0ca078bf33a76d4163736165e3594"
dependencies = [
"proc-macro2",
"quote",
"unicode-xid",
]
[[package]]
name = "control_plane"
version = "0.1.0"
@@ -335,7 +315,7 @@ dependencies = [
"bytes",
"hex",
"lazy_static",
"nix 0.20.0",
"nix",
"pageserver",
"postgres",
"postgres_ffi",
@@ -924,9 +904,9 @@ checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55"
[[package]]
name = "libc"
version = "0.2.103"
version = "0.2.101"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6"
checksum = "3cb00336871be5ed2c8ed44b60ae9959dc5b9f08539422ed43f09e34ecaeba21"
[[package]]
name = "libloading"
@@ -1072,19 +1052,6 @@ dependencies = [
"libc",
]
[[package]]
name = "nix"
version = "0.23.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f305c2c2e4c39a82f7bf0bf65fb557f9070ce06781d4f2454295cc34b1c43188"
dependencies = [
"bitflags",
"cc",
"cfg-if 1.0.0",
"libc",
"memoffset",
]
[[package]]
name = "nom"
version = "6.1.2"
@@ -1212,17 +1179,14 @@ dependencies = [
"bytes",
"chrono",
"clap",
"const_format",
"crc32c",
"daemonize",
"futures",
"hex",
"hex-literal",
"humantime",
"hyper",
"lazy_static",
"log",
"nix 0.23.0",
"postgres",
"postgres-protocol",
"postgres-types",
@@ -1442,7 +1406,6 @@ dependencies = [
"hex",
"md5",
"rand",
"reqwest",
"rustls",
"serde",
"serde_json",
@@ -2367,7 +2330,6 @@ dependencies = [
"regex",
"rust-s3",
"serde",
"serde_json",
"tokio",
"tokio-stream",
"walkdir",
@@ -2592,7 +2554,6 @@ version = "0.1.0"
dependencies = [
"lazy_static",
"libc",
"once_cell",
"prometheus",
]
@@ -2622,7 +2583,6 @@ dependencies = [
"slog-scope",
"slog-stdlog",
"slog-term",
"tempfile",
"thiserror",
"tokio",
"webpki",

View File

@@ -11,19 +11,38 @@ WORKDIR /zenith
COPY ./vendor/postgres vendor/postgres
COPY ./Makefile Makefile
RUN make -j $(getconf _NPROCESSORS_ONLN) -s postgres
RUN rm -rf postgres_install/build
#
# Calculate cargo dependencies.
# This will always run, but only generate recipe.json with list of dependencies without
# installing them.
#
FROM zenithdb/build:buster AS cargo-deps-inspect
WORKDIR /zenith
COPY . .
RUN cargo chef prepare --recipe-path /zenith/recipe.json
#
# Build cargo dependencies.
# This temp cantainner should be rebuilt only if recipe.json was changed.
#
FROM zenithdb/build:buster AS deps-build
WORKDIR /zenith
COPY --from=pg-build /zenith/tmp_install/include/postgresql/server tmp_install/include/postgresql/server
COPY --from=cargo-deps-inspect /usr/local/cargo/bin/cargo-chef /usr/local/cargo/bin/
COPY --from=cargo-deps-inspect /zenith/recipe.json recipe.json
RUN ROCKSDB_LIB_DIR=/usr/lib/ cargo chef cook --release --recipe-path recipe.json
#
# Build zenith binaries
#
# TODO: build cargo deps as separate layer. We used cargo-chef before but that was
# net time waste in a lot of cases. Copying Cargo.lock with empty lib.rs should do the work.
#
FROM zenithdb/build:buster AS build
WORKDIR /zenith
COPY --from=pg-build /zenith/tmp_install/include/postgresql/server tmp_install/include/postgresql/server
COPY . .
# Copy cached dependencies
COPY --from=pg-build /zenith/tmp_install/include/postgresql/server tmp_install/include/postgresql/server
COPY --from=deps-build /zenith/target target
COPY --from=deps-build /usr/local/cargo/ /usr/local/cargo/
RUN cargo build --release
#
@@ -32,7 +51,7 @@ RUN cargo build --release
FROM debian:buster-slim
WORKDIR /data
RUN apt-get update && apt-get -yq install libreadline-dev libseccomp-dev openssl ca-certificates && \
RUN apt-get update && apt-get -yq install librocksdb-dev libseccomp-dev openssl && \
mkdir zenith_install
COPY --from=build /zenith/target/release/pageserver /usr/local/bin

View File

@@ -9,7 +9,7 @@ WORKDIR /zenith
# Install postgres and zenith build dependencies
# clang is for rocksdb
RUN apt-get update && apt-get -yq install automake libtool build-essential bison flex libreadline-dev zlib1g-dev libxml2-dev \
libseccomp-dev pkg-config libssl-dev clang
libseccomp-dev pkg-config libssl-dev librocksdb-dev clang
# Install rust tools
RUN rustup component add clippy && cargo install cargo-audit
RUN rustup component add clippy && cargo install cargo-chef cargo-audit

View File

@@ -25,7 +25,7 @@ Pageserver consists of:
On Ubuntu or Debian this set of packages should be sufficient to build the code:
```text
apt install build-essential libtool libreadline-dev zlib1g-dev flex bison libseccomp-dev \
libssl-dev clang pkg-config libpq-dev
libssl-dev clang
```
[Rust] 1.52 or later is also required.
@@ -108,13 +108,6 @@ postgres=# insert into t values(2,2);
INSERT 0 1
```
6. If you want to run tests afterwards (see below), you have to stop pageserver and all postgres instances you have just started:
```sh
> ./target/debug/zenith pg stop migration_check
> ./target/debug/zenith pg stop main
> ./target/debug/zenith stop
```
## Running tests
```sh

View File

@@ -1,16 +1,17 @@
use std::collections::BTreeMap;
use std::fs::{self, File};
use std::fs::{self, File, OpenOptions};
use std::io::Write;
use std::net::SocketAddr;
use std::net::TcpStream;
use std::os::unix::fs::PermissionsExt;
use std::path::PathBuf;
use std::process::{Command, Stdio};
use std::str::FromStr;
use std::sync::Arc;
use std::time::Duration;
use std::{collections::BTreeMap, path::PathBuf};
use anyhow::{Context, Result};
use lazy_static::lazy_static;
use regex::Regex;
use zenith_utils::connstring::connection_host_port;
use zenith_utils::lsn::Lsn;
use zenith_utils::postgres_backend::AuthType;
@@ -18,7 +19,6 @@ use zenith_utils::zid::ZTenantId;
use zenith_utils::zid::ZTimelineId;
use crate::local_env::LocalEnv;
use crate::postgresql_conf::PostgresConf;
use crate::storage::PageServerNode;
//
@@ -144,25 +144,76 @@ impl PostgresNode {
);
}
lazy_static! {
static ref CONF_PORT_RE: Regex = Regex::new(r"(?m)^\s*port\s*=\s*(\d+)\s*$").unwrap();
static ref CONF_TIMELINE_RE: Regex =
Regex::new(r"(?m)^\s*zenith.zenith_timeline\s*=\s*'(\w+)'\s*$").unwrap();
static ref CONF_TENANT_RE: Regex =
Regex::new(r"(?m)^\s*zenith.zenith_tenant\s*=\s*'(\w+)'\s*$").unwrap();
}
// parse data directory name
let fname = entry.file_name();
let name = fname.to_str().unwrap().to_string();
// Read config file into memory
// find out tcp port in config file
let cfg_path = entry.path().join("postgresql.conf");
let cfg_path_str = cfg_path.to_string_lossy();
let mut conf_file = File::open(&cfg_path)
.with_context(|| format!("failed to open config file in {}", cfg_path_str))?;
let conf = PostgresConf::read(&mut conf_file)
.with_context(|| format!("failed to read config file in {}", cfg_path_str))?;
let config = fs::read_to_string(cfg_path.clone()).with_context(|| {
format!(
"failed to read config file in {}",
cfg_path.to_str().unwrap()
)
})?;
// Read a few options from the config file
let context = format!("in config file {}", cfg_path_str);
let port: u16 = conf.parse_field("port", &context)?;
let timelineid: ZTimelineId = conf.parse_field("zenith.zenith_timeline", &context)?;
let tenantid: ZTenantId = conf.parse_field("zenith.zenith_tenant", &context)?;
// parse port
let err_msg = format!(
"failed to find port definition in config file {}",
cfg_path.to_str().unwrap()
);
let port: u16 = CONF_PORT_RE
.captures(config.as_str())
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 1"))?
.iter()
.last()
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 2"))?
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 3"))?
.as_str()
.parse()
.with_context(|| err_msg)?;
let uses_wal_proposer = conf.get("wal_acceptors").is_some();
// parse timeline
let err_msg = format!(
"failed to find timeline definition in config file {}",
cfg_path.to_str().unwrap()
);
let timelineid: ZTimelineId = CONF_TIMELINE_RE
.captures(config.as_str())
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 1"))?
.iter()
.last()
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 2"))?
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 3"))?
.as_str()
.parse()
.with_context(|| err_msg)?;
// parse tenant
let err_msg = format!(
"failed to find tenant definition in config file {}",
cfg_path.to_str().unwrap()
);
let tenantid = CONF_TENANT_RE
.captures(config.as_str())
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 1"))?
.iter()
.last()
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 2"))?
.ok_or_else(|| anyhow::Error::msg(err_msg.clone() + " 3"))?
.as_str()
.parse()
.with_context(|| err_msg)?;
let uses_wal_proposer = config.contains("wal_acceptors");
// ok now
Ok(PostgresNode {
@@ -257,58 +308,74 @@ impl PostgresNode {
// Connect to a page server, get base backup, and untar it to initialize a
// new data directory
fn setup_pg_conf(&self, auth_type: AuthType) -> Result<()> {
let mut conf = PostgresConf::new();
conf.append("max_wal_senders", "10");
File::create(self.pgdata().join("postgresql.conf").to_str().unwrap())?;
// wal_log_hints is mandatory when running against pageserver (see gh issue#192)
// TODO: is it possible to check wal_log_hints at pageserver side via XLOG_PARAMETER_CHANGE?
conf.append("wal_log_hints", "on");
conf.append("max_replication_slots", "10");
conf.append("hot_standby", "on");
conf.append("shared_buffers", "1MB");
conf.append("fsync", "off");
conf.append("max_connections", "100");
conf.append("wal_sender_timeout", "0");
conf.append("wal_level", "replica");
conf.append("listen_addresses", &self.address.ip().to_string());
conf.append("port", &self.address.port().to_string());
self.append_conf(
"postgresql.conf",
&format!(
"max_wal_senders = 10\n\
wal_log_hints = on\n\
max_replication_slots = 10\n\
hot_standby = on\n\
shared_buffers = 1MB\n\
fsync = off\n\
max_connections = 100\n\
wal_sender_timeout = 0\n\
wal_level = replica\n\
zenith.file_cache_size = 4096\n\
zenith.file_cache_path = '/tmp/file.cache'\n\
listen_addresses = '{address}'\n\
port = {port}\n",
address = self.address.ip(),
port = self.address.port()
),
)?;
// Never clean up old WAL. TODO: We should use a replication
// slot or something proper, to prevent the compute node
// from removing WAL that hasn't been streamed to the safekeeper or
// page server yet. (gh issue #349)
conf.append("wal_keep_size", "10TB");
self.append_conf("postgresql.conf", "wal_keep_size='10TB'\n")?;
// Configure the node to fetch pages from pageserver
let pageserver_connstr = {
let (host, port) = connection_host_port(&self.pageserver.pg_connection_config);
// Set up authentication
//
// $ZENITH_AUTH_TOKEN will be replaced with value from environment
// variable during compute pg startup. It is done this way because
// otherwise user will be able to retrieve the value using SHOW
// command or pg_settings
let password = if let AuthType::ZenithJWT = auth_type {
"$ZENITH_AUTH_TOKEN"
} else {
""
};
format!("host={} port={} password={}", host, port, password)
// set up authentication
let password = if let AuthType::ZenithJWT = auth_type {
"$ZENITH_AUTH_TOKEN"
} else {
""
};
conf.append("shared_preload_libraries", "zenith");
conf.append_line("");
conf.append("zenith.page_server_connstring", &pageserver_connstr);
conf.append("zenith.zenith_tenant", &self.tenantid.to_string());
conf.append("zenith.zenith_timeline", &self.timelineid.to_string());
conf.append_line("");
// Configure that node to take pages from pageserver
let (host, port) = connection_host_port(&self.pageserver.pg_connection_config);
self.append_conf(
"postgresql.conf",
format!(
concat!(
"shared_preload_libraries = zenith\n",
// $ZENITH_AUTH_TOKEN will be replaced with value from environment variable during compute pg startup
// it is done this way because otherwise user will be able to retrieve the value using SHOW command or pg_settings
"zenith.page_server_connstring = 'host={} port={} password={}'\n",
"zenith.zenith_timeline='{}'\n",
"zenith.zenith_tenant='{}'\n",
),
host, port, password, self.timelineid, self.tenantid,
)
.as_str(),
)?;
// Configure the node to stream WAL directly to the pageserver
conf.append("synchronous_standby_names", "pageserver"); // TODO: add a new function arg?
conf.append("zenith.callmemaybe_connstring", &self.connstr());
let mut file = File::create(self.pgdata().join("postgresql.conf"))?;
file.write_all(conf.to_string().as_bytes())?;
self.append_conf(
"postgresql.conf",
format!(
concat!(
"synchronous_standby_names = 'pageserver'\n", // TODO: add a new function arg?
"zenith.callmemaybe_connstring = '{}'\n", // FIXME escaping
),
self.connstr(),
)
.as_str(),
)?;
Ok(())
}
@@ -351,6 +418,14 @@ impl PostgresNode {
}
}
pub fn append_conf(&self, config: &str, opts: &str) -> Result<()> {
OpenOptions::new()
.append(true)
.open(self.pgdata().join(config).to_str().unwrap())?
.write_all(opts.as_bytes())?;
Ok(())
}
fn pg_ctl(&self, args: &[&str], auth_token: &Option<String>) -> Result<()> {
let pg_ctl_path = self.env.pg_bin_dir().join("pg_ctl");
let mut cmd = Command::new(pg_ctl_path);
@@ -452,7 +527,9 @@ impl PostgresNode {
.output()
.expect("failed to execute whoami");
assert!(output.status.success(), "whoami failed");
if !output.status.success() {
panic!("whoami failed");
}
String::from_utf8(output.stdout).unwrap().trim().to_string()
}

View File

@@ -12,7 +12,6 @@ use std::path::Path;
pub mod compute;
pub mod local_env;
pub mod postgresql_conf;
pub mod storage;
/// Read a PID file

View File

@@ -1,212 +0,0 @@
///
/// Module for parsing postgresql.conf file.
///
/// NOTE: This doesn't implement the full, correct postgresql.conf syntax. Just
/// enough to extract a few settings we need in Zenith, assuming you don't do
/// funny stuff like include-directives or funny escaping.
use anyhow::{anyhow, bail, Context, Result};
use lazy_static::lazy_static;
use regex::Regex;
use std::collections::HashMap;
use std::fmt;
use std::io::BufRead;
use std::str::FromStr;
/// In-memory representation of a postgresql.conf file
#[derive(Default)]
pub struct PostgresConf {
lines: Vec<String>,
hash: HashMap<String, String>,
}
lazy_static! {
static ref CONF_LINE_RE: Regex = Regex::new(r"^((?:\w|\.)+)\s*=\s*(\S+)$").unwrap();
}
impl PostgresConf {
pub fn new() -> PostgresConf {
PostgresConf::default()
}
/// Read file into memory
pub fn read(read: impl std::io::Read) -> Result<PostgresConf> {
let mut result = Self::new();
for line in std::io::BufReader::new(read).lines() {
let line = line?;
// Store each line in a vector, in original format
result.lines.push(line.clone());
// Also parse each line and insert key=value lines into a hash map.
//
// FIXME: This doesn't match exactly the flex/bison grammar in PostgreSQL.
// But it's close enough for our usage.
let line = line.trim();
if line.starts_with('#') {
// comment, ignore
continue;
} else if let Some(caps) = CONF_LINE_RE.captures(line) {
let name = caps.get(1).unwrap().as_str();
let raw_val = caps.get(2).unwrap().as_str();
if let Ok(val) = deescape_str(raw_val) {
// Note: if there's already an entry in the hash map for
// this key, this will replace it. That's the behavior what
// we want; when PostgreSQL reads the file, each line
// overrides any previous value for the same setting.
result.hash.insert(name.to_string(), val.to_string());
}
}
}
Ok(result)
}
/// Return the current value of 'option'
pub fn get(&self, option: &str) -> Option<&str> {
self.hash.get(option).map(|x| x.as_ref())
}
/// Return the current value of a field, parsed to the right datatype.
///
/// This calls the FromStr::parse() function on the value of the field. If
/// the field does not exist, or parsing fails, returns an error.
///
pub fn parse_field<T>(&self, field_name: &str, context: &str) -> Result<T>
where
T: FromStr,
<T as FromStr>::Err: std::error::Error + Send + Sync + 'static,
{
self.get(field_name)
.ok_or_else(|| anyhow!("could not find '{}' option {}", field_name, context))?
.parse::<T>()
.with_context(|| format!("could not parse '{}' option {}", field_name, context))
}
///
/// Note: if you call this multiple times for the same option, the config
/// file will a line for each call. It would be nice to have a function
/// to change an existing line, but that's a TODO.
///
pub fn append(&mut self, option: &str, value: &str) {
self.lines
.push(format!("{}={}\n", option, escape_str(value)));
self.hash.insert(option.to_string(), value.to_string());
}
/// Append an arbitrary non-setting line to the config file
pub fn append_line(&mut self, line: &str) {
self.lines.push(line.to_string());
}
}
impl fmt::Display for PostgresConf {
/// Return the whole configuration file as a string
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
for line in self.lines.iter() {
f.write_str(line)?;
}
Ok(())
}
}
/// Escape a value for putting in postgresql.conf.
fn escape_str(s: &str) -> String {
// If the string doesn't contain anything that needs quoting or escaping, return it
// as it is.
//
// The first part of the regex, before the '|', matches the INTEGER rule in the
// PostgreSQL flex grammar (guc-file.l). It matches plain integers like "123" and
// "-123", and also accepts units like "10MB". The second part of the regex matches
// the UNQUOTED_STRING rule, and accepts strings that contain a single word, beginning
// with a letter. That covers words like "off" or "posix". Everything else is quoted.
//
// This regex is a bit more conservative than the rules in guc-file.l, so we quote some
// strings that PostgreSQL would accept without quoting, but that's OK.
lazy_static! {
static ref UNQUOTED_RE: Regex =
Regex::new(r"(^[-+]?[0-9]+[a-zA-Z]*$)|(^[a-zA-Z][a-zA-Z0-9]*$)").unwrap();
}
if UNQUOTED_RE.is_match(s) {
s.to_string()
} else {
// Otherwise escape and quote it
let s = s
.replace('\\', "\\\\")
.replace('\n', "\\n")
.replace('\'', "''");
"\'".to_owned() + &s + "\'"
}
}
/// De-escape a possibly-quoted value.
///
/// See `DeescapeQuotedString` function in PostgreSQL sources for how PostgreSQL
/// does this.
fn deescape_str(s: &str) -> Result<String> {
// If the string has a quote at the beginning and end, strip them out.
if s.len() >= 2 && s.starts_with('\'') && s.ends_with('\'') {
let mut result = String::new();
let mut iter = s[1..(s.len() - 1)].chars().peekable();
while let Some(c) = iter.next() {
let newc = if c == '\\' {
match iter.next() {
Some('b') => '\x08',
Some('f') => '\x0c',
Some('n') => '\n',
Some('r') => '\r',
Some('t') => '\t',
Some('0'..='7') => {
// TODO
bail!("octal escapes not supported");
}
Some(n) => n,
None => break,
}
} else if c == '\'' && iter.peek() == Some(&'\'') {
// doubled quote becomes just one quote
iter.next().unwrap()
} else {
c
};
result.push(newc);
}
Ok(result)
} else {
Ok(s.to_string())
}
}
#[test]
fn test_postgresql_conf_escapes() -> Result<()> {
assert_eq!(escape_str("foo bar"), "'foo bar'");
// these don't need to be quoted
assert_eq!(escape_str("foo"), "foo");
assert_eq!(escape_str("123"), "123");
assert_eq!(escape_str("+123"), "+123");
assert_eq!(escape_str("-10"), "-10");
assert_eq!(escape_str("1foo"), "1foo");
assert_eq!(escape_str("foo1"), "foo1");
assert_eq!(escape_str("10MB"), "10MB");
assert_eq!(escape_str("-10kB"), "-10kB");
// these need quoting and/or escaping
assert_eq!(escape_str("foo bar"), "'foo bar'");
assert_eq!(escape_str("fo'o"), "'fo''o'");
assert_eq!(escape_str("fo\no"), "'fo\\no'");
assert_eq!(escape_str("fo\\o"), "'fo\\\\o'");
assert_eq!(escape_str("10 cats"), "'10 cats'");
// Test de-escaping
assert_eq!(deescape_str(&escape_str("foo"))?, "foo");
assert_eq!(deescape_str(&escape_str("fo'o\nba\\r"))?, "fo'o\nba\\r");
assert_eq!(deescape_str("'\\b\\f\\n\\r\\t'")?, "\x08\x0c\n\r\t");
// octal-escapes are currently not supported
assert!(deescape_str("'foo\\7\\07\\007'").is_err());
Ok(())
}

View File

@@ -10,5 +10,5 @@
- [pageserver/README](/pageserver/README) — pageserver overview.
- [postgres_ffi/README](/postgres_ffi/README) — Postgres FFI overview.
- [test_runner/README.md](/test_runner/README.md) — tests infrastructure overview.
- [walkeeper/README](/walkeeper/README) — WAL service overview.
- [walkeeper/README](/walkeeper/README.md) — WAL service overview.
- [core_changes.md](core_changes.md) - Description of Zenith changes in Postgres core

View File

@@ -34,13 +34,8 @@ toml = "0.5"
scopeguard = "1.1.0"
rust-s3 = { version = "0.27.0-rc4", features = ["no-verify-ssl"] }
async-trait = "0.1"
const_format = "0.2.21"
nix = "0.23.0"
postgres_ffi = { path = "../postgres_ffi" }
zenith_metrics = { path = "../zenith_metrics" }
zenith_utils = { path = "../zenith_utils" }
workspace_hack = { path = "../workspace_hack" }
[dev-dependencies]
hex-literal = "0.3"

View File

@@ -7,9 +7,8 @@ The Page Server has a few different duties:
- Replay WAL that's applicable to the chunks that the Page Server maintains
- Backup to S3
S3 is the main fault-tolerant storage of all data, as there are no Page Server
replicas. We use a separate fault-tolerant WAL service to reduce latency. It
keeps track of WAL records which are not syncted to S3 yet.
The Page Server consists of multiple threads that operate on a shared
repository of page versions:

View File

@@ -9,28 +9,22 @@ use std::{
env,
net::TcpListener,
path::{Path, PathBuf},
process::exit,
str::FromStr,
thread,
};
use zenith_utils::{auth::JwtAuth, logging, postgres_backend::AuthType};
use anyhow::{bail, ensure, Context, Result};
use anyhow::{bail, ensure, Result};
use clap::{App, Arg, ArgMatches};
use daemonize::Daemonize;
use pageserver::{
branches,
defaults::{
DEFAULT_HTTP_LISTEN_ADDR, DEFAULT_PG_LISTEN_ADDR,
DEFAULT_RELISH_STORAGE_MAX_CONCURRENT_SYNC_LIMITS,
},
http, page_service, relish_storage, tenant_mgr, PageServerConf, RelishStorageConfig,
RelishStorageKind, S3Config, LOG_FILE_NAME,
branches, http, page_service, tenant_mgr, PageServerConf, RelishStorageConfig, S3Config,
LOG_FILE_NAME,
};
use zenith_utils::http::endpoint;
use const_format::formatcp;
/// String arguments that can be declared via CLI or config file
#[derive(Serialize, Deserialize)]
struct CfgFileParams {
@@ -45,7 +39,6 @@ struct CfgFileParams {
auth_type: Option<String>,
// see https://github.com/alexcrichton/toml-rs/blob/6c162e6562c3e432bf04c82a3d1d789d80761a86/examples/enum_external.rs for enum deserialisation examples
relish_storage: Option<RelishStorage>,
relish_storage_max_concurrent_sync: Option<String>,
}
#[derive(Serialize, Deserialize, Clone)]
@@ -96,7 +89,6 @@ impl CfgFileParams {
auth_validation_public_key_path: get_arg("auth-validation-public-key-path"),
auth_type: get_arg("auth-type"),
relish_storage,
relish_storage_max_concurrent_sync: get_arg("relish-storage-max-concurrent-sync"),
}
}
@@ -116,9 +108,6 @@ impl CfgFileParams {
.or(other.auth_validation_public_key_path),
auth_type: self.auth_type.or(other.auth_type),
relish_storage: self.relish_storage.or(other.relish_storage),
relish_storage_max_concurrent_sync: self
.relish_storage_max_concurrent_sync
.or(other.relish_storage_max_concurrent_sync),
}
}
@@ -187,34 +176,25 @@ impl CfgFileParams {
);
}
let max_concurrent_sync = match self.relish_storage_max_concurrent_sync.as_deref() {
Some(relish_storage_max_concurrent_sync) => {
relish_storage_max_concurrent_sync.parse()?
}
None => DEFAULT_RELISH_STORAGE_MAX_CONCURRENT_SYNC_LIMITS,
};
let relish_storage_config = self.relish_storage.as_ref().map(|storage_params| {
let storage = match storage_params.clone() {
RelishStorage::Local { local_path } => {
RelishStorageKind::LocalFs(PathBuf::from(local_path))
}
RelishStorage::AwsS3 {
bucket_name,
bucket_region,
access_key_id,
secret_access_key,
} => RelishStorageKind::AwsS3(S3Config {
bucket_name,
bucket_region,
access_key_id,
secret_access_key,
}),
};
RelishStorageConfig {
max_concurrent_sync,
storage,
}
});
let relish_storage_config =
self.relish_storage
.as_ref()
.map(|storage_params| match storage_params.clone() {
RelishStorage::Local { local_path } => {
RelishStorageConfig::LocalFs(PathBuf::from(local_path))
}
RelishStorage::AwsS3 {
bucket_name,
bucket_region,
access_key_id,
secret_access_key,
} => RelishStorageConfig::AwsS3(S3Config {
bucket_name,
bucket_region,
access_key_id,
secret_access_key,
}),
});
Ok(PageServerConf {
daemonize: false,
@@ -240,7 +220,6 @@ impl CfgFileParams {
}
fn main() -> Result<()> {
zenith_metrics::set_common_metrics_prefix("pageserver");
let arg_matches = App::new("Zenith page server")
.about("Materializes WAL stream to pages and serves them to the postgres")
.arg(
@@ -249,14 +228,14 @@ fn main() -> Result<()> {
.long("listen-pg")
.alias("listen") // keep some compatibility
.takes_value(true)
.help(formatcp!("listen for incoming page requests on ip:port (default: {DEFAULT_PG_LISTEN_ADDR})")),
.help("listen for incoming page requests on ip:port (default: 127.0.0.1:5430)"),
)
.arg(
Arg::with_name("listen-http")
.long("listen-http")
.alias("http_endpoint") // keep some compatibility
.takes_value(true)
.help(formatcp!("http endpoint address for metrics and management API calls on ip:port (default: {DEFAULT_HTTP_LISTEN_ADDR})")),
.help("http endpoint address for for metrics and management API calls ip:port (default: 127.0.0.1:5430)"),
)
.arg(
Arg::with_name("daemonize")
@@ -364,19 +343,10 @@ fn main() -> Result<()> {
.takes_value(true)
.help("Credentials to access the AWS S3 bucket"),
)
.arg(
Arg::with_name("relish-storage-max-concurrent-sync")
.long("relish-storage-max-concurrent-sync")
.takes_value(true)
.help("Maximum allowed concurrent synchronisations with storage"),
)
.get_matches();
let workdir = Path::new(arg_matches.value_of("workdir").unwrap_or(".zenith"));
let cfg_file_path = workdir
.canonicalize()
.with_context(|| format!("Error opening workdir '{}'", workdir.display()))?
.join("pageserver.toml");
let cfg_file_path = workdir.canonicalize()?.join("pageserver.toml");
let args_params = CfgFileParams::from_args(&arg_matches);
@@ -388,37 +358,22 @@ fn main() -> Result<()> {
args_params
} else {
// Supplement the CLI arguments with the config file
let cfg_file_contents = std::fs::read_to_string(&cfg_file_path)
.with_context(|| format!("No pageserver config at '{}'", cfg_file_path.display()))?;
let file_params: CfgFileParams = toml::from_str(&cfg_file_contents).with_context(|| {
format!(
"Failed to read '{}' as pageserver config",
cfg_file_path.display()
)
})?;
let cfg_file_contents = std::fs::read_to_string(&cfg_file_path)?;
let file_params: CfgFileParams = toml::from_str(&cfg_file_contents)?;
args_params.or(file_params)
};
// Set CWD to workdir for non-daemon modes
env::set_current_dir(&workdir).with_context(|| {
format!(
"Failed to set application's current dir to '{}'",
workdir.display()
)
})?;
env::set_current_dir(&workdir)?;
// Ensure the config is valid, even if just init-ing
let mut conf = params.try_into_config().with_context(|| {
format!(
"Pageserver config at '{}' is not valid",
cfg_file_path.display()
)
})?;
let mut conf = params.try_into_config()?;
conf.daemonize = arg_matches.is_present("daemonize");
if init && conf.daemonize {
bail!("--daemonize cannot be used with --init")
eprintln!("--daemonize cannot be used with --init");
exit(1);
}
// The configuration is all set up now. Turn it into a 'static
@@ -428,21 +383,16 @@ fn main() -> Result<()> {
// Create repo and exit if init was requested
if init {
branches::init_pageserver(conf, create_tenant).context("Failed to init pageserver")?;
branches::init_pageserver(conf, create_tenant)?;
// write the config file
let cfg_file_contents = toml::to_string_pretty(&params)
.context("Failed to create pageserver config contents for initialisation")?;
let cfg_file_contents = toml::to_string_pretty(&params)?;
// TODO support enable-auth flag
std::fs::write(&cfg_file_path, cfg_file_contents).with_context(|| {
format!(
"Failed to initialize pageserver config at '{}'",
cfg_file_path.display()
)
})?;
Ok(())
} else {
start_pageserver(conf).context("Failed to start pageserver")
std::fs::write(&cfg_file_path, cfg_file_contents)?;
return Ok(());
}
start_pageserver(conf)
}
fn start_pageserver(conf: &'static PageServerConf) -> Result<()> {
@@ -480,20 +430,16 @@ fn start_pageserver(conf: &'static PageServerConf) -> Result<()> {
match daemonize.start() {
Ok(_) => info!("Success, daemonized"),
Err(e) => error!("could not daemonize: {:#}", e),
Err(e) => error!("Error, {}", e),
}
}
// keep join handles for spawned threads
// don't spawn threads before daemonizing
let mut join_handles = Vec::new();
if let Some(handle) = relish_storage::run_storage_sync_thread(conf)? {
join_handles.push(handle);
}
// Initialize tenant manager.
tenant_mgr::init(conf);
// keep join handles for spawned threads
let mut join_handles = vec![];
// initialize authentication for incoming connections
let auth = match &conf.auth_type {
AuthType::Trust | AuthType::MD5 => None,

View File

@@ -17,7 +17,6 @@ use std::{
use zenith_utils::zid::{ZTenantId, ZTimelineId};
use log::*;
use zenith_utils::crashsafe_dir;
use zenith_utils::logging;
use zenith_utils::lsn::Lsn;
@@ -119,7 +118,7 @@ pub fn init_pageserver(conf: &'static PageServerConf, create_tenant: Option<&str
println!("initializing tenantid {}", tenantid);
create_repo(conf, tenantid, dummy_redo_mgr).with_context(|| "failed to create repo")?;
}
crashsafe_dir::create_dir_all(conf.tenants_path())?;
fs::create_dir_all(conf.tenants_path())?;
println!("pageserver init succeeded");
Ok(())
@@ -136,12 +135,12 @@ pub fn create_repo(
}
// top-level dir may exist if we are creating it through CLI
crashsafe_dir::create_dir_all(&repo_dir)
fs::create_dir_all(&repo_dir)
.with_context(|| format!("could not create directory {}", repo_dir.display()))?;
crashsafe_dir::create_dir(conf.timelines_path(&tenantid))?;
crashsafe_dir::create_dir_all(conf.branches_path(&tenantid))?;
crashsafe_dir::create_dir_all(conf.tags_path(&tenantid))?;
fs::create_dir(conf.timelines_path(&tenantid))?;
fs::create_dir_all(conf.branches_path(&tenantid))?;
fs::create_dir_all(conf.tags_path(&tenantid))?;
info!("created directory structure in {}", repo_dir.display());
@@ -151,13 +150,12 @@ pub fn create_repo(
conf,
wal_redo_manager,
tenantid,
false,
));
// Load data into pageserver
// TODO To implement zenith import we need to
// move data loading out of create_repo()
bootstrap_timeline(conf, tenantid, tli, repo.as_ref())?;
bootstrap_timeline(conf, tenantid, tli, &*repo)?;
Ok(repo)
}
@@ -223,11 +221,7 @@ fn bootstrap_timeline(
// Import the contents of the data directory at the initial checkpoint
// LSN, and any WAL after that.
let timeline = repo.create_empty_timeline(tli)?;
restore_local_repo::import_timeline_from_postgres_datadir(
&pgdata_path,
timeline.as_ref(),
lsn,
)?;
restore_local_repo::import_timeline_from_postgres_datadir(&pgdata_path, &*timeline, lsn)?;
timeline.checkpoint()?;
println!(

View File

@@ -11,7 +11,7 @@
//! parent timeline, and the last LSN that has been written to disk.
//!
use anyhow::{anyhow, bail, ensure, Context, Result};
use anyhow::{bail, Context, Result};
use bookfile::Book;
use bytes::Bytes;
use lazy_static::lazy_static;
@@ -22,8 +22,7 @@ use serde::{Deserialize, Serialize};
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::collections::{BTreeSet, HashSet};
use std::convert::TryInto;
use std::fs::{File, OpenOptions};
use std::fs::File;
use std::io::Write;
use std::ops::Bound::Included;
use std::path::{Path, PathBuf};
@@ -34,9 +33,8 @@ use std::{fs, thread};
use crate::layered_repository::inmemory_layer::FreezeLayers;
use crate::relish::*;
use crate::relish_storage::schedule_timeline_upload;
use crate::relish_storage::storage_uploader::QueueBasedRelishUploader;
use crate::repository::{GcResult, Repository, Timeline, WALRecord};
use crate::walreceiver::IS_WAL_RECEIVER;
use crate::walredo::WalRedoManager;
use crate::PageServerConf;
use crate::{ZTenantId, ZTimelineId};
@@ -46,7 +44,6 @@ use zenith_metrics::{
};
use zenith_metrics::{register_histogram_vec, HistogramVec};
use zenith_utils::bin_ser::BeSer;
use zenith_utils::crashsafe_dir;
use zenith_utils::lsn::{AtomicLsn, Lsn, RecordLsn};
use zenith_utils::seqwait::SeqWait;
@@ -55,7 +52,6 @@ mod delta_layer;
mod filename;
mod image_layer;
mod inmemory_layer;
mod interval_tree;
mod layer_map;
mod storage_layer;
@@ -75,11 +71,6 @@ static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; 8192]);
// Timeout when waiting for WAL receiver to catch up to an LSN given in a GetPage@LSN call.
static TIMEOUT: Duration = Duration::from_secs(60);
// Taken from PG_CONTROL_MAX_SAFE_SIZE
const METADATA_MAX_SAFE_SIZE: usize = 512;
const METADATA_CHECKSUM_SIZE: usize = std::mem::size_of::<u32>();
const METADATA_MAX_DATA_SIZE: usize = METADATA_MAX_SAFE_SIZE - METADATA_CHECKSUM_SIZE;
// Metrics collected on operations on the storage repository.
lazy_static! {
static ref STORAGE_TIME: HistogramVec = register_histogram_vec!(
@@ -119,9 +110,7 @@ pub struct LayeredRepository {
timelines: Mutex<HashMap<ZTimelineId, Arc<LayeredTimeline>>>,
walredo_mgr: Arc<dyn WalRedoManager + Send + Sync>,
/// Makes evey repo's timelines to backup their files to remote storage,
/// when they get frozen.
upload_relishes: bool,
relish_uploader: Option<Arc<QueueBasedRelishUploader>>,
}
/// Public interface
@@ -136,7 +125,7 @@ impl Repository for LayeredRepository {
let mut timelines = self.timelines.lock().unwrap();
// Create the timeline directory, and write initial metadata to file.
crashsafe_dir::create_dir_all(self.conf.timeline_path(&timelineid, &self.tenantid))?;
std::fs::create_dir_all(self.conf.timeline_path(&timelineid, &self.tenantid))?;
let metadata = TimelineMetadata {
disk_consistent_lsn: Lsn(0),
@@ -144,7 +133,7 @@ impl Repository for LayeredRepository {
ancestor_timeline: None,
ancestor_lsn: Lsn(0),
};
Self::save_metadata(self.conf, timelineid, self.tenantid, &metadata, true)?;
Self::save_metadata(self.conf, timelineid, self.tenantid, &metadata)?;
let timeline = LayeredTimeline::new(
self.conf,
@@ -153,8 +142,8 @@ impl Repository for LayeredRepository {
timelineid,
self.tenantid,
Arc::clone(&self.walredo_mgr),
self.relish_uploader.as_ref().map(Arc::clone),
0,
false,
)?;
let timeline_rc = Arc::new(timeline);
@@ -188,8 +177,8 @@ impl Repository for LayeredRepository {
ancestor_timeline: Some(src),
ancestor_lsn: start_lsn,
};
crashsafe_dir::create_dir_all(self.conf.timeline_path(&dst, &self.tenantid))?;
Self::save_metadata(self.conf, dst, self.tenantid, &metadata, true)?;
std::fs::create_dir_all(self.conf.timeline_path(&dst, &self.tenantid))?;
Self::save_metadata(self.conf, dst, self.tenantid, &metadata)?;
info!("branched timeline {} from {} at {}", dst, src, start_lsn);
@@ -203,12 +192,12 @@ impl Repository for LayeredRepository {
&self,
target_timelineid: Option<ZTimelineId>,
horizon: u64,
checkpoint_before_gc: bool,
compact: bool,
) -> Result<GcResult> {
STORAGE_TIME
.with_label_values(&["gc"])
.observe_closure_duration(|| {
self.gc_iteration_internal(target_timelineid, horizon, checkpoint_before_gc)
self.gc_iteration_internal(target_timelineid, horizon, compact)
})
}
}
@@ -226,7 +215,6 @@ impl LayeredRepository {
Some(timeline) => Ok(timeline.clone()),
None => {
let metadata = Self::load_metadata(self.conf, timelineid, self.tenantid)?;
let disk_consistent_lsn = metadata.disk_consistent_lsn;
// Recurse to look up the ancestor timeline.
//
@@ -246,12 +234,12 @@ impl LayeredRepository {
timelineid,
self.tenantid,
Arc::clone(&self.walredo_mgr),
0, // init with 0 and update after layers are loaded,
self.upload_relishes,
self.relish_uploader.as_ref().map(Arc::clone),
0, // init with 0 and update after layers are loaded
)?;
// List the layers on disk, and load them into the layer map
timeline.load_layer_map(disk_consistent_lsn)?;
timeline.load_layer_map()?;
// needs to be after load_layer_map
timeline.init_current_logical_size()?;
@@ -280,14 +268,15 @@ impl LayeredRepository {
conf: &'static PageServerConf,
walredo_mgr: Arc<dyn WalRedoManager + Send + Sync>,
tenantid: ZTenantId,
upload_relishes: bool,
) -> LayeredRepository {
LayeredRepository {
tenantid,
conf,
timelines: Mutex::new(HashMap::new()),
walredo_mgr,
upload_relishes,
relish_uploader: conf.relish_storage_config.as_ref().map(|config| {
Arc::new(QueueBasedRelishUploader::new(config, &conf.workdir).unwrap())
}),
}
}
@@ -362,36 +351,13 @@ impl LayeredRepository {
timelineid: ZTimelineId,
tenantid: ZTenantId,
data: &TimelineMetadata,
first_save: bool,
) -> Result<PathBuf> {
let timeline_path = conf.timeline_path(&timelineid, &tenantid);
let path = timeline_path.join("metadata");
// use OpenOptions to ensure file presence is consistent with first_save
let mut file = OpenOptions::new()
.write(true)
.create_new(first_save)
.open(&path)?;
let path = conf.timeline_path(&timelineid, &tenantid).join("metadata");
let mut file = File::create(&path)?;
info!("saving metadata {}", path.display());
let mut metadata_bytes = TimelineMetadata::ser(data)?;
assert!(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));
if file.write(&metadata_bytes)? != metadata_bytes.len() {
bail!("Could not write all the metadata bytes in a single call");
}
file.sync_all()?;
// fsync the parent directory to ensure the directory entry is durable
if first_save {
let timeline_dir = File::open(&timeline_path)?;
timeline_dir.sync_all()?;
}
file.write_all(&TimelineMetadata::ser(data)?)?;
Ok(path)
}
@@ -402,18 +368,9 @@ impl LayeredRepository {
tenantid: ZTenantId,
) -> Result<TimelineMetadata> {
let path = conf.timeline_path(&timelineid, &tenantid).join("metadata");
let metadata_bytes = std::fs::read(&path)?;
ensure!(metadata_bytes.len() == METADATA_MAX_SAFE_SIZE);
let data = std::fs::read(&path)?;
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);
ensure!(calculated_checksum == expected_checksum);
let data = TimelineMetadata::des_prefix(data)?;
let data = TimelineMetadata::des(&data)?;
assert!(data.disk_consistent_lsn.is_aligned());
Ok(data)
@@ -440,14 +397,14 @@ impl LayeredRepository {
// don't cover any branch point LSNs.
//
// TODO:
// - if a relation has a non-incremental persistent layer on a child branch, then we
// - if a relation has been modified on a child branch, then we
// don't need to keep that in the parent anymore. But currently
// we do.
fn gc_iteration_internal(
&self,
target_timelineid: Option<ZTimelineId>,
horizon: u64,
checkpoint_before_gc: bool,
compact: bool,
) -> Result<GcResult> {
let mut totals: GcResult = Default::default();
let now = Instant::now();
@@ -459,71 +416,63 @@ impl LayeredRepository {
// Scan all timelines. For each timeline, remember the timeline ID and
// the branch point where it was created.
//
let mut timelineids: Vec<ZTimelineId> = Vec::new();
// We scan the directory, not the in-memory hash table, because the hash
// table only contains entries for timelines that have been accessed. We
// need to take all timelines into account, not only the active ones.
let mut timelineids: Vec<ZTimelineId> = Vec::new();
let mut all_branchpoints: BTreeSet<(ZTimelineId, Lsn)> = BTreeSet::new();
let timelines_path = self.conf.timelines_path(&self.tenantid);
for direntry in fs::read_dir(timelines_path)? {
let direntry = direntry?;
if let Some(fname) = direntry.file_name().to_str() {
if let Ok(timelineid) = fname.parse::<ZTimelineId>() {
timelineids.push(timelineid);
}
}
}
//Now collect info about branchpoints
let mut all_branchpoints: BTreeSet<(ZTimelineId, Lsn)> = BTreeSet::new();
for timelineid in &timelineids {
let timeline = self.get_timeline_locked(*timelineid, &mut *timelines)?;
if let Some(ancestor_timeline) = &timeline.ancestor_timeline {
// If target_timeline is specified, we only need to know branchpoints of its childs
if let Some(timelineid) = target_timelineid {
if ancestor_timeline.timelineid == timelineid {
all_branchpoints
.insert((ancestor_timeline.timelineid, timeline.ancestor_lsn));
// Read the metadata of this timeline to get its parent timeline.
//
// We read the ancestor information directly from the file, instead
// of calling get_timeline(). We don't want to load the timeline
// into memory just for GC.
//
// FIXME: we open the timeline in the loop below with
// get_timeline_locked() anyway, so maybe we should just do it
// here, too.
let metadata = Self::load_metadata(self.conf, timelineid, self.tenantid)?;
if let Some(ancestor_timeline) = metadata.ancestor_timeline {
all_branchpoints.insert((ancestor_timeline, metadata.ancestor_lsn));
}
}
// Collect branchpoints for all timelines
else {
all_branchpoints.insert((ancestor_timeline.timelineid, timeline.ancestor_lsn));
}
}
}
// Ok, we now know all the branch points.
// Perform GC for each timeline.
// Ok, we now know all the branch points. Iterate through them.
for timelineid in timelineids {
// We have already loaded all timelines above
// so this operation is just a quick map lookup.
let timeline = self.get_timeline_locked(timelineid, &mut *timelines)?;
// If target_timeline is specified, only GC it
if let Some(target_timelineid) = target_timelineid {
if timelineid != target_timelineid {
// If a target timeline was specified, leave the other timelines alone.
// This is a bit inefficient, because we still collect the information for
// all the timelines above.
if let Some(x) = target_timelineid {
if x != timelineid {
continue;
}
}
if let Some(cutoff) = timeline.get_last_record_lsn().checked_sub(horizon) {
let branchpoints: Vec<Lsn> = all_branchpoints
.range((
Included((timelineid, Lsn(0))),
Included((timelineid, Lsn(u64::MAX))),
))
.map(|&x| x.1)
.collect();
let branchpoints: Vec<Lsn> = all_branchpoints
.range((
Included((timelineid, Lsn(0))),
Included((timelineid, Lsn(u64::MAX))),
))
.map(|&x| x.1)
.collect();
// If requested, force flush all in-memory layers to disk first,
// so that they too can be garbage collected. That's
let timeline = self.get_timeline_locked(timelineid, &mut *timelines)?;
let last_lsn = timeline.get_last_record_lsn();
if let Some(cutoff) = last_lsn.checked_sub(horizon) {
// If GC was explicitly requested by the admin, force flush all in-memory
// layers to disk first, so that they too can be garbage collected. That's
// used in tests, so we want as deterministic results as possible.
if checkpoint_before_gc {
if compact {
timeline.checkpoint()?;
info!("timeline {} checkpoint_before_gc done", timelineid);
}
let result = timeline.gc_timeline(branchpoints, cutoff)?;
@@ -543,7 +492,7 @@ impl LayeredRepository {
error!("timeline size calculation diverged, incremental doesn't match non incremental. incremental={} non_incremental={}", incremental, non_incremental);
}
}
Err(e) => error!("failed to calculate non incremental timeline size: {:#}", e),
Err(e) => error!("failed to calculate non incremental timeline size: {}", e),
}
}
@@ -589,6 +538,8 @@ pub struct LayeredTimeline {
// WAL redo manager
walredo_mgr: Arc<dyn WalRedoManager + Sync + Send>,
relish_uploader: Option<Arc<QueueBasedRelishUploader>>,
// What page versions do we hold in the repository? If we get a
// request > last_record_lsn, we need to wait until we receive all
// the WAL up to the request. The SeqWait provides functions for
@@ -636,9 +587,6 @@ pub struct LayeredTimeline {
// 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,
/// If `true`, will backup its timeline files to remote storage after freezing.
upload_relishes: bool,
}
/// Public interface functions
@@ -646,11 +594,8 @@ impl Timeline for LayeredTimeline {
/// Wait until WAL has been received up to the given LSN.
fn wait_lsn(&self, lsn: Lsn) -> Result<()> {
// This should never be called from the WAL receiver thread, because that could lead
// to a deadlock.
assert!(
!IS_WAL_RECEIVER.with(|c| c.get()),
"wait_lsn called by WAL receiver thread"
);
// to a deadlock. FIXME: Is there a less hacky way to check that?
assert_ne!(thread::current().name(), Some("WAL receiver thread"));
self.last_record_lsn
.wait_for_timeout(lsn, TIMEOUT)
@@ -824,7 +769,6 @@ impl Timeline for LayeredTimeline {
rel
);
}
ensure!(rec.lsn.is_aligned(), "unaligned record LSN");
let seg = SegmentTag::from_blknum(rel, blknum);
let delta_size = self.perform_write_op(seg, rec.lsn, |layer| {
@@ -838,51 +782,42 @@ impl Timeline for LayeredTimeline {
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
.get_relish_size(rel, self.get_last_record_lsn())?
.ok_or_else(|| {
anyhow!(
"attempted to truncate non-existent relish {} at {}",
if let Some(oldsize) = self.get_relish_size(rel, self.get_last_record_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,
lsn
)
})?;
segno: remove_segno,
};
self.perform_write_op(seg, lsn, |layer| layer.drop_segment(lsn))?;
}
if oldsize <= relsize {
return Ok(());
// 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,
};
self.perform_write_op(seg, lsn, |layer| {
layer.put_truncation(lsn, relsize % RELISH_SEG_SIZE)
})?;
}
self.decrease_current_logical_size((oldsize - relsize) * BLCKSZ as u32);
}
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,
};
self.perform_write_op(seg, lsn, |layer| 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,
};
self.perform_write_op(seg, lsn, |layer| {
layer.put_truncation(lsn, relsize % RELISH_SEG_SIZE)
})?;
}
self.decrease_current_logical_size((oldsize - relsize) * BLCKSZ as u32);
Ok(())
}
@@ -929,7 +864,6 @@ impl Timeline for LayeredTimeline {
rel
);
}
ensure!(lsn.is_aligned(), "unaligned record LSN");
let seg = SegmentTag::from_blknum(rel, blknum);
@@ -1022,8 +956,8 @@ impl LayeredTimeline {
timelineid: ZTimelineId,
tenantid: ZTenantId,
walredo_mgr: Arc<dyn WalRedoManager + Send + Sync>,
relish_uploader: Option<Arc<QueueBasedRelishUploader>>,
current_logical_size: usize,
upload_relishes: bool,
) -> Result<LayeredTimeline> {
let current_logical_size_gauge = LOGICAL_TIMELINE_SIZE
.get_metric_with_label_values(&[&tenantid.to_string(), &timelineid.to_string()])
@@ -1035,6 +969,7 @@ impl LayeredTimeline {
layers: Mutex::new(LayerMap::default()),
walredo_mgr,
relish_uploader,
// initialize in-memory 'last_record_lsn' from 'disk_consistent_lsn'.
last_record_lsn: SeqWait::new(RecordLsn {
@@ -1047,7 +982,6 @@ impl LayeredTimeline {
ancestor_lsn: metadata.ancestor_lsn,
current_logical_size: AtomicUsize::new(current_logical_size),
current_logical_size_gauge,
upload_relishes,
};
Ok(timeline)
}
@@ -1055,7 +989,7 @@ impl LayeredTimeline {
///
/// Scan the timeline directory to populate the layer map
///
fn load_layer_map(&self, disk_consistent_lsn: Lsn) -> anyhow::Result<()> {
fn load_layer_map(&self) -> anyhow::Result<()> {
info!(
"loading layer map for timeline {} into memory",
self.timelineid
@@ -1064,20 +998,8 @@ impl LayeredTimeline {
let (imgfilenames, mut deltafilenames) =
filename::list_files(self.conf, self.timelineid, self.tenantid)?;
let timeline_path = self.conf.timeline_path(&self.timelineid, &self.tenantid);
// First create ImageLayer structs for each image file.
for filename in imgfilenames.iter() {
if filename.lsn > disk_consistent_lsn {
warn!(
"found future image layer {} on timeline {}",
filename, self.timelineid
);
rename_to_backup(timeline_path.join(filename.to_string()))?;
continue;
}
let layer = ImageLayer::new(self.conf, self.timelineid, self.tenantid, filename);
info!(
@@ -1095,17 +1017,6 @@ impl LayeredTimeline {
deltafilenames.sort();
for filename in deltafilenames.iter() {
ensure!(filename.start_lsn < filename.end_lsn);
if filename.end_lsn > disk_consistent_lsn {
warn!(
"found future delta layer {} on timeline {}",
filename, self.timelineid
);
rename_to_backup(timeline_path.join(filename.to_string()))?;
continue;
}
let predecessor = layers.get(&filename.seg, filename.start_lsn);
let predecessor_str: String = if let Some(prec) = &predecessor {
@@ -1253,57 +1164,36 @@ impl LayeredTimeline {
assert!(lsn.is_aligned());
let last_record_lsn = self.get_last_record_lsn();
assert!(
lsn > last_record_lsn,
"cannot modify relation after advancing last_record_lsn (incoming_lsn={}, last_record_lsn={})",
lsn,
last_record_lsn,
);
if lsn <= last_record_lsn {
panic!(
"cannot modify relation after advancing last_record_lsn (incoming_lsn={}, last_record_lsn={})",
lsn,
last_record_lsn
);
}
// 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(layer) = layers.get_open(&seg) {
if layer.get_start_lsn() > 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() {
assert!(!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,
lsn,
)?;
} else {
return Ok(open_layer);
}
return Ok(layer);
}
// No writeable layer for this relation. Create one.
// No (writeable) layer for this relation yet. 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)?
{
let layer;
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;
trace!(
"creating layer for write for {} at branch point {}/{}",
"creating file for write for {} at branch point {}/{}",
seg,
self.timelineid,
start_lsn
@@ -1311,7 +1201,7 @@ impl LayeredTimeline {
} else {
start_lsn = prev_layer.get_end_lsn();
trace!(
"creating layer for write for {} after previous layer {}/{}",
"creating file for write for {} after previous layer {}/{}",
seg,
self.timelineid,
start_lsn
@@ -1378,8 +1268,6 @@ impl LayeredTimeline {
last_record_lsn
);
let timeline_dir = File::open(self.conf.timeline_path(&self.timelineid, &self.tenantid))?;
// 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.
@@ -1391,8 +1279,6 @@ impl LayeredTimeline {
// a lot of memory and/or aren't receiving much updates anymore.
let mut disk_consistent_lsn = last_record_lsn;
let mut created_historics = false;
while let Some((oldest_layer, oldest_generation)) = layers.peek_oldest_open() {
let oldest_pending_lsn = oldest_layer.get_oldest_pending_lsn();
@@ -1437,25 +1323,21 @@ impl LayeredTimeline {
layers.insert_open(new_open);
}
// We temporarily insert InMemory layer into historic list here.
// TODO: check that all possible concurrent users of 'historic' treat it right
layers.insert_historic(frozen.clone());
// Write the now-frozen layer to disk. That could take a while, so release the lock while do it
drop(layers);
let new_historics = frozen.write_to_disk(self)?;
layers = self.layers.lock().unwrap();
if !new_historics.is_empty() {
created_historics = true;
if let Some(relish_uploader) = &self.relish_uploader {
for label_path in new_historics.iter().filter_map(|layer| layer.path()) {
relish_uploader.schedule_upload(self.timelineid, label_path);
}
}
// Finally, replace the frozen in-memory layer with the new on-disk layers
layers.remove_historic(frozen.clone());
layers.remove_historic(frozen.as_ref());
// If we created a successor InMemoryLayer, its predecessor is
// currently the frozen layer. We need to update the predecessor
// to be the latest on-disk layer.
if let Some(last_historic) = new_historics.last() {
if let Some(new_open) = &maybe_new_open {
let maybe_old_predecessor =
@@ -1466,7 +1348,6 @@ impl LayeredTimeline {
}
}
// Add the historics to the LayerMap
for n in new_historics {
layers.insert_historic(n);
}
@@ -1479,14 +1360,6 @@ impl LayeredTimeline {
layer.unload()?;
}
drop(layers);
if created_historics {
// We must fsync the timeline dir to ensure the directory entries for
// new layer files are durable
timeline_dir.sync_all()?;
}
// Save the metadata, with updated 'disk_consistent_lsn', to a
// file in the timeline dir. After crash, we will restart WAL
// streaming and processing from that point.
@@ -1511,23 +1384,10 @@ impl LayeredTimeline {
ancestor_timeline: ancestor_timelineid,
ancestor_lsn: self.ancestor_lsn,
};
let _metadata_path = LayeredRepository::save_metadata(
self.conf,
self.timelineid,
self.tenantid,
&metadata,
false,
)?;
if self.upload_relishes {
schedule_timeline_upload(())
// schedule_timeline_upload(LocalTimeline {
// tenant_id: self.tenantid,
// timeline_id: self.timelineid,
// metadata_path,
// image_layers: image_layer_uploads,
// delta_layers: delta_layer_uploads,
// disk_consistent_lsn,
// });
let metadata_path =
LayeredRepository::save_metadata(self.conf, self.timelineid, self.tenantid, &metadata)?;
if let Some(relish_uploader) = &self.relish_uploader {
relish_uploader.schedule_upload(self.timelineid, metadata_path);
}
// Also update the in-memory copy
@@ -1566,18 +1426,15 @@ impl LayeredTimeline {
"running GC on timeline {}, cutoff {}",
self.timelineid, cutoff
);
info!("retain_lsns: {:?}", retain_lsns);
let mut layers_to_remove: Vec<Arc<dyn Layer>> = Vec::new();
// Scan all on-disk layers in the timeline.
//
// 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;
// Scan all layer files in the directory. For each file, if a newer file
// exists, we can remove the old one.
//
// Determine for each file if it needs to be retained
// FIXME: also scan open in-memory layers. Normally we cannot remove the
// latest layer of any seg, but if it was dropped it's possible
let mut layers = self.layers.lock().unwrap();
'outer: for l in layers.iter_historic_layers() {
let seg = l.get_seg_tag();
@@ -1588,7 +1445,7 @@ impl LayeredTimeline {
result.ondisk_nonrelfiles_total += 1;
}
// 1. Is it newer than cutoff point?
// Is it newer than cutoff point?
if l.get_end_lsn() > cutoff {
info!(
"keeping {} {}-{} because it's newer than cutoff {}",
@@ -1605,10 +1462,10 @@ impl LayeredTimeline {
continue 'outer;
}
// 2. Is it needed by a child branch?
// Is it needed by a child branch?
for retain_lsn in &retain_lsns {
// start_lsn is inclusive and end_lsn is exclusive
if l.get_start_lsn() <= *retain_lsn && *retain_lsn < l.get_end_lsn() {
// FIXME: are the bounds inclusive or exclusive?
if l.get_start_lsn() <= *retain_lsn && *retain_lsn <= l.get_end_lsn() {
info!(
"keeping {} {}-{} because it's needed by branch point {}",
seg,
@@ -1625,15 +1482,9 @@ impl LayeredTimeline {
}
}
// 3. Is there a later on-disk layer for this relation?
// Unless the relation was dropped, is there a later image file for this relation?
if !l.is_dropped() && !layers.newer_image_layer_exists(l.get_seg_tag(), l.get_end_lsn())
{
info!(
"keeping {} {}-{} because it is the latest layer",
seg,
l.get_start_lsn(),
l.get_end_lsn()
);
if seg.rel.is_relation() {
result.ondisk_relfiles_not_updated += 1;
} else {
@@ -1642,77 +1493,6 @@ impl LayeredTimeline {
continue 'outer;
}
// 4. Does this layer serve as a tombstome 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 {
info!(
"earlier layer exists at {} in {}",
prior_lsn, self.timelineid
);
}
// Now check ancestor timelines, if any
else if let Some(ancestor) = &self.ancestor_timeline {
let prior_lsn = ancestor.get_last_record_lsn();
if seg.rel.is_blocky() {
info!(
"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 = SegmentTag::from_blknum(seg.rel, size - 1);
info!(
"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;
}
}
_ => {
info!("blocky rel doesn't exist");
}
}
} else {
info!(
"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 {
info!(
"keeping {} {}-{} because this layer servers as a tombstome 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.
info!(
"garbage collecting {} {}-{} {}",
@@ -1729,7 +1509,7 @@ impl LayeredTimeline {
// while iterating it. BTreeMap::retain() would be another option)
for doomed_layer in layers_to_remove {
doomed_layer.delete()?;
layers.remove_historic(doomed_layer.clone());
layers.remove_historic(&*doomed_layer);
match (
doomed_layer.is_dropped(),
@@ -1960,23 +1740,3 @@ fn layer_ptr_eq(l1: &dyn Layer, l2: &dyn Layer) -> bool {
// see here for more https://github.com/rust-lang/rust/issues/46139
std::ptr::eq(l1_ptr as *const (), l2_ptr as *const ())
}
/// Add a suffix to a layer file's name: .{num}.old
/// Uses the first available num (starts at 0)
fn rename_to_backup(path: PathBuf) -> anyhow::Result<()> {
let filename = path.file_name().unwrap().to_str().unwrap();
let mut new_path = path.clone();
for i in 0u32.. {
new_path.set_file_name(format!("{}.{}.old", filename, i));
if !new_path.exists() {
std::fs::rename(&path, &new_path)?;
return Ok(());
}
}
Err(anyhow!(
"couldn't find an unused backup number for {:?}",
path
))
}

View File

@@ -25,13 +25,11 @@ OnDisk layers can be Image or Delta:
Dropped segments are always represented on disk by DeltaLayer.
LSN range defined by start_lsn and end_lsn:
- start_lsn is inclusive.
- end_lsn is exclusive.
For an open in-memory layer, the end_lsn is MAX_LSN. For a frozen
in-memory 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
- start_lsn is always inclusive.
- end_lsn depends on layer kind:
- InMemoryLayer is either unbounded (end_lsn = MAX_LSN) or dropped (end_lsn = drop_lsn)
- ImageLayer represents snapshot at one LSN, so end_lsn = lsn.
- DeltaLayer has explicit end_lsn, which represents end of incremental layer.
Layers can be open or historical:
- Open layer is a writeable one. Only InMemory layer can be open.

View File

@@ -42,10 +42,12 @@ use crate::layered_repository::filename::{DeltaFileName, PathOrConf};
use crate::layered_repository::storage_layer::{
Layer, PageReconstructData, PageReconstructResult, PageVersion, SegmentTag,
};
use crate::repository::WALRecord;
use crate::waldecoder;
use crate::PageServerConf;
use crate::{ZTenantId, ZTimelineId};
use anyhow::{bail, Result};
use bytes::Bytes;
use log::*;
use serde::{Deserialize, Serialize};
use std::collections::BTreeMap;
@@ -107,6 +109,12 @@ impl From<&DeltaLayer> for Summary {
}
}
#[derive(Serialize, Deserialize)]
struct PageVersionMeta {
page_image_range: Option<BlobRange>,
record_range: Option<BlobRange>,
}
///
/// DeltaLayer is the in-memory data structure associated with an
/// on-disk delta file. We keep a DeltaLayer in memory for each
@@ -144,7 +152,7 @@ pub struct DeltaLayerInner {
/// All versions of all pages in the file are are kept here.
/// Indexed by block number and LSN.
page_version_metas: BTreeMap<(u32, Lsn), BlobRange>,
page_version_metas: BTreeMap<(u32, Lsn), PageVersionMeta>,
/// `relsizes` tracks the size of the relation at different points in time.
relsizes: BTreeMap<Lsn, u32>,
@@ -221,15 +229,15 @@ impl Layer for DeltaLayer {
let mut iter = inner
.page_version_metas
.range((Included(&minkey), Included(&maxkey)));
while let Some(((_blknum, _entry_lsn), blob_range)) = iter.next_back() {
let pv = PageVersion::des(&read_blob(&page_version_reader, blob_range)?)?;
if let Some(img) = pv.page_image {
while let Some(((_blknum, _entry_lsn), entry)) = iter.next_back() {
if let Some(img_range) = &entry.page_image_range {
// Found a page image, return it
let img = Bytes::from(read_blob(&page_version_reader, img_range)?);
reconstruct_data.page_img = Some(img);
need_image = false;
break;
} else if let Some(rec) = pv.record {
} else if let Some(rec_range) = &entry.record_range {
let rec = WALRecord::des(&read_blob(&page_version_reader, rec_range)?)?;
let will_init = rec.will_init;
reconstruct_data.records.push(rec);
if will_init {
@@ -332,16 +340,16 @@ impl Layer for DeltaLayer {
println!("--- page versions ---");
let (_path, book) = self.open_book()?;
let chapter = book.chapter_reader(PAGE_VERSIONS_CHAPTER)?;
for ((blk, lsn), blob_range) in inner.page_version_metas.iter() {
for (k, v) in inner.page_version_metas.iter() {
let mut desc = String::new();
let buf = read_blob(&chapter, blob_range)?;
let pv = PageVersion::des(&buf)?;
if let Some(img) = pv.page_image.as_ref() {
write!(&mut desc, " img {} bytes", img.len())?;
if let Some(page_image_range) = v.page_image_range.as_ref() {
let image = read_blob(&chapter, page_image_range)?;
write!(&mut desc, " img {} bytes", image.len())?;
}
if let Some(rec) = pv.record.as_ref() {
if let Some(record_range) = v.record_range.as_ref() {
let record_bytes = read_blob(&chapter, record_range)?;
let rec = WALRecord::des(&record_bytes)?;
let wal_desc = waldecoder::describe_wal_record(&rec.rec);
write!(
&mut desc,
@@ -351,7 +359,7 @@ impl Layer for DeltaLayer {
wal_desc
)?;
}
println!(" blk {} at {}: {}", blk, lsn, desc);
println!(" blk {} at {}: {}", k.0, k.1, desc);
}
Ok(())
@@ -373,15 +381,14 @@ impl DeltaLayer {
}
}
/// Create a new delta file, using the given page versions and relsizes.
/// The page versions are passed by an iterator; the iterator must return
/// page versions in blknum+lsn order.
/// Create a new delta file, using the given btreemaps containing the page versions and
/// relsizes.
///
/// This is used to write the in-memory layer to disk. The in-memory layer uses the same
/// data structure with two btreemaps as we do, so passing the btreemaps is currently
/// expedient.
#[allow(clippy::too_many_arguments)]
pub fn create<'a>(
pub fn create(
conf: &'static PageServerConf,
timelineid: ZTimelineId,
tenantid: ZTenantId,
@@ -390,7 +397,7 @@ impl DeltaLayer {
end_lsn: Lsn,
dropped: bool,
predecessor: Option<Arc<dyn Layer>>,
page_versions: impl Iterator<Item = (&'a (u32, Lsn), &'a PageVersion)>,
page_versions: BTreeMap<(u32, Lsn), PageVersion>,
relsizes: BTreeMap<Lsn, u32>,
) -> Result<DeltaLayer> {
let delta_layer = DeltaLayer {
@@ -424,10 +431,26 @@ impl DeltaLayer {
let mut page_version_writer = BlobWriter::new(book, PAGE_VERSIONS_CHAPTER);
for (key, page_version) in page_versions {
let buf = PageVersion::ser(page_version)?;
let blob_range = page_version_writer.write_blob(&buf)?;
let page_image_range = page_version
.page_image
.map(|page_image| page_version_writer.write_blob(page_image.as_ref()))
.transpose()?;
let old = inner.page_version_metas.insert(*key, blob_range);
let record_range = page_version
.record
.map(|record| {
let buf = WALRecord::ser(&record)?;
page_version_writer.write_blob(&buf)
})
.transpose()?;
let old = inner.page_version_metas.insert(
key,
PageVersionMeta {
page_image_range,
record_range,
},
);
assert!(old.is_none());
}
@@ -461,8 +484,7 @@ impl DeltaLayer {
let book = chapter.close()?;
// This flushes the underlying 'buf_writer'.
let writer = book.close()?;
writer.get_ref().sync_all()?;
book.close()?;
trace!("saved {}", &path.display());

View File

@@ -290,11 +290,7 @@ pub fn list_files(
deltafiles.push(deltafilename);
} else if let Some(imgfilename) = ImageFileName::from_str(fname) {
imgfiles.push(imgfilename);
} else if fname == "wal"
|| fname == "metadata"
|| fname == "ancestor"
|| fname.ends_with(".old")
{
} else if fname == "wal" || fname == "metadata" || fname == "ancestor" {
// ignore these
} else {
warn!("unrecognized filename in timeline dir: {}", fname);

View File

@@ -152,8 +152,7 @@ impl Layer for ImageLayer {
}
fn get_end_lsn(&self) -> Lsn {
// End-bound is exclusive
self.lsn + 1
self.lsn
}
/// Look up given page in the file
@@ -337,8 +336,7 @@ impl ImageLayer {
let book = chapter.close()?;
// This flushes the underlying 'buf_writer'.
let writer = book.close()?;
writer.get_ref().sync_all()?;
book.close()?;
trace!("saved {}", &path.display());

View File

@@ -15,13 +15,11 @@ use crate::{ZTenantId, ZTimelineId};
use anyhow::{bail, Result};
use bytes::Bytes;
use log::*;
use std::cmp::min;
use std::collections::BTreeMap;
use std::ops::Bound::Included;
use std::path::PathBuf;
use std::sync::{Arc, RwLock};
use std::sync::{Arc, Mutex};
use zenith_utils::accum::Accum;
use zenith_utils::lsn::Lsn;
pub struct InMemoryLayer {
@@ -44,7 +42,7 @@ pub struct InMemoryLayer {
/// The above fields never change. The parts that do change are in 'inner',
/// and protected by mutex.
inner: RwLock<InMemoryLayerInner>,
inner: Mutex<InMemoryLayerInner>,
}
pub struct InMemoryLayerInner {
@@ -95,7 +93,7 @@ impl Layer for InMemoryLayer {
// An in-memory layer doesn't really have a filename as it's not stored on disk,
// but we construct a filename as if it was a delta layer
fn filename(&self) -> PathBuf {
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
let end_lsn;
let dropped;
@@ -139,7 +137,7 @@ impl Layer for InMemoryLayer {
return Lsn(end_lsn.0 + 1);
}
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
if let Some(drop_lsn) = inner.drop_lsn {
drop_lsn
@@ -149,7 +147,7 @@ impl Layer for InMemoryLayer {
}
fn is_dropped(&self) -> bool {
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
inner.drop_lsn.is_some()
}
@@ -167,7 +165,7 @@ impl Layer for InMemoryLayer {
let predecessor: Option<Arc<dyn Layer>>;
{
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
// Scan the BTreeMap backwards, starting from reconstruct_data.lsn.
let minkey = (blknum, Lsn(0));
@@ -214,13 +212,13 @@ impl Layer for InMemoryLayer {
fn get_seg_size(&self, lsn: Lsn) -> Result<u32> {
assert!(lsn >= self.start_lsn);
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
Ok(inner.get_seg_size(lsn))
}
/// Does this segment exist at given LSN?
fn get_seg_exists(&self, lsn: Lsn) -> Result<bool> {
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
// If the segment created after requested LSN,
// it doesn't exist in the layer. But we shouldn't
@@ -252,13 +250,13 @@ impl Layer for InMemoryLayer {
}
fn is_incremental(&self) -> bool {
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
inner.predecessor.is_some()
}
/// debugging function to print out the contents of the layer
fn dump(&self) -> Result<()> {
let inner = self.inner.read().unwrap();
let inner = self.inner.lock().unwrap();
let end_str = inner
.drop_lsn
@@ -340,7 +338,7 @@ impl InMemoryLayer {
start_lsn,
end_lsn: None,
oldest_pending_lsn,
inner: RwLock::new(InMemoryLayerInner {
inner: Mutex::new(InMemoryLayerInner {
drop_lsn: None,
page_versions: BTreeMap::new(),
segsizes: BTreeMap::new(),
@@ -389,7 +387,7 @@ impl InMemoryLayer {
self.timelineid,
lsn
);
let mut inner = self.inner.write().unwrap();
let mut inner = self.inner.lock().unwrap();
inner.check_writeable()?;
@@ -407,7 +405,7 @@ impl InMemoryLayer {
if self.seg.rel.is_blocky() {
let newsize = blknum - self.seg.segno * RELISH_SEG_SIZE + 1;
// use inner get_seg_size, since calling self.get_seg_size will try to acquire the lock,
// use inner get_seg_size, since calling self.get_seg_size will try to acquire self.inner.lock
// which we've just acquired above
let oldsize = inner.get_seg_size(lsn);
if newsize > oldsize {
@@ -460,7 +458,7 @@ impl InMemoryLayer {
pub fn put_truncation(&self, lsn: Lsn, segsize: u32) -> WriteResult<()> {
self.assert_not_frozen();
let mut inner = self.inner.write().unwrap();
let mut inner = self.inner.lock().unwrap();
inner.check_writeable()?;
// check that this we truncate to a smaller size than segment was before the truncation
@@ -481,15 +479,14 @@ impl InMemoryLayer {
pub fn drop_segment(&self, lsn: Lsn) -> WriteResult<()> {
self.assert_not_frozen();
let mut inner = self.inner.write().unwrap();
let mut inner = self.inner.lock().unwrap();
inner.check_writeable()?;
assert!(inner.drop_lsn.is_none());
inner.drop_lsn = Some(lsn);
inner.writeable = false;
trace!("dropped segment {} at {}", self.seg, lsn);
info!("dropped segment {} at {}", self.seg, lsn);
Ok(())
}
@@ -508,9 +505,6 @@ impl InMemoryLayer {
) -> Result<InMemoryLayer> {
let seg = src.get_seg_tag();
assert!(oldest_pending_lsn.is_aligned());
assert!(oldest_pending_lsn >= start_lsn);
trace!(
"initializing new InMemoryLayer for writing {} on timeline {} at {}",
seg,
@@ -533,7 +527,7 @@ impl InMemoryLayer {
start_lsn,
end_lsn: None,
oldest_pending_lsn,
inner: RwLock::new(InMemoryLayerInner {
inner: Mutex::new(InMemoryLayerInner {
drop_lsn: None,
page_versions: BTreeMap::new(),
segsizes,
@@ -543,73 +537,65 @@ impl InMemoryLayer {
})
}
pub fn is_writeable(&self) -> bool {
let inner = self.inner.read().unwrap();
inner.writeable
}
/// Splits `self` into two InMemoryLayers: `frozen` and `open`.
/// All data up to and including `cutoff_lsn`
/// All data up to and including `cutoff_lsn` (or the drop LSN, if dropped)
/// is copied to `frozen`, while the remaining data is copied to `open`.
/// After completion, self is non-writeable, but not frozen.
pub fn freeze(self: Arc<Self>, cutoff_lsn: Lsn) -> Result<FreezeLayers> {
pub fn freeze(&self, cutoff_lsn: Lsn) -> Result<FreezeLayers> {
info!(
"freezing in memory layer {} on timeline {} at {} (oldest {})",
self.filename().display(),
self.timelineid,
cutoff_lsn,
self.oldest_pending_lsn
"freezing in memory layer for {} on timeline {} at {}",
self.seg, self.timelineid, cutoff_lsn
);
self.assert_not_frozen();
let self_ref = self.clone();
let mut inner = self_ref.inner.write().unwrap();
// Dropped layers don't need any special freeze actions,
// they are marked as non-writeable at drop and just
// written out to disk by checkpointer.
if inner.drop_lsn.is_some() {
assert!(!inner.writeable);
info!(
"freezing in memory layer for {} on timeline {} is dropped at {}",
self.seg,
self.timelineid,
inner.drop_lsn.unwrap()
);
// There should be no newer layer that refers this non-writeable layer,
// because layer that is created after dropped one represents a new rel.
return Ok(FreezeLayers {
frozen: self,
open: None,
});
}
let mut inner = self.inner.lock().unwrap();
assert!(inner.writeable);
inner.writeable = false;
// Divide all the page versions into old and new
// at the 'cutoff_lsn' point.
let mut before_segsizes = BTreeMap::new();
let mut after_segsizes = BTreeMap::new();
let mut after_oldest_lsn: Accum<Lsn> = Accum(None);
for (lsn, size) in inner.segsizes.iter() {
if *lsn > cutoff_lsn {
after_segsizes.insert(*lsn, *size);
after_oldest_lsn.accum(min, *lsn);
} else {
before_segsizes.insert(*lsn, *size);
}
}
// Normally, use the cutoff LSN as the end of the frozen layer.
// But if the relation was dropped, we know that there are no
// more changes coming in for it, and in particular we know that
// there are no changes "in flight" for the LSN anymore, so we use
// the drop LSN instead. The drop-LSN could be ahead of the
// caller-specified LSN!
let dropped = inner.drop_lsn.is_some();
let end_lsn = if dropped {
inner.drop_lsn.unwrap()
} else {
cutoff_lsn
};
let mut before_page_versions = BTreeMap::new();
let mut after_page_versions = BTreeMap::new();
for ((blknum, lsn), pv) in inner.page_versions.iter() {
if *lsn > cutoff_lsn {
after_page_versions.insert((*blknum, *lsn), pv.clone());
after_oldest_lsn.accum(min, *lsn);
} else {
before_page_versions.insert((*blknum, *lsn), pv.clone());
// Divide all the page versions into old and new at the 'end_lsn' cutoff point.
let mut before_page_versions;
let mut before_segsizes;
let mut after_page_versions;
let mut after_segsizes;
if !dropped {
before_segsizes = BTreeMap::new();
after_segsizes = BTreeMap::new();
for (lsn, size) in inner.segsizes.iter() {
if *lsn > end_lsn {
after_segsizes.insert(*lsn, *size);
} else {
before_segsizes.insert(*lsn, *size);
}
}
before_page_versions = BTreeMap::new();
after_page_versions = BTreeMap::new();
for ((blknum, lsn), pv) in inner.page_versions.iter() {
if *lsn > end_lsn {
after_page_versions.insert((*blknum, *lsn), pv.clone());
} else {
before_page_versions.insert((*blknum, *lsn), pv.clone());
}
}
} else {
before_page_versions = inner.page_versions.clone();
before_segsizes = inner.segsizes.clone();
after_segsizes = BTreeMap::new();
after_page_versions = BTreeMap::new();
}
let frozen = Arc::new(InMemoryLayer {
@@ -618,9 +604,9 @@ impl InMemoryLayer {
timelineid: self.timelineid,
seg: self.seg,
start_lsn: self.start_lsn,
end_lsn: Some(cutoff_lsn),
end_lsn: Some(end_lsn),
oldest_pending_lsn: self.start_lsn,
inner: RwLock::new(InMemoryLayerInner {
inner: Mutex::new(InMemoryLayerInner {
drop_lsn: inner.drop_lsn,
page_versions: before_page_versions,
segsizes: before_segsizes,
@@ -629,14 +615,14 @@ impl InMemoryLayer {
}),
});
let open = if !after_segsizes.is_empty() || !after_page_versions.is_empty() {
let open = if !dropped && (!after_segsizes.is_empty() || !after_page_versions.is_empty()) {
let mut new_open = Self::create_successor_layer(
self.conf,
frozen.clone(),
self.timelineid,
self.tenantid,
cutoff_lsn + 1,
after_oldest_lsn.0.unwrap(),
end_lsn,
end_lsn,
)?;
let new_inner = new_open.inner.get_mut().unwrap();
@@ -648,6 +634,7 @@ impl InMemoryLayer {
None
};
// TODO could we avoid creating the `frozen` if it contains no data
Ok(FreezeLayers { frozen, open })
}
@@ -660,62 +647,35 @@ impl InMemoryLayer {
/// 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) -> Result<Vec<Arc<dyn Layer>>> {
trace!(
"write_to_disk {} end_lsn is {} get_end_lsn is {}",
self.filename().display(),
self.end_lsn.unwrap_or(Lsn(0)),
self.get_end_lsn()
);
let end_lsn = self.end_lsn.expect("can only write frozen layers to disk");
// 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 aquire the
// write lock on it, so we shouldn't block anyone. There's one exception
// though: another thread might have grabbed a reference to this layer
// in `get_layer_for_write' just before the checkpointer called
// `freeze`, and then `write_to_disk` on it. When the thread gets the
// lock, it will see that it's not writeable anymore and retry, but it
// would have to wait until we release it. That race condition is very
// rare though, so we just accept the potential latency hit for now.
let inner = self.inner.read().unwrap();
assert!(!inner.writeable);
let inner = self.inner.lock().unwrap();
let drop_lsn = inner.drop_lsn;
let predecessor = inner.predecessor.clone();
if let Some(drop_lsn) = inner.drop_lsn {
let delta_layer = DeltaLayer::create(
self.conf,
self.timelineid,
self.tenantid,
self.seg,
self.start_lsn,
drop_lsn,
true,
predecessor,
inner.page_versions.iter(),
inner.segsizes.clone(),
)?;
trace!(
"freeze: created delta layer for dropped segment {} {}-{}",
self.seg,
self.start_lsn,
drop_lsn
);
return Ok(vec![Arc::new(delta_layer)]);
}
let end_lsn = self.end_lsn.unwrap();
let mut before_segsizes = BTreeMap::new();
for (lsn, size) in inner.segsizes.iter() {
if *lsn <= end_lsn {
before_segsizes.insert(*lsn, *size);
let mut before_page_versions;
let mut before_segsizes;
if inner.drop_lsn.is_none() {
before_segsizes = BTreeMap::new();
for (lsn, size) in inner.segsizes.iter() {
if *lsn <= end_lsn {
before_segsizes.insert(*lsn, *size);
}
}
}
let mut before_page_versions = inner.page_versions.iter().filter(|tup| {
let ((_blknum, lsn), _pv) = tup;
*lsn < end_lsn
});
before_page_versions = BTreeMap::new();
for ((blknum, lsn), pv) in inner.page_versions.iter() {
if *lsn < end_lsn {
before_page_versions.insert((*blknum, *lsn), pv.clone());
}
}
} else {
before_page_versions = inner.page_versions.clone();
before_segsizes = inner.segsizes.clone();
}
drop(inner);
let mut frozen_layers: Vec<Arc<dyn Layer>> = Vec::new();
@@ -728,7 +688,7 @@ impl InMemoryLayer {
self.seg,
self.start_lsn,
end_lsn,
false,
drop_lsn.is_some(),
predecessor,
before_page_versions,
before_segsizes,
@@ -741,21 +701,21 @@ impl InMemoryLayer {
end_lsn
);
} else {
assert!(before_page_versions.next().is_none());
assert!(before_page_versions.is_empty());
}
drop(inner);
// Write a new base image layer at the cutoff point
let image_layer = ImageLayer::create_from_src(self.conf, timeline, self, end_lsn)?;
frozen_layers.push(Arc::new(image_layer));
trace!("freeze: created image layer {} at {}", self.seg, end_lsn);
if drop_lsn.is_none() {
// Write a new base image layer at the cutoff point
let image_layer = ImageLayer::create_from_src(self.conf, timeline, self, end_lsn)?;
frozen_layers.push(Arc::new(image_layer));
trace!("freeze: created image layer {} at {}", self.seg, end_lsn);
}
Ok(frozen_layers)
}
pub fn update_predecessor(&self, predecessor: Arc<dyn Layer>) -> Option<Arc<dyn Layer>> {
let mut inner = self.inner.write().unwrap();
let mut inner = self.inner.lock().unwrap();
inner.predecessor.replace(predecessor)
}
}

View File

@@ -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<I: ?Sized>
where
I: IntervalItem,
{
points: BTreeMap<I::Key, Point<I>>,
}
struct Point<I: ?Sized> {
/// 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<Arc<I>>,
}
/// 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::Key> {
self.start_key()..self.end_key()
}
}
impl<I: ?Sized> IntervalTree<I>
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<Arc<I>> {
// 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<I> {
IntervalIter {
point_iter: self.points.range(key..),
elem_iter: None,
}
}
/// Iterate over all items
pub fn iter(&self) -> IntervalIter<I> {
IntervalIter {
point_iter: self.points.range(..),
elem_iter: None,
}
}
pub fn insert(&mut self, item: Arc<I>) {
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<Arc<I>> = 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<I>) {
// 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<I::Key> = 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<I>>,
elem_iter: Option<(I::Key, std::slice::Iter<'a, Arc<I>>)>,
}
impl<'a, I> Iterator for IntervalIter<'a, I>
where
I: IntervalItem + ?Sized,
{
type Item = Arc<I>;
fn next(&mut self) -> Option<Self::Item> {
// 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<I: ?Sized> Default for IntervalTree<I>
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<MockItem>,
key: u32,
expected: &[&str],
) -> Option<Arc<MockItem>> {
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<MockItem>, expected: &[&str]) {
let mut contents: Vec<String> = tree.iter().map(|e| e.to_string()).collect();
contents.sort();
assert_eq!(contents, expected);
}
fn dump_tree(tree: &IntervalTree<MockItem>) {
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<MockItem> = 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<MockItem> = 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<MockItem> = 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<MockItem> = 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<MockItem> = 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
}
}

View File

@@ -9,14 +9,13 @@
//! new image and delta layers and 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::InMemoryLayer;
use crate::relish::*;
use anyhow::Result;
use lazy_static::lazy_static;
use std::cmp::Ordering;
use std::collections::{BinaryHeap, HashMap};
use std::collections::{BTreeMap, BinaryHeap, HashMap};
use std::sync::Arc;
use zenith_metrics::{register_int_gauge, IntGauge};
use zenith_utils::lsn::Lsn;
@@ -77,14 +76,7 @@ impl LayerMap {
pub fn insert_open(&mut self, layer: Arc<InMemoryLayer>) {
let segentry = self.segs.entry(layer.get_seg_tag()).or_default();
segentry.update_open(Arc::clone(&layer));
let oldest_pending_lsn = layer.get_oldest_pending_lsn();
// After a crash and restart, 'oldest_pending_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_pending_lsn.is_aligned());
segentry.insert_open(Arc::clone(&layer));
// Also add it to the binary heap
let open_layer_entry = OpenLayerEntry {
@@ -105,14 +97,11 @@ impl LayerMap {
// Also remove it from the SegEntry of this segment
let mut segentry = self.segs.get_mut(&segtag).unwrap();
if Arc::ptr_eq(segentry.open.as_ref().unwrap(), &oldest_entry.layer) {
segentry.open = None;
} else {
// We could have already updated segentry.open for
// dropped (non-writeable) layer. This is fine.
assert!(!oldest_entry.layer.is_writeable());
assert!(oldest_entry.layer.is_dropped());
}
assert!(Arc::ptr_eq(
segentry.open.as_ref().unwrap(),
&oldest_entry.layer
));
segentry.open = None;
NUM_INMEMORY_LAYERS.dec();
}
@@ -132,11 +121,12 @@ impl LayerMap {
///
/// This should be called when the corresponding file on disk has been deleted.
///
pub fn remove_historic(&mut self, layer: Arc<dyn Layer>) {
pub fn remove_historic(&mut self, layer: &dyn Layer) {
let tag = layer.get_seg_tag();
let start_lsn = layer.get_start_lsn();
if let Some(segentry) = self.segs.get_mut(&tag) {
segentry.historic.remove(&layer);
segentry.historic.remove(&start_lsn);
}
NUM_ONDISK_LAYERS.dec();
}
@@ -154,7 +144,7 @@ impl LayerMap {
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)? {
if let Some(exists) = segentry.exists_at_lsn(lsn) {
rels.insert(seg.rel, exists);
}
}
@@ -162,7 +152,7 @@ impl LayerMap {
}
_ => {
if tag == None {
if let Some(exists) = segentry.exists_at_lsn(lsn)? {
if let Some(exists) = segentry.exists_at_lsn(lsn) {
rels.insert(seg.rel, exists);
}
}
@@ -184,20 +174,6 @@ impl LayerMap {
}
}
/// Is there any layer for given segment that is alive at the lsn?
///
/// This is a public wrapper for SegEntry fucntion,
/// 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<bool> {
Ok(if let Some(segentry) = self.segs.get(&seg) {
segentry.exists_at_lsn(lsn)?.unwrap_or(false)
} else {
false
})
}
/// Return the oldest in-memory layer, along with its generation number.
pub fn peek_oldest_open(&self) -> Option<(Arc<InMemoryLayer>, u64)> {
self.open_layers
@@ -215,7 +191,7 @@ impl LayerMap {
pub fn iter_historic_layers(&self) -> HistoricLayerIter {
HistoricLayerIter {
seg_iter: self.segs.iter(),
segiter: self.segs.iter(),
iter: None,
}
}
@@ -229,7 +205,7 @@ impl LayerMap {
open.dump()?;
}
for layer in segentry.historic.iter() {
for (_, layer) in segentry.historic.iter() {
layer.dump()?;
}
}
@@ -238,40 +214,34 @@ impl LayerMap {
}
}
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.
/// each segment. The older layers, stored on disk, are kept in a
/// BTreeMap keyed by the layer's start LSN.
#[derive(Default)]
struct SegEntry {
open: Option<Arc<InMemoryLayer>>,
historic: IntervalTree<dyn Layer>,
pub open: Option<Arc<InMemoryLayer>>,
pub historic: BTreeMap<Lsn, Arc<dyn Layer>>,
}
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<Option<bool>> {
if let Some(layer) = self.get(lsn) {
Ok(Some(layer.get_seg_exists(lsn)?))
} else {
Ok(None)
fn exists_at_lsn(&self, lsn: Lsn) -> Option<bool> {
if let Some(layer) = &self.open {
if layer.get_start_lsn() <= lsn && lsn <= layer.get_end_lsn() {
let exists = layer.get_seg_exists(lsn).ok()?;
return Some(exists);
}
} else if let Some((_, layer)) = self.historic.range(..=lsn).next_back() {
let exists = layer.get_seg_exists(lsn).ok()?;
return Some(exists);
}
None
}
pub fn get(&self, lsn: Lsn) -> Option<Arc<dyn Layer>> {
@@ -282,30 +252,40 @@ impl SegEntry {
}
}
self.historic.search(lsn)
if let Some((_start_lsn, layer)) = self.historic.range(..=lsn).next_back() {
Some(Arc::clone(layer))
} else {
None
}
}
pub fn newer_image_layer_exists(&self, lsn: Lsn) -> bool {
// We only check on-disk layers, because
// in-memory layers are not durable
self.historic
.iter_newer(lsn)
.any(|layer| !layer.is_incremental())
for (_newer_lsn, layer) in self.historic.range(lsn..) {
// Ignore incremental layers.
if layer.is_incremental() {
continue;
}
if layer.get_end_lsn() > lsn {
return true;
} else {
continue;
}
}
false
}
// 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<InMemoryLayer>) {
if let Some(prev_open) = &self.open {
assert!(!prev_open.is_writeable());
}
pub fn insert_open(&mut self, layer: Arc<InMemoryLayer>) {
assert!(self.open.is_none());
self.open = Some(layer);
}
pub fn insert_historic(&mut self, layer: Arc<dyn Layer>) {
self.historic.insert(layer);
let start_lsn = layer.get_start_lsn();
self.historic.insert(start_lsn, layer);
}
}
@@ -344,8 +324,8 @@ 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<IntervalIter<'a, dyn Layer>>,
segiter: std::collections::hash_map::Iter<'a, SegmentTag, SegEntry>,
iter: Option<std::collections::btree_map::Iter<'a, Lsn, Arc<dyn Layer>>>,
}
impl<'a> Iterator for HistoricLayerIter<'a> {
@@ -355,11 +335,11 @@ impl<'a> Iterator for HistoricLayerIter<'a> {
loop {
if let Some(x) = &mut self.iter {
if let Some(x) = x.next() {
return Some(Arc::clone(&x));
return Some(Arc::clone(&*x.1));
}
}
if let Some((_tag, segentry)) = self.seg_iter.next() {
self.iter = Some(segentry.historic.iter());
if let Some(seg) = self.segiter.next() {
self.iter = Some(seg.1.historic.iter());
continue;
} else {
return None;
@@ -414,14 +394,14 @@ mod tests {
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)));
layers.insert_open(dummy_inmem_layer(conf, 0, Lsn(100), Lsn(100)));
layers.insert_open(dummy_inmem_layer(conf, 1, Lsn(100), Lsn(200)));
layers.insert_open(dummy_inmem_layer(conf, 2, Lsn(100), Lsn(120)));
layers.insert_open(dummy_inmem_layer(conf, 3, Lsn(100), Lsn(110)));
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)));
layers.insert_open(dummy_inmem_layer(conf, 4, Lsn(100), Lsn(110)));
layers.insert_open(dummy_inmem_layer(conf, 5, Lsn(100), Lsn(100)));
// A helper function (closure) to pop the next oldest open entry from the layer map,
// and assert that it is what we'd expect
@@ -432,12 +412,12 @@ mod tests {
layers.pop_oldest_open();
};
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
assert_pop_layer(0, gen1); // 100
assert_pop_layer(5, gen2); // 100
assert_pop_layer(3, gen1); // 110
assert_pop_layer(4, gen2); // 110
assert_pop_layer(2, gen1); // 120
assert_pop_layer(1, gen1); // 200
Ok(())
}

View File

@@ -111,14 +111,15 @@ pub trait Layer: Send + Sync {
/// Identify the relish segment
fn get_seg_tag(&self) -> SegmentTag;
/// Inclusive start bound of the LSN range that this layer holds
/// Inclusive start bound of the LSN range that this layer hold
fn get_start_lsn(&self) -> Lsn;
/// Exclusive end bound of the LSN range that this layer holds.
/// 'end_lsn' meaning depends on the layer kind:
/// - in-memory layer is either unbounded (end_lsn = MAX_LSN) or dropped (end_lsn = drop_lsn)
/// - image layer represents snapshot at one LSN, so end_lsn = lsn
/// - delta layer has end_lsn
///
/// - 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
/// TODO Is end_lsn always exclusive for all layer kinds?
fn get_end_lsn(&self) -> Lsn;
/// Is the segment represented by this layer dropped by PostgreSQL?

View File

@@ -13,7 +13,7 @@ pub mod http;
pub mod layered_repository;
pub mod page_service;
pub mod relish;
pub mod relish_storage;
mod relish_storage;
pub mod repository;
pub mod restore_local_repo;
pub mod tenant_mgr;
@@ -22,13 +22,12 @@ pub mod walreceiver;
pub mod walredo;
pub mod defaults {
use const_format::formatcp;
use std::time::Duration;
pub const DEFAULT_PG_LISTEN_PORT: u16 = 64000;
pub const DEFAULT_PG_LISTEN_ADDR: &str = formatcp!("127.0.0.1:{DEFAULT_PG_LISTEN_PORT}");
pub const DEFAULT_PG_LISTEN_ADDR: &str = "127.0.0.1:64000"; // can't format! const yet...
pub const DEFAULT_HTTP_LISTEN_PORT: u16 = 9898;
pub const DEFAULT_HTTP_LISTEN_ADDR: &str = formatcp!("127.0.0.1:{DEFAULT_HTTP_LISTEN_PORT}");
pub const DEFAULT_HTTP_LISTEN_ADDR: &str = "127.0.0.1:9898";
// FIXME: This current value is very low. I would imagine something like 1 GB or 10 GB
// would be more appropriate. But a low value forces the code to be exercised more,
@@ -40,7 +39,6 @@ pub mod defaults {
pub const DEFAULT_GC_PERIOD: Duration = Duration::from_secs(100);
pub const DEFAULT_SUPERUSER: &str = "zenith_admin";
pub const DEFAULT_RELISH_STORAGE_MAX_CONCURRENT_SYNC_LIMITS: usize = 100;
}
lazy_static! {
@@ -155,8 +153,8 @@ impl PageServerConf {
checkpoint_period: Duration::from_secs(10),
gc_horizon: defaults::DEFAULT_GC_HORIZON,
gc_period: Duration::from_secs(10),
listen_pg_addr: defaults::DEFAULT_PG_LISTEN_ADDR.to_string(),
listen_http_addr: defaults::DEFAULT_HTTP_LISTEN_ADDR.to_string(),
listen_pg_addr: "127.0.0.1:5430".to_string(),
listen_http_addr: "127.0.0.1:9898".to_string(),
superuser: "zenith_admin".to_string(),
workdir: repo_dir,
pg_distrib_dir: "".into(),
@@ -169,37 +167,18 @@ impl PageServerConf {
/// External relish storage configuration, enough for creating a client for that storage.
#[derive(Debug, Clone)]
pub struct RelishStorageConfig {
/// Limits the number of concurrent sync operations between pageserver and relish storage.
pub max_concurrent_sync: usize,
/// The storage connection configuration.
pub storage: RelishStorageKind,
}
/// A kind of a relish storage to connect to, with its connection configuration.
#[derive(Debug, Clone)]
pub enum RelishStorageKind {
/// Storage based on local file system.
/// Specify a root folder to place all stored relish data into.
pub enum RelishStorageConfig {
/// Root folder to place all stored relish data into.
LocalFs(PathBuf),
/// AWS S3 based storage, storing all relishes into the root
/// of the S3 bucket from the config.
AwsS3(S3Config),
}
/// AWS S3 bucket coordinates and access credentials to manage the bucket contents (read and write).
#[derive(Clone)]
pub struct S3Config {
/// Name of the bucket to connect to.
pub bucket_name: String,
/// The region where the bucket is located at.
pub bucket_region: String,
/// "Login" to use when connecting to bucket.
/// Can be empty for cases like AWS k8s IAM
/// where we can allow certain pods to connect
/// to the bucket directly without any credentials.
pub access_key_id: Option<String>,
/// "Password" to use when connecting to bucket.
pub secret_access_key: Option<String>,
}

View File

@@ -194,7 +194,7 @@ pub fn thread_main(
let local_auth = auth.clone();
thread::spawn(move || {
if let Err(err) = page_service_conn_main(conf, local_auth, socket, auth_type) {
error!("page server thread exiting with error: {:#}", err);
error!("error: {}", err);
}
});
}
@@ -293,9 +293,7 @@ impl PageServerHandler {
};
let response = response.unwrap_or_else(|e| {
// print the all details to the log with {:#}, but for the client the
// error message is enough
error!("error reading relation or page version: {:#}", e);
error!("error reading relation or page version: {}", e);
PagestreamBeMessage::Error(PagestreamErrorResponse {
message: e.to_string(),
})
@@ -656,14 +654,12 @@ impl postgres_backend::Handler for PageServerHandler {
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"elapsed"),
@@ -683,12 +679,6 @@ impl postgres_backend::Handler for PageServerHandler {
.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()),
@@ -705,12 +695,6 @@ impl postgres_backend::Handler for PageServerHandler {
.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.elapsed.as_millis().to_string().as_bytes()),

View File

@@ -8,43 +8,14 @@
mod local_fs;
mod rust_s3;
/// A queue-based storage with the background machinery behind it to synchronize
/// local page server layer files with external storage.
mod synced_storage;
/// A queue and the background machinery behind it to upload
/// local page server layer files to external storage.
pub mod storage_uploader;
use std::path::Path;
use std::thread;
use anyhow::Context;
use self::local_fs::LocalFs;
pub use self::synced_storage::schedule_timeline_upload;
use crate::relish_storage::rust_s3::RustS3;
use crate::{PageServerConf, RelishStorageKind};
pub fn run_storage_sync_thread(
config: &'static PageServerConf,
) -> anyhow::Result<Option<thread::JoinHandle<anyhow::Result<()>>>> {
match &config.relish_storage_config {
Some(relish_storage_config) => {
let max_concurrent_sync = relish_storage_config.max_concurrent_sync;
match &relish_storage_config.storage {
RelishStorageKind::LocalFs(root) => synced_storage::run_storage_sync_thread(
config,
LocalFs::new(root.clone())?,
max_concurrent_sync,
),
RelishStorageKind::AwsS3(s3_config) => synced_storage::run_storage_sync_thread(
config,
RustS3::new(s3_config)?,
max_concurrent_sync,
),
}
}
None => Ok(None),
}
}
/// Storage (potentially remote) API to manage its state.
#[async_trait::async_trait]
pub trait RelishStorage: Send + Sync {

View File

@@ -0,0 +1,116 @@
use std::{
collections::VecDeque,
path::{Path, PathBuf},
sync::{Arc, Mutex},
thread,
};
use zenith_utils::zid::ZTimelineId;
use crate::{relish_storage::RelishStorage, RelishStorageConfig};
use super::{local_fs::LocalFs, rust_s3::RustS3};
pub struct QueueBasedRelishUploader {
upload_queue: Arc<Mutex<VecDeque<(ZTimelineId, PathBuf)>>>,
}
impl QueueBasedRelishUploader {
pub fn new(
config: &RelishStorageConfig,
page_server_workdir: &'static Path,
) -> anyhow::Result<Self> {
let upload_queue = Arc::new(Mutex::new(VecDeque::new()));
let _handle = match config {
RelishStorageConfig::LocalFs(root) => {
let relish_storage = LocalFs::new(root.clone())?;
create_upload_thread(
Arc::clone(&upload_queue),
relish_storage,
page_server_workdir,
)?
}
RelishStorageConfig::AwsS3(s3_config) => {
let relish_storage = RustS3::new(s3_config)?;
create_upload_thread(
Arc::clone(&upload_queue),
relish_storage,
page_server_workdir,
)?
}
};
Ok(Self { upload_queue })
}
pub fn schedule_upload(&self, timeline_id: ZTimelineId, relish_path: PathBuf) {
self.upload_queue
.lock()
.unwrap()
.push_back((timeline_id, relish_path))
}
}
fn create_upload_thread<P, S: 'static + RelishStorage<RelishStoragePath = P>>(
upload_queue: Arc<Mutex<VecDeque<(ZTimelineId, PathBuf)>>>,
relish_storage: S,
page_server_workdir: &'static Path,
) -> std::io::Result<thread::JoinHandle<()>> {
let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()?;
thread::Builder::new()
.name("Queue based relish uploader".to_string())
.spawn(move || loop {
runtime.block_on(async {
upload_loop_step(&upload_queue, &relish_storage, page_server_workdir).await;
})
})
}
async fn upload_loop_step<P, S: 'static + RelishStorage<RelishStoragePath = P>>(
upload_queue: &Mutex<VecDeque<(ZTimelineId, PathBuf)>>,
relish_storage: &S,
page_server_workdir: &Path,
) {
let mut queue_accessor = upload_queue.lock().unwrap();
log::debug!("current upload queue length: {}", queue_accessor.len());
let next_upload = queue_accessor.pop_front();
drop(queue_accessor);
let (relish_timeline_id, relish_local_path) = match next_upload {
Some(data) => data,
None => {
// Don't spin and allow others to use the queue.
// In future, could be improved to be more clever about delays depending on relish upload stats
thread::sleep(std::time::Duration::from_secs(1));
return;
}
};
if let Err(e) = upload_relish(relish_storage, page_server_workdir, &relish_local_path).await {
log::error!(
"Failed to upload relish '{}' for timeline {}, reason: {}",
relish_local_path.display(),
relish_timeline_id,
e
);
upload_queue
.lock()
.unwrap()
.push_back((relish_timeline_id, relish_local_path))
} else {
log::debug!("Relish successfully uploaded");
}
}
async fn upload_relish<P, S: RelishStorage<RelishStoragePath = P>>(
relish_storage: &S,
page_server_workdir: &Path,
relish_local_path: &Path,
) -> anyhow::Result<()> {
let destination = S::derive_destination(page_server_workdir, relish_local_path)?;
relish_storage
.upload_relish(relish_local_path, &destination)
.await
}

View File

@@ -1,52 +0,0 @@
use std::time::Duration;
use std::{collections::BinaryHeap, sync::Mutex, thread};
use crate::{relish_storage::RelishStorage, PageServerConf};
lazy_static::lazy_static! {
static ref UPLOAD_QUEUE: Mutex<BinaryHeap<SyncTask>> = Mutex::new(BinaryHeap::new());
}
pub fn schedule_timeline_upload(_local_timeline: ()) {
// UPLOAD_QUEUE
// .lock()
// .unwrap()
// .push(SyncTask::Upload(local_timeline))
}
#[derive(Debug, PartialEq, Eq, PartialOrd, Ord)]
enum SyncTask {}
pub fn run_storage_sync_thread<
P: std::fmt::Debug,
S: 'static + RelishStorage<RelishStoragePath = P>,
>(
config: &'static PageServerConf,
relish_storage: S,
max_concurrent_sync: usize,
) -> anyhow::Result<Option<thread::JoinHandle<anyhow::Result<()>>>> {
let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()?;
let handle = thread::Builder::new()
.name("Queue based relish storage sync".to_string())
.spawn(move || loop {
let mut queue_accessor = UPLOAD_QUEUE.lock().unwrap();
log::debug!("Upload queue length: {}", queue_accessor.len());
let next_task = queue_accessor.pop();
drop(queue_accessor);
match next_task {
Some(task) => runtime.block_on(async {
// suppress warnings
let _ = (config, task, &relish_storage, max_concurrent_sync);
todo!("omitted for brevity")
}),
None => {
thread::sleep(Duration::from_secs(1));
continue;
}
}
})?;
Ok(Some(handle))
}

View File

@@ -28,14 +28,18 @@ pub trait Repository: Send + Sync {
///
/// 'timelineid' specifies the timeline to GC, or None for all.
/// `horizon` specifies delta from last lsn to preserve all object versions (pitr interval).
/// `checkpoint_before_gc` parameter is used to force compaction of storage before CG
/// to make tests more deterministic.
/// TODO Do we still need it or we can call checkpoint explicitly in tests where needed?
/// `compact` parameter is used to force compaction of storage.
/// some storage implementation are based on lsm tree and require periodic merge (compaction).
/// usually storage implementation determines itself when compaction should be performed.
/// but for gc tests it way be useful to force compaction just after completion of gc iteration
/// to make sure that all detected garbage is removed.
/// so right now `compact` is set to true when gc explicitly requested through page srver api,
/// and is st to false in gc threads which infinitely repeats gc iterations in loop.
fn gc_iteration(
&self,
timelineid: Option<ZTimelineId>,
horizon: u64,
checkpoint_before_gc: bool,
compact: bool,
) -> Result<GcResult>;
}
@@ -48,7 +52,6 @@ pub struct GcResult {
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
@@ -56,7 +59,6 @@ pub struct GcResult {
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
@@ -69,7 +71,6 @@ impl AddAssign for GcResult {
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;
@@ -77,7 +78,6 @@ impl AddAssign for GcResult {
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;
@@ -212,18 +212,12 @@ mod tests {
use crate::layered_repository::LayeredRepository;
use crate::walredo::{WalRedoError, WalRedoManager};
use crate::PageServerConf;
use hex_literal::hex;
use postgres_ffi::pg_constants;
use postgres_ffi::xlog_utils::SIZEOF_CHECKPOINT;
use std::fs;
use std::path::PathBuf;
use std::str::FromStr;
use zenith_utils::zid::ZTenantId;
const TIMELINE_ID: ZTimelineId =
ZTimelineId::from_array(hex!("11223344556677881122334455667788"));
const NEW_TIMELINE_ID: ZTimelineId =
ZTimelineId::from_array(hex!("AA223344556677881122334455667788"));
/// Arbitrary relation tag, for testing.
const TESTREL_A: RelishTag = RelishTag::Relation(RelTag {
spcnode: 0,
@@ -259,53 +253,39 @@ mod tests {
static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; 8192]);
static ZERO_CHECKPOINT: Bytes = Bytes::from_static(&[0u8; SIZEOF_CHECKPOINT]);
struct RepoHarness {
conf: &'static PageServerConf,
tenant_id: ZTenantId,
}
fn get_test_repo(test_name: &str) -> Result<Box<dyn Repository>> {
let repo_dir = PageServerConf::test_repo_dir(test_name);
let _ = fs::remove_dir_all(&repo_dir);
fs::create_dir_all(&repo_dir)?;
fs::create_dir_all(&repo_dir.join("timelines"))?;
impl RepoHarness {
fn create(test_name: &'static str) -> Result<Self> {
let repo_dir = PageServerConf::test_repo_dir(test_name);
let _ = fs::remove_dir_all(&repo_dir);
fs::create_dir_all(&repo_dir)?;
fs::create_dir_all(&repo_dir.join("timelines"))?;
let conf = PageServerConf::dummy_conf(repo_dir);
// Make a static copy of the config. This can never be free'd, but that's
// OK in a test.
let conf: &'static PageServerConf = Box::leak(Box::new(conf));
let tenantid = ZTenantId::generate();
fs::create_dir_all(conf.tenant_path(&tenantid)).unwrap();
let conf = PageServerConf::dummy_conf(repo_dir);
// Make a static copy of the config. This can never be free'd, but that's
// OK in a test.
let conf: &'static PageServerConf = Box::leak(Box::new(conf));
let walredo_mgr = TestRedoManager {};
let tenant_id = ZTenantId::generate();
fs::create_dir_all(conf.tenant_path(&tenant_id))?;
let repo = Box::new(LayeredRepository::new(
conf,
Arc::new(walredo_mgr),
tenantid,
));
Ok(Self { conf, tenant_id })
}
fn load(&self) -> Box<dyn Repository> {
let walredo_mgr = Arc::new(TestRedoManager);
Box::new(LayeredRepository::new(
self.conf,
walredo_mgr,
self.tenant_id,
false,
))
}
fn timeline_path(&self, timeline_id: &ZTimelineId) -> PathBuf {
self.conf.timeline_path(timeline_id, &self.tenant_id)
}
Ok(repo)
}
#[test]
fn test_relsize() -> Result<()> {
let repo = RepoHarness::create("test_relsize")?.load();
let repo = get_test_repo("test_relsize")?;
// get_timeline() with non-existent timeline id should fail
//repo.get_timeline("11223344556677881122334455667788");
// Create timeline to work on
let tline = repo.create_empty_timeline(TIMELINE_ID)?;
let timelineid = ZTimelineId::from_str("11223344556677881122334455667788").unwrap();
let tline = repo.create_empty_timeline(timelineid)?;
tline.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?;
tline.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?;
@@ -415,148 +395,13 @@ mod tests {
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)?;
tline.put_page_image(TESTREL_A, 0, Lsn(0x20), TEST_IMG("foo blk 0 at 2"))?;
tline.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
tline.drop_relish(TESTREL_A, Lsn(0x30))?;
tline.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
tline.put_page_image(TESTREL_A, 0, Lsn(0x40), TEST_IMG("foo blk 0 at 4"))?;
tline.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)?;
//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);
tline.put_page_image(TESTREL_A, blkno, lsn, TEST_IMG(&data))?;
}
tline.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
tline.put_truncation(TESTREL_A, Lsn(0x60), 1)?;
tline.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);
tline.put_page_image(TESTREL_A, blkno, lsn, TEST_IMG(&data))?;
}
tline.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)?;
let repo = get_test_repo("test_large_rel")?;
let timelineid = ZTimelineId::from_str("11223344556677881122334455667788").unwrap();
let tline = repo.create_empty_timeline(timelineid)?;
let mut lsn = 0x10;
for blknum in 0..pg_constants::RELSEG_SIZE + 1 {
@@ -616,9 +461,13 @@ mod tests {
/// Test list_rels() function, with branches and dropped relations
///
#[test]
// FIXME: The last assertion in this test is currently failing, see
// https://github.com/zenithdb/zenith/issues/502. Ignore the failure until that's fixed.
#[ignore]
fn test_list_rels_drop() -> Result<()> {
let repo = RepoHarness::create("test_list_rels_drop")?.load();
let tline = repo.create_empty_timeline(TIMELINE_ID)?;
let repo = get_test_repo("test_list_rels_drop")?;
let timelineid = ZTimelineId::from_str("11223344556677881122334455667788").unwrap();
let tline = repo.create_empty_timeline(timelineid)?;
const TESTDB: u32 = 111;
// Import initial dummy checkpoint record, otherwise the get_timeline() call
@@ -636,8 +485,9 @@ mod tests {
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(NEW_TIMELINE_ID)?;
let newtimelineid = ZTimelineId::from_str("AA223344556677881122334455667788").unwrap();
repo.branch_timeline(timelineid, newtimelineid, Lsn(0x30))?;
let newtline = repo.get_timeline(newtimelineid)?;
assert!(newtline
.list_rels(0, TESTDB, Lsn(0x30))?
@@ -657,8 +507,9 @@ mod tests {
// Run checkpoint and garbage collection and check that it's still not visible
newtline.checkpoint()?;
repo.gc_iteration(Some(NEW_TIMELINE_ID), 0, true)?;
repo.gc_iteration(Some(newtimelineid), 0, true)?;
// FIXME: this is currently failing
assert!(!newtline
.list_rels(0, TESTDB, Lsn(0x40))?
.contains(&TESTREL_A));
@@ -671,8 +522,9 @@ mod tests {
///
#[test]
fn test_branch() -> Result<()> {
let repo = RepoHarness::create("test_branch")?.load();
let tline = repo.create_empty_timeline(TIMELINE_ID)?;
let repo = get_test_repo("test_branch")?;
let timelineid = ZTimelineId::from_str("11223344556677881122334455667788").unwrap();
let tline = repo.create_empty_timeline(timelineid)?;
// Import initial dummy checkpoint record, otherwise the get_timeline() call
// after branching fails below
@@ -690,8 +542,9 @@ mod tests {
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))?;
let newtline = repo.get_timeline(NEW_TIMELINE_ID)?;
let newtimelineid = ZTimelineId::from_str("AA223344556677881122334455667788").unwrap();
repo.branch_timeline(timelineid, newtimelineid, Lsn(0x30))?;
let newtline = repo.get_timeline(newtimelineid)?;
newtline.put_page_image(TESTREL_A, 0, Lsn(0x40), TEST_IMG("bar blk 0 at 4"))?;
newtline.advance_last_record_lsn(Lsn(0x40));
@@ -719,89 +572,8 @@ mod tests {
Ok(())
}
#[test]
fn corrupt_metadata() -> Result<()> {
const TEST_NAME: &str = "corrupt_metadata";
let harness = RepoHarness::create(TEST_NAME)?;
let repo = harness.load();
repo.create_empty_timeline(TIMELINE_ID)?;
drop(repo);
let metadata_path = harness.timeline_path(&TIMELINE_ID).join("metadata");
assert!(metadata_path.is_file());
let mut metadata_bytes = std::fs::read(&metadata_path)?;
assert_eq!(metadata_bytes.len(), 512);
metadata_bytes[512 - 4 - 2] ^= 1;
std::fs::write(metadata_path, metadata_bytes)?;
let new_repo = harness.load();
let err = new_repo.get_timeline(TIMELINE_ID).err().unwrap();
assert!(err.to_string().contains("checksum"));
Ok(())
}
#[test]
fn future_layerfiles() -> Result<()> {
const TEST_NAME: &str = "future_layerfiles";
let harness = RepoHarness::create(TEST_NAME)?;
let repo = harness.load();
repo.create_empty_timeline(TIMELINE_ID)?;
drop(repo);
let timeline_path = harness.timeline_path(&TIMELINE_ID);
let make_empty_file = |filename: &str| -> std::io::Result<()> {
let path = timeline_path.join(filename);
assert!(!path.exists());
std::fs::write(&path, &[])?;
Ok(())
};
let image_filename = format!("pg_control_0_{:016X}", 8000);
let delta_filename = format!("pg_control_0_{:016X}_{:016X}", 8000, 8008);
make_empty_file(&image_filename)?;
make_empty_file(&delta_filename)?;
let new_repo = harness.load();
new_repo.get_timeline(TIMELINE_ID).unwrap();
drop(new_repo);
let check_old = |filename: &str, num: u32| {
let path = timeline_path.join(filename);
assert!(!path.exists());
let backup_path = timeline_path.join(format!("{}.{}.old", filename, num));
assert!(backup_path.exists());
};
check_old(&image_filename, 0);
check_old(&delta_filename, 0);
make_empty_file(&image_filename)?;
make_empty_file(&delta_filename)?;
let new_repo = harness.load();
new_repo.get_timeline(TIMELINE_ID).unwrap();
drop(new_repo);
check_old(&image_filename, 0);
check_old(&delta_filename, 0);
check_old(&image_filename, 1);
check_old(&delta_filename, 1);
Ok(())
}
// Mock WAL redo manager that doesn't do much
struct TestRedoManager;
struct TestRedoManager {}
impl WalRedoManager for TestRedoManager {
fn request_redo(

View File

@@ -11,7 +11,7 @@ use std::fs::File;
use std::io::Read;
use std::path::Path;
use anyhow::{bail, Result};
use anyhow::Result;
use bytes::{Buf, Bytes};
use crate::relish::*;
@@ -173,7 +173,8 @@ fn import_relfile(
break;
}
_ => {
bail!("error reading file {}: {:#}", path.display(), e);
error!("error reading file: {:?} ({})", path, e);
break;
}
},
};
@@ -267,7 +268,8 @@ fn import_slru_file(timeline: &dyn Timeline, lsn: Lsn, slru: SlruKind, path: &Pa
break;
}
_ => {
bail!("error reading file {}: {:#}", path.display(), e);
error!("error reading file: {:?} ({})", path, e);
break;
}
},
};

View File

@@ -9,84 +9,46 @@ use crate::PageServerConf;
use anyhow::{anyhow, bail, Context, Result};
use lazy_static::lazy_static;
use log::info;
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::fs;
use std::str::FromStr;
use std::sync::{Arc, Mutex, MutexGuard};
use std::sync::{Arc, Mutex};
use zenith_utils::zid::{ZTenantId, ZTimelineId};
lazy_static! {
static ref REPOSITORY: Mutex<HashMap<ZTenantId, Arc<dyn Repository>>> =
pub static ref REPOSITORY: Mutex<HashMap<ZTenantId, Arc<dyn Repository>>> =
Mutex::new(HashMap::new());
}
fn access_repository() -> MutexGuard<'static, HashMap<ZTenantId, Arc<dyn Repository>>> {
REPOSITORY.lock().unwrap()
}
pub fn init(conf: &'static PageServerConf) {
let mut m = access_repository();
let mut m = REPOSITORY.lock().unwrap();
for dir_entry in fs::read_dir(conf.tenants_path()).unwrap() {
let tenantid =
ZTenantId::from_str(dir_entry.unwrap().file_name().to_str().unwrap()).unwrap();
let repo = init_repo(conf, tenantid);
// Set up a WAL redo manager, for applying WAL records.
let walredo_mgr = PostgresRedoManager::new(conf, tenantid);
// Set up an object repository, for actual data storage.
let repo = Arc::new(LayeredRepository::new(
conf,
Arc::new(walredo_mgr),
tenantid,
));
LayeredRepository::launch_checkpointer_thread(conf, repo.clone());
LayeredRepository::launch_gc_thread(conf, repo.clone());
info!("initialized storage for tenant: {}", &tenantid);
m.insert(tenantid, repo);
}
}
fn init_repo(conf: &'static PageServerConf, tenant_id: ZTenantId) -> Arc<LayeredRepository> {
// 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::new(LayeredRepository::new(
conf,
Arc::new(walredo_mgr),
tenant_id,
true,
));
LayeredRepository::launch_checkpointer_thread(conf, repo.clone());
LayeredRepository::launch_gc_thread(conf, repo.clone());
repo
}
// TODO kb Currently unused function, will later be used when the relish storage downloads a new layer.
// Relevant PR: https://github.com/zenithdb/zenith/pull/686
pub fn register_relish_download(
conf: &'static PageServerConf,
tenant_id: ZTenantId,
timeline_id: ZTimelineId,
) {
log::info!(
"Registering new download, tenant id {}, timeline id: {}",
tenant_id,
timeline_id
);
match access_repository().entry(tenant_id) {
Entry::Occupied(o) => init_timeline(o.get().as_ref(), timeline_id),
Entry::Vacant(v) => {
log::info!("New repo initialized");
let new_repo = init_repo(conf, tenant_id);
init_timeline(new_repo.as_ref(), timeline_id);
v.insert(new_repo);
}
}
}
fn init_timeline(repo: &dyn Repository, timeline_id: ZTimelineId) {
match repo.get_timeline(timeline_id) {
Ok(_timeline) => log::info!("Successfully initialized timeline {}", timeline_id),
Err(e) => log::error!("Failed to init timeline {}, reason: {:#}", timeline_id, e),
}
}
pub fn create_repository_for_tenant(
conf: &'static PageServerConf,
tenantid: ZTenantId,
) -> Result<()> {
let mut m = access_repository();
let mut m = REPOSITORY.lock().unwrap();
// First check that the tenant doesn't exist already
if m.get(&tenantid).is_some() {
@@ -101,13 +63,14 @@ pub fn create_repository_for_tenant(
}
pub fn insert_repository_for_tenant(tenantid: ZTenantId, repo: Arc<dyn Repository>) {
access_repository().insert(tenantid, repo);
let o = &mut REPOSITORY.lock().unwrap();
o.insert(tenantid, repo);
}
pub fn get_repository_for_tenant(tenantid: ZTenantId) -> Result<Arc<dyn Repository>> {
access_repository()
.get(&tenantid)
.map(Arc::clone)
let o = &REPOSITORY.lock().unwrap();
o.get(&tenantid)
.map(|repo| Arc::clone(repo))
.ok_or_else(|| anyhow!("repository not found for tenant name {}", tenantid))
}

View File

@@ -10,7 +10,7 @@ use crate::restore_local_repo;
use crate::tenant_mgr;
use crate::waldecoder::*;
use crate::PageServerConf;
use anyhow::{bail, Error, Result};
use anyhow::{Error, Result};
use lazy_static::lazy_static;
use log::*;
use postgres::fallible_iterator::FallibleIterator;
@@ -20,7 +20,6 @@ use postgres_ffi::xlog_utils::*;
use postgres_ffi::*;
use postgres_protocol::message::backend::ReplicationMessage;
use postgres_types::PgLsn;
use std::cell::Cell;
use std::cmp::{max, min};
use std::collections::HashMap;
use std::fs;
@@ -28,7 +27,6 @@ use std::str::FromStr;
use std::sync::Mutex;
use std::thread;
use std::thread::sleep;
use std::thread_local;
use std::time::{Duration, SystemTime};
use zenith_utils::lsn::Lsn;
use zenith_utils::zid::ZTenantId;
@@ -46,13 +44,6 @@ lazy_static! {
Mutex::new(HashMap::new());
}
thread_local! {
// Boolean that is true only for WAL receiver threads
//
// This is used in `wait_lsn` to guard against usage that might lead to a deadlock.
pub(crate) static IS_WAL_RECEIVER: Cell<bool> = Cell::new(false);
}
// Launch a new WAL receiver, or tell one that's running about change in connection string
pub fn launch_wal_receiver(
conf: &'static PageServerConf,
@@ -73,10 +64,12 @@ pub fn launch_wal_receiver(
receivers.insert(timelineid, receiver);
// Also launch a new thread to handle this connection
//
// NOTE: This thread name is checked in the assertion in wait_lsn. If you change
// this, make sure you update the assertion too.
let _walreceiver_thread = thread::Builder::new()
.name("WAL receiver thread".into())
.spawn(move || {
IS_WAL_RECEIVER.with(|c| c.set(true));
thread_main(conf, timelineid, tenantid);
})
.unwrap();
@@ -165,7 +158,7 @@ fn walreceiver_main(
let mut startpoint = last_rec_lsn;
if startpoint == Lsn(0) {
bail!("No previous WAL position");
error!("No previous WAL position");
}
// There might be some padding after the last full record, skip it.

698
pageserver/src/walredo.rs Normal file
View File

@@ -0,0 +1,698 @@
//!
//! WAL redo. This service runs PostgreSQL in a special wal_redo mode
//! to apply given WAL records over an old page image and return new
//! page image.
//!
//! We rely on Postgres to perform WAL redo for us. We launch a
//! postgres process in special "wal redo" mode that's similar to
//! single-user mode. We then pass the previous page image, if any,
//! and all the WAL records we want to apply, to the postgres
//! process. Then we get the page image back. Communication with the
//! postgres process happens via stdin/stdout
//!
//! See src/backend/tcop/zenith_wal_redo.c for the other side of
//! this communication.
//!
//! The Postgres process is assumed to be secure against malicious WAL
//! records. It achieves it by dropping privileges before replaying
//! any WAL records, so that even if an attacker hijacks the Postgres
//! process, he cannot escape out of it.
//!
use byteorder::{ByteOrder, LittleEndian};
use bytes::{Buf, BufMut, Bytes, BytesMut};
use lazy_static::lazy_static;
use log::*;
use serde::{Deserialize, Serialize};
use std::cell::RefCell;
use std::fs;
use std::fs::OpenOptions;
use std::io::prelude::*;
use std::io::Error;
use std::path::PathBuf;
use std::process::Stdio;
use std::sync::Mutex;
use std::time::Duration;
use std::time::Instant;
use tokio::io::AsyncBufReadExt;
use tokio::io::{AsyncReadExt, AsyncWriteExt};
use tokio::process::{ChildStdin, ChildStdout, Command};
use tokio::time::timeout;
use zenith_metrics::{register_histogram, register_int_counter, Histogram, IntCounter};
use zenith_utils::bin_ser::BeSer;
use zenith_utils::lsn::Lsn;
use zenith_utils::zid::ZTenantId;
use crate::relish::*;
use crate::repository::WALRecord;
use crate::waldecoder::XlMultiXactCreate;
use crate::waldecoder::XlXactParsedRecord;
use crate::PageServerConf;
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;
use postgres_ffi::nonrelfile_utils::transaction_id_set_status;
use postgres_ffi::pg_constants;
use postgres_ffi::XLogRecord;
///
/// `RelTag` + block number (`blknum`) gives us a unique id of the page in the cluster.
///
/// In Postgres `BufferTag` structure is used for exactly the same purpose.
/// [See more related comments here](https://github.com/postgres/postgres/blob/99c5852e20a0987eca1c38ba0c09329d4076b6a0/src/include/storage/buf_internals.h#L91).
///
#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Serialize, Deserialize)]
pub struct BufferTag {
pub rel: RelTag,
pub blknum: u32,
}
///
/// WAL Redo Manager is responsible for replaying WAL records.
///
/// Callers use the WAL redo manager through this abstract interface,
/// which makes it easy to mock it in tests.
pub trait WalRedoManager: Send + Sync {
/// Apply some WAL records.
///
/// The caller passes an old page image, and WAL records that should be
/// applied over it. The return value is a new page image, after applying
/// the reords.
fn request_redo(
&self,
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError>;
}
///
/// A dummy WAL Redo Manager implementation that doesn't allow replaying
/// anything. Currently used during bootstrapping (zenith init), to create
/// a Repository object without launching the real WAL redo process.
///
pub struct DummyRedoManager {}
impl crate::walredo::WalRedoManager for DummyRedoManager {
fn request_redo(
&self,
_rel: RelishTag,
_blknum: u32,
_lsn: Lsn,
_base_img: Option<Bytes>,
_records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError> {
Err(WalRedoError::InvalidState)
}
}
static TIMEOUT: Duration = Duration::from_secs(20);
// Metrics collected on WAL redo operations
//
// We collect the time spent in actual WAL redo ('redo'), and time waiting
// for access to the postgres process ('wait') since there is only one for
// each tenant.
lazy_static! {
static ref WAL_REDO_TIME: Histogram =
register_histogram!("pageserver_wal_redo_time", "Time spent on WAL redo")
.expect("failed to define a metric");
static ref WAL_REDO_WAIT_TIME: Histogram = register_histogram!(
"pageserver_wal_redo_wait_time",
"Time spent waiting for access to the WAL redo process"
)
.expect("failed to define a metric");
static ref WAL_REDO_RECORD_COUNTER: IntCounter = register_int_counter!(
"pageserver_wal_records_replayed",
"Number of WAL records replayed"
)
.unwrap();
}
///
/// This is the real implementation that uses a Postgres process to
/// perform WAL replay. Only one thread can use the processs at a time,
/// that is controlled by the Mutex. In the future, we might want to
/// launch a pool of processes to allow concurrent replay of multiple
/// records.
///
pub struct PostgresRedoManager {
tenantid: ZTenantId,
conf: &'static PageServerConf,
runtime: tokio::runtime::Runtime,
process: Mutex<Option<PostgresRedoProcess>>,
}
#[derive(Debug)]
struct WalRedoRequest {
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
}
/// An error happened in WAL redo
#[derive(Debug, thiserror::Error)]
pub enum WalRedoError {
#[error(transparent)]
IoError(#[from] std::io::Error),
#[error("cannot perform WAL redo now")]
InvalidState,
}
///
/// Public interface of WAL redo manager
///
impl WalRedoManager for PostgresRedoManager {
///
/// Request the WAL redo manager to apply some WAL records
///
/// The WAL redo is handled by a separate thread, so this just sends a request
/// to the thread and waits for response.
///
fn request_redo(
&self,
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError> {
let start_time;
let lock_time;
let end_time;
let request = WalRedoRequest {
rel,
blknum,
lsn,
base_img,
records,
};
start_time = Instant::now();
let result = {
let mut process_guard = self.process.lock().unwrap();
lock_time = Instant::now();
// launch the WAL redo process on first use
if process_guard.is_none() {
let p = self
.runtime
.block_on(PostgresRedoProcess::launch(self.conf, &self.tenantid))?;
*process_guard = Some(p);
}
let process = (*process_guard).as_ref().unwrap();
self.runtime
.block_on(self.handle_apply_request(process, &request))
};
end_time = Instant::now();
WAL_REDO_WAIT_TIME.observe(lock_time.duration_since(start_time).as_secs_f64());
WAL_REDO_TIME.observe(end_time.duration_since(lock_time).as_secs_f64());
result
}
}
impl PostgresRedoManager {
///
/// Create a new PostgresRedoManager.
///
pub fn new(conf: &'static PageServerConf, tenantid: ZTenantId) -> PostgresRedoManager {
// We block on waiting for requests on the walredo request channel, but
// use async I/O to communicate with the child process. Initialize the
// runtime for the async part.
let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()
.unwrap();
// The actual process is launched lazily, on first request.
PostgresRedoManager {
runtime,
tenantid,
conf,
process: Mutex::new(None),
}
}
///
/// Process one request for WAL redo.
///
async fn handle_apply_request(
&self,
process: &PostgresRedoProcess,
request: &WalRedoRequest,
) -> Result<Bytes, WalRedoError> {
let rel = request.rel;
let blknum = request.blknum;
let lsn = request.lsn;
let base_img = request.base_img.clone();
let records = &request.records;
let nrecords = records.len();
let start = Instant::now();
let apply_result: Result<Bytes, Error>;
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).await;
} else {
// Non-relational WAL records are handled here, with custom code that has the
// same effects as the corresponding Postgres WAL redo function.
const ZERO_PAGE: [u8; 8192] = [0u8; 8192];
let mut page = BytesMut::new();
if let Some(fpi) = base_img {
// If full-page image is provided, then use it...
page.extend_from_slice(&fpi[..]);
} else {
// otherwise initialize page with zeros
page.extend_from_slice(&ZERO_PAGE);
}
// Apply all collected WAL records
for record in records {
let mut buf = record.rec.clone();
WAL_REDO_RECORD_COUNTER.inc();
// 1. Parse XLogRecord struct
// FIXME: refactor to avoid code duplication.
let xlogrec = XLogRecord::from_bytes(&mut buf);
//move to main data
// TODO probably, we should store some records in our special format
// to avoid this weird parsing on replay
let skip = (record.main_data_offset - pg_constants::SIZEOF_XLOGRECORD) as usize;
if buf.remaining() > skip {
buf.advance(skip);
}
if xlogrec.xl_rmid == pg_constants::RM_XACT_ID {
// Transaction manager stuff
let rec_segno = match rel {
RelishTag::Slru { slru, segno } => {
if slru != SlruKind::Clog {
panic!("Not valid XACT relish tag {:?}", rel);
}
segno
}
_ => panic!("Not valid XACT relish tag {:?}", rel),
};
let parsed_xact =
XlXactParsedRecord::decode(&mut buf, xlogrec.xl_xid, xlogrec.xl_info);
if parsed_xact.info == pg_constants::XLOG_XACT_COMMIT
|| parsed_xact.info == pg_constants::XLOG_XACT_COMMIT_PREPARED
{
transaction_id_set_status(
parsed_xact.xid,
pg_constants::TRANSACTION_STATUS_COMMITTED,
&mut page,
);
for subxact in &parsed_xact.subxacts {
let pageno = *subxact as u32 / pg_constants::CLOG_XACTS_PER_PAGE;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
// only update xids on the requested page
if rec_segno == segno && blknum == rpageno {
transaction_id_set_status(
*subxact,
pg_constants::TRANSACTION_STATUS_COMMITTED,
&mut page,
);
}
}
} else if parsed_xact.info == pg_constants::XLOG_XACT_ABORT
|| parsed_xact.info == pg_constants::XLOG_XACT_ABORT_PREPARED
{
transaction_id_set_status(
parsed_xact.xid,
pg_constants::TRANSACTION_STATUS_ABORTED,
&mut page,
);
for subxact in &parsed_xact.subxacts {
let pageno = *subxact as u32 / pg_constants::CLOG_XACTS_PER_PAGE;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
// only update xids on the requested page
if rec_segno == segno && blknum == rpageno {
transaction_id_set_status(
*subxact,
pg_constants::TRANSACTION_STATUS_ABORTED,
&mut page,
);
}
}
}
} else if xlogrec.xl_rmid == pg_constants::RM_MULTIXACT_ID {
// Multixact operations
let info = xlogrec.xl_info & pg_constants::XLR_RMGR_INFO_MASK;
if info == pg_constants::XLOG_MULTIXACT_CREATE_ID {
let xlrec = XlMultiXactCreate::decode(&mut buf);
if let RelishTag::Slru {
slru,
segno: rec_segno,
} = rel
{
if slru == SlruKind::MultiXactMembers {
for i in 0..xlrec.nmembers {
let pageno =
i / pg_constants::MULTIXACT_MEMBERS_PER_PAGE as u32;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
if segno == rec_segno && rpageno == blknum {
// update only target block
let offset = xlrec.moff + i;
let memberoff = mx_offset_to_member_offset(offset);
let flagsoff = mx_offset_to_flags_offset(offset);
let bshift = mx_offset_to_flags_bitshift(offset);
let mut flagsval =
LittleEndian::read_u32(&page[flagsoff..flagsoff + 4]);
flagsval &= !(((1
<< pg_constants::MXACT_MEMBER_BITS_PER_XACT)
- 1)
<< bshift);
flagsval |= xlrec.members[i as usize].status << bshift;
LittleEndian::write_u32(
&mut page[flagsoff..flagsoff + 4],
flagsval,
);
LittleEndian::write_u32(
&mut page[memberoff..memberoff + 4],
xlrec.members[i as usize].xid,
);
}
}
} else {
// Multixact offsets SLRU
let offs = (xlrec.mid
% pg_constants::MULTIXACT_OFFSETS_PER_PAGE as u32
* 4) as usize;
LittleEndian::write_u32(&mut page[offs..offs + 4], xlrec.moff);
}
} else {
panic!();
}
} else {
panic!();
}
}
}
apply_result = Ok::<Bytes, Error>(page.freeze());
}
let duration = start.elapsed();
let result: Result<Bytes, WalRedoError>;
debug!(
"applied {} WAL records in {} ms to reconstruct page image at LSN {}",
nrecords,
duration.as_millis(),
lsn
);
if let Err(e) = apply_result {
error!("could not apply WAL records: {}", e);
result = Err(WalRedoError::IoError(e));
} else {
let img = apply_result.unwrap();
result = Ok(img);
}
// The caller is responsible for sending the response
result
}
}
///
/// Handle to the Postgres WAL redo process
///
struct PostgresRedoProcess {
stdin: RefCell<ChildStdin>,
stdout: RefCell<ChildStdout>,
}
impl PostgresRedoProcess {
//
// Start postgres binary in special WAL redo mode.
//
async fn launch(
conf: &PageServerConf,
tenantid: &ZTenantId,
) -> Result<PostgresRedoProcess, Error> {
// FIXME: We need a dummy Postgres cluster to run the process in. Currently, we
// just create one with constant name. That fails if you try to launch more than
// one WAL redo manager concurrently.
let datadir = conf.tenant_path(tenantid).join("wal-redo-datadir");
// Create empty data directory for wal-redo postgres, deleting old one first.
if datadir.exists() {
info!("directory {:?} exists, removing", &datadir);
if let Err(e) = fs::remove_dir_all(&datadir) {
error!("could not remove old wal-redo-datadir: {:?}", e);
}
}
info!("running initdb in {:?}", datadir.display());
let initdb = Command::new(conf.pg_bin_dir().join("initdb"))
.args(&["-D", datadir.to_str().unwrap()])
.arg("-N")
.env_clear()
.env("LD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("DYLD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.output()
.await
.expect("failed to execute initdb");
if !initdb.status.success() {
panic!(
"initdb failed: {}\nstderr:\n{}",
std::str::from_utf8(&initdb.stdout).unwrap(),
std::str::from_utf8(&initdb.stderr).unwrap()
);
} else {
// Limit shared cache for wal-redo-postres
let mut config = OpenOptions::new()
.append(true)
.open(PathBuf::from(&datadir).join("postgresql.conf"))?;
config.write_all(b"shared_buffers=128kB\n")?;
config.write_all(b"fsync=off\n")?;
config.write_all(b"shared_preload_libraries=zenith\n")?;
config.write_all(b"zenith.wal_redo=on\n")?;
}
// Start postgres itself
let mut child = Command::new(conf.pg_bin_dir().join("postgres"))
.arg("--wal-redo")
.stdin(Stdio::piped())
.stderr(Stdio::piped())
.stdout(Stdio::piped())
.env_clear()
.env("LD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("DYLD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("PGDATA", &datadir)
.spawn()
.expect("postgres --wal-redo command failed to start");
info!(
"launched WAL redo postgres process on {:?}",
datadir.display()
);
let stdin = child.stdin.take().expect("failed to open child's stdin");
let stderr = child.stderr.take().expect("failed to open child's stderr");
let stdout = child.stdout.take().expect("failed to open child's stdout");
// This async block reads the child's stderr, and forwards it to the logger
let f_stderr = async {
let mut stderr_buffered = tokio::io::BufReader::new(stderr);
let mut line = String::new();
loop {
let res = stderr_buffered.read_line(&mut line).await;
if res.is_err() {
debug!("could not convert line to utf-8");
continue;
}
if res.unwrap() == 0 {
break;
}
error!("wal-redo-postgres: {}", line.trim());
line.clear();
}
Ok::<(), Error>(())
};
tokio::spawn(f_stderr);
Ok(PostgresRedoProcess {
stdin: RefCell::new(stdin),
stdout: RefCell::new(stdout),
})
}
//
// Apply given WAL records ('records') over an old page image. Returns
// new page image.
//
async fn apply_wal_records(
&self,
tag: BufferTag,
base_img: Option<Bytes>,
records: &[WALRecord],
) -> Result<Bytes, std::io::Error> {
let mut stdin = self.stdin.borrow_mut();
let mut stdout = self.stdout.borrow_mut();
// We do three things simultaneously: send the old base image and WAL records to
// the child process's stdin, read the result from child's stdout, and forward any logging
// information that the child writes to its stderr to the page server's log.
//
// 'f_stdin' handles writing the base image and WAL records to the child process.
// 'f_stdout' below reads the result back. And 'f_stderr', which was spawned into the
// tokio runtime in the 'launch' function already, forwards the logging.
let f_stdin = async {
// Send base image, if any. (If the record initializes the page, previous page
// version is not needed.)
timeout(
TIMEOUT,
stdin.write_all(&build_begin_redo_for_block_msg(tag)),
)
.await??;
if base_img.is_some() {
timeout(
TIMEOUT,
stdin.write_all(&build_push_page_msg(tag, base_img.unwrap())),
)
.await??;
}
// Send WAL records.
for rec in records.iter() {
let r = rec.clone();
WAL_REDO_RECORD_COUNTER.inc();
stdin
.write_all(&build_apply_record_msg(r.lsn, r.rec))
.await?;
//debug!("sent WAL record to wal redo postgres process ({:X}/{:X}",
// r.lsn >> 32, r.lsn & 0xffff_ffff);
}
//debug!("sent {} WAL records to wal redo postgres process ({:X}/{:X}",
// records.len(), lsn >> 32, lsn & 0xffff_ffff);
// Send GetPage command to get the result back
timeout(TIMEOUT, stdin.write_all(&build_get_page_msg(tag))).await??;
timeout(TIMEOUT, stdin.flush()).await??;
//debug!("sent GetPage for {}", tag.blknum);
Ok::<(), Error>(())
};
// Read back new page image
let f_stdout = async {
let mut buf = [0u8; 8192];
timeout(TIMEOUT, stdout.read_exact(&mut buf)).await??;
//debug!("got response for {}", tag.blknum);
Ok::<[u8; 8192], Error>(buf)
};
let res = tokio::try_join!(f_stdout, f_stdin)?;
let buf = res.0;
Ok::<Bytes, Error>(Bytes::from(std::vec::Vec::from(buf)))
}
}
// Functions for constructing messages to send to the postgres WAL redo
// process. See vendor/postgres/src/backend/tcop/zenith_wal_redo.c for
// explanation of the protocol.
fn build_begin_redo_for_block_msg(tag: BufferTag) -> Bytes {
let len = 4 + 1 + 4 * 4;
let mut buf = BytesMut::with_capacity(1 + len);
buf.put_u8(b'B');
buf.put_u32(len as u32);
// FIXME: this is a temporary hack that should go away when we refactor
// the postgres protocol serialization + handlers.
//
// BytesMut is a dynamic growable buffer, used a lot in tokio code but
// not in the std library. To write to a BytesMut from a serde serializer,
// we need to either:
// - pre-allocate the required buffer space. This is annoying because we
// shouldn't care what the exact serialized size is-- that's the
// serializer's job.
// - Or, we need to create a temporary "writer" (which implements the
// `Write` trait). It's a bit awkward, because the writer consumes the
// underlying BytesMut, and we need to extract it later with
// `into_inner`.
let mut writer = buf.writer();
tag.ser_into(&mut writer)
.expect("serialize BufferTag should always succeed");
let buf = writer.into_inner();
debug_assert!(buf.len() == 1 + len);
buf.freeze()
}
fn build_push_page_msg(tag: BufferTag, base_img: Bytes) -> Bytes {
assert!(base_img.len() == 8192);
let len = 4 + 1 + 4 * 4 + base_img.len();
let mut buf = BytesMut::with_capacity(1 + len);
buf.put_u8(b'P');
buf.put_u32(len as u32);
let mut writer = buf.writer();
tag.ser_into(&mut writer)
.expect("serialize BufferTag should always succeed");
let mut buf = writer.into_inner();
buf.put(base_img);
debug_assert!(buf.len() == 1 + len);
buf.freeze()
}
fn build_apply_record_msg(endlsn: Lsn, rec: Bytes) -> Bytes {
let len = 4 + 8 + rec.len();
let mut buf = BytesMut::with_capacity(1 + len);
buf.put_u8(b'A');
buf.put_u32(len as u32);
buf.put_u64(endlsn.0);
buf.put(rec);
debug_assert!(buf.len() == 1 + len);
buf.freeze()
}
fn build_get_page_msg(tag: BufferTag) -> Bytes {
let len = 4 + 1 + 4 * 4;
let mut buf = BytesMut::with_capacity(1 + len);
buf.put_u8(b'G');
buf.put_u32(len as u32);
let mut writer = buf.writer();
tag.ser_into(&mut writer)
.expect("serialize BufferTag should always succeed");
let buf = writer.into_inner();
debug_assert!(buf.len() == 1 + len);
buf.freeze()
}

View File

@@ -1,398 +0,0 @@
//!
//! WAL redo. This service runs PostgreSQL in a special wal_redo mode
//! to apply given WAL records over an old page image and return new
//! page image.
//!
//! We rely on Postgres to perform WAL redo for us. We launch a
//! postgres process in special "wal redo" mode that's similar to
//! single-user mode. We then pass the previous page image, if any,
//! and all the WAL records we want to apply, to the postgres
//! process. Then we get the page image back. Communication with the
//! postgres process happens via stdin/stdout
//!
//! See src/backend/tcop/zenith_wal_redo.c for the other side of
//! this communication.
//!
//! The Postgres process is assumed to be secure against malicious WAL
//! records. It achieves it by dropping privileges before replaying
//! any WAL records, so that even if an attacker hijacks the Postgres
//! process, he cannot escape out of it.
mod nonrel;
mod process_utils;
mod request;
use bytes::Bytes;
use lazy_static::lazy_static;
use log::*;
use serde::Serialize;
use std::fs;
use std::fs::OpenOptions;
use std::io::prelude::*;
use std::io::BufReader;
use std::io::Error;
use std::path::PathBuf;
use std::process::ChildStdin;
use std::process::ChildStdout;
use std::process::Command;
use std::process::Stdio;
use std::sync::Mutex;
use std::time::Duration;
use std::time::Instant;
use zenith_metrics::{register_histogram, register_int_counter, Histogram, IntCounter};
use zenith_utils::lsn::Lsn;
use zenith_utils::zid::ZTenantId;
use crate::relish::*;
use crate::repository::WALRecord;
use crate::PageServerConf;
///
/// `RelTag` + block number (`blknum`) gives us a unique id of the page in the cluster.
///
/// In Postgres `BufferTag` structure is used for exactly the same purpose.
/// [See more related comments here](https://github.com/postgres/postgres/blob/99c5852e20a0987eca1c38ba0c09329d4076b6a0/src/include/storage/buf_internals.h#L91).
///
#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Serialize)]
pub struct BufferTag {
pub rel: RelTag,
pub blknum: u32,
}
///
/// WAL Redo Manager is responsible for replaying WAL records.
///
/// Callers use the WAL redo manager through this abstract interface,
/// which makes it easy to mock it in tests.
pub trait WalRedoManager: Send + Sync {
/// Apply some WAL records.
///
/// The caller passes an old page image, and WAL records that should be
/// applied over it. The return value is a new page image, after applying
/// the reords.
fn request_redo(
&self,
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError>;
}
///
/// A dummy WAL Redo Manager implementation that doesn't allow replaying
/// anything. Currently used during bootstrapping (zenith init), to create
/// a Repository object without launching the real WAL redo process.
///
pub struct DummyRedoManager {}
impl WalRedoManager for DummyRedoManager {
fn request_redo(
&self,
_rel: RelishTag,
_blknum: u32,
_lsn: Lsn,
_base_img: Option<Bytes>,
_records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError> {
Err(WalRedoError::InvalidState)
}
}
const TIMEOUT: Duration = Duration::from_secs(20);
// Metrics collected on WAL redo operations
//
// We collect the time spent in actual WAL redo ('redo'), and time waiting
// for access to the postgres process ('wait') since there is only one for
// each tenant.
lazy_static! {
static ref WAL_REDO_TIME: Histogram =
register_histogram!("pageserver_wal_redo_time", "Time spent on WAL redo")
.expect("failed to define a metric");
static ref WAL_REDO_WAIT_TIME: Histogram = register_histogram!(
"pageserver_wal_redo_wait_time",
"Time spent waiting for access to the WAL redo process"
)
.expect("failed to define a metric");
static ref WAL_REDO_RECORD_COUNTER: IntCounter = register_int_counter!(
"pageserver_wal_records_replayed",
"Number of WAL records replayed"
)
.unwrap();
}
///
/// This is the real implementation that uses a Postgres process to
/// perform WAL replay. Only one thread can use the processs at a time,
/// that is controlled by the Mutex. In the future, we might want to
/// launch a pool of processes to allow concurrent replay of multiple
/// records.
///
pub struct PostgresRedoManager {
tenantid: ZTenantId,
conf: &'static PageServerConf,
process: Mutex<Option<PostgresRedoProcess>>,
}
#[derive(Debug)]
struct WalRedoRequest {
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
}
/// An error happened in WAL redo
#[derive(Debug, thiserror::Error)]
pub enum WalRedoError {
#[error(transparent)]
IoError(#[from] std::io::Error),
#[error("cannot perform WAL redo now")]
InvalidState,
}
///
/// Public interface of WAL redo manager
///
impl WalRedoManager for PostgresRedoManager {
///
/// Request the WAL redo manager to apply some WAL records
///
/// The WAL redo is handled by a separate thread, so this just sends a request
/// to the thread and waits for response.
///
fn request_redo(
&self,
rel: RelishTag,
blknum: u32,
lsn: Lsn,
base_img: Option<Bytes>,
records: Vec<WALRecord>,
) -> Result<Bytes, WalRedoError> {
let start_time;
let lock_time;
let end_time;
let request = WalRedoRequest {
rel,
blknum,
lsn,
base_img,
records,
};
start_time = Instant::now();
let result = {
let mut process_guard = self.process.lock().unwrap();
lock_time = Instant::now();
// launch the WAL redo process on first use
if process_guard.is_none() {
let redo_process = PostgresRedoProcess::launch(self.conf, &self.tenantid)?;
*process_guard = Some(redo_process);
}
let process = process_guard.as_mut().unwrap();
self.handle_apply_request(process, &request)
};
end_time = Instant::now();
WAL_REDO_WAIT_TIME.observe(lock_time.duration_since(start_time).as_secs_f64());
WAL_REDO_TIME.observe(end_time.duration_since(lock_time).as_secs_f64());
result
}
}
impl PostgresRedoManager {
///
/// Create a new PostgresRedoManager.
///
pub fn new(conf: &'static PageServerConf, tenantid: ZTenantId) -> Self {
// The actual process is launched lazily, on first request.
Self {
tenantid,
conf,
process: Mutex::new(None),
}
}
///
/// Process one request for WAL redo.
///
fn handle_apply_request(
&self,
process: &mut PostgresRedoProcess,
request: &WalRedoRequest,
) -> Result<Bytes, WalRedoError> {
let start = Instant::now();
let apply_result = if let RelishTag::Relation(rel) = request.rel {
// Relational WAL records are applied using wal-redo-postgres
let buf_tag = BufferTag {
rel,
blknum: request.blknum,
};
process.apply_wal_records(buf_tag, &request.base_img, &request.records)
} else {
Ok(nonrel::apply_nonrel(request))
};
let duration = start.elapsed();
let result: Result<Bytes, WalRedoError>;
debug!(
"applied {} WAL records in {} ms to reconstruct page image at LSN {}",
request.records.len(),
duration.as_millis(),
request.lsn
);
if let Err(e) = apply_result {
error!("could not apply WAL records: {:#}", e);
result = Err(WalRedoError::IoError(e));
} else {
let img = apply_result.unwrap();
result = Ok(img);
}
// The caller is responsible for sending the response
result
}
}
///
/// Handle to the Postgres WAL redo process
///
struct PostgresRedoProcess {
stdin: ChildStdin,
stdout: ChildStdout,
}
impl PostgresRedoProcess {
//
// Start postgres binary in special WAL redo mode.
//
fn launch(conf: &PageServerConf, tenantid: &ZTenantId) -> Result<PostgresRedoProcess, Error> {
// FIXME: We need a dummy Postgres cluster to run the process in. Currently, we
// just create one with constant name. That fails if you try to launch more than
// one WAL redo manager concurrently.
let datadir = conf.tenant_path(tenantid).join("wal-redo-datadir");
// Create empty data directory for wal-redo postgres, deleting old one first.
if datadir.exists() {
info!("directory {:?} exists, removing", &datadir);
if let Err(e) = fs::remove_dir_all(&datadir) {
error!("could not remove old wal-redo-datadir: {:#}", e);
}
}
info!("running initdb in {:?}", datadir.display());
let initdb = Command::new(conf.pg_bin_dir().join("initdb"))
.args(&["-D", datadir.to_str().unwrap()])
.arg("-N")
.env_clear()
.env("LD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("DYLD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.output()
.expect("failed to execute initdb");
if !initdb.status.success() {
panic!(
"initdb failed: {}\nstderr:\n{}",
std::str::from_utf8(&initdb.stdout).unwrap(),
std::str::from_utf8(&initdb.stderr).unwrap()
);
} else {
// Limit shared cache for wal-redo-postres
let mut config = OpenOptions::new()
.append(true)
.open(PathBuf::from(&datadir).join("postgresql.conf"))?;
config.write_all(b"shared_buffers=128kB\n")?;
config.write_all(b"fsync=off\n")?;
config.write_all(b"shared_preload_libraries=zenith\n")?;
config.write_all(b"zenith.wal_redo=on\n")?;
}
// Start postgres itself
let mut child = Command::new(conf.pg_bin_dir().join("postgres"))
.arg("--wal-redo")
.stdin(Stdio::piped())
.stderr(Stdio::piped())
.stdout(Stdio::piped())
.env_clear()
.env("LD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("DYLD_LIBRARY_PATH", conf.pg_lib_dir().to_str().unwrap())
.env("PGDATA", &datadir)
.spawn()
.expect("postgres --wal-redo command failed to start");
info!(
"launched WAL redo postgres process on {:?}",
datadir.display()
);
let stdin = child.stdin.take().expect("failed to open child's stdin");
let stderr = child.stderr.take().expect("failed to open child's stderr");
let stdout = child.stdout.take().expect("failed to open child's stdout");
process_utils::set_nonblocking(&stdin)?;
process_utils::set_nonblocking(&stdout)?;
std::thread::Builder::new()
.name("wal-redo-stderr-proxy".to_string())
.spawn(|| {
let mut stderr_buffered = BufReader::new(stderr);
let mut line = String::new();
loop {
line.clear();
match stderr_buffered.read_line(&mut line) {
Ok(0) => break,
Ok(_) => error!("wal-redo-postgres: {}", line.trim()),
Err(e) => debug!("error reading wal-redo stderr: {:#?}", e),
}
}
})
.unwrap(); // TODO propogate error
Ok(PostgresRedoProcess { stdin, stdout })
}
//
// Apply given WAL records ('records') over an old page image. Returns
// new page image.
//
fn apply_wal_records(
&mut self,
tag: BufferTag,
base_img_opt: &Option<Bytes>,
records: &[WALRecord],
) -> std::io::Result<Bytes> {
let mut timeout_writer = process_utils::TimeoutWriter {
timeout: TIMEOUT,
writer: &mut self.stdin,
};
let buf = request::serialize_request(tag, base_img_opt, records);
timeout_writer.write_all(&buf)?;
drop(timeout_writer);
let mut buf = vec![0u8; 8192];
let mut timeout_reader = process_utils::TimeoutReader {
timeout: TIMEOUT,
reader: &mut self.stdout,
};
timeout_reader.read_exact(&mut buf)?;
Ok(Bytes::from(buf))
}
}

View File

@@ -1,161 +0,0 @@
use byteorder::{ByteOrder, LittleEndian};
use bytes::{Buf, Bytes, BytesMut};
use postgres_ffi::{
nonrelfile_utils::{
mx_offset_to_flags_bitshift, mx_offset_to_flags_offset, mx_offset_to_member_offset,
transaction_id_set_status,
},
pg_constants, XLogRecord,
};
use crate::{
relish::{RelishTag, SlruKind},
waldecoder::{XlMultiXactCreate, XlXactParsedRecord},
};
use super::WalRedoRequest;
pub(super) fn apply_nonrel(request: &WalRedoRequest) -> Bytes {
let rel = request.rel;
let blknum = request.blknum;
// Non-relational WAL records are handled here, with custom code that has the
// same effects as the corresponding Postgres WAL redo function.
const ZERO_PAGE: [u8; 8192] = [0u8; 8192];
let mut page = BytesMut::new();
if let Some(fpi) = &request.base_img {
// If full-page image is provided, then use it...
page.extend_from_slice(&fpi[..]);
} else {
// otherwise initialize page with zeros
page.extend_from_slice(&ZERO_PAGE);
}
// Apply all collected WAL records
for record in &request.records {
let mut buf = record.rec.clone();
super::WAL_REDO_RECORD_COUNTER.inc();
// 1. Parse XLogRecord struct
// FIXME: refactor to avoid code duplication.
let xlogrec = XLogRecord::from_bytes(&mut buf);
//move to main data
// TODO probably, we should store some records in our special format
// to avoid this weird parsing on replay
let skip = (record.main_data_offset - pg_constants::SIZEOF_XLOGRECORD) as usize;
if buf.remaining() > skip {
buf.advance(skip);
}
if xlogrec.xl_rmid == pg_constants::RM_XACT_ID {
// Transaction manager stuff
let rec_segno = match rel {
RelishTag::Slru { slru, segno } => {
assert!(
slru == SlruKind::Clog,
"Not valid XACT relish tag {:?}",
rel
);
segno
}
_ => panic!("Not valid XACT relish tag {:?}", rel),
};
let parsed_xact = XlXactParsedRecord::decode(&mut buf, xlogrec.xl_xid, xlogrec.xl_info);
if parsed_xact.info == pg_constants::XLOG_XACT_COMMIT
|| parsed_xact.info == pg_constants::XLOG_XACT_COMMIT_PREPARED
{
transaction_id_set_status(
parsed_xact.xid,
pg_constants::TRANSACTION_STATUS_COMMITTED,
&mut page,
);
for subxact in &parsed_xact.subxacts {
let pageno = *subxact as u32 / pg_constants::CLOG_XACTS_PER_PAGE;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
// only update xids on the requested page
if rec_segno == segno && blknum == rpageno {
transaction_id_set_status(
*subxact,
pg_constants::TRANSACTION_STATUS_COMMITTED,
&mut page,
);
}
}
} else if parsed_xact.info == pg_constants::XLOG_XACT_ABORT
|| parsed_xact.info == pg_constants::XLOG_XACT_ABORT_PREPARED
{
transaction_id_set_status(
parsed_xact.xid,
pg_constants::TRANSACTION_STATUS_ABORTED,
&mut page,
);
for subxact in &parsed_xact.subxacts {
let pageno = *subxact as u32 / pg_constants::CLOG_XACTS_PER_PAGE;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
// only update xids on the requested page
if rec_segno == segno && blknum == rpageno {
transaction_id_set_status(
*subxact,
pg_constants::TRANSACTION_STATUS_ABORTED,
&mut page,
);
}
}
}
} else if xlogrec.xl_rmid == pg_constants::RM_MULTIXACT_ID {
// Multixact operations
let info = xlogrec.xl_info & pg_constants::XLR_RMGR_INFO_MASK;
if info == pg_constants::XLOG_MULTIXACT_CREATE_ID {
let xlrec = XlMultiXactCreate::decode(&mut buf);
if let RelishTag::Slru {
slru,
segno: rec_segno,
} = rel
{
if slru == SlruKind::MultiXactMembers {
for i in 0..xlrec.nmembers {
let pageno = i / pg_constants::MULTIXACT_MEMBERS_PER_PAGE as u32;
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
if segno == rec_segno && rpageno == blknum {
// update only target block
let offset = xlrec.moff + i;
let memberoff = mx_offset_to_member_offset(offset);
let flagsoff = mx_offset_to_flags_offset(offset);
let bshift = mx_offset_to_flags_bitshift(offset);
let mut flagsval =
LittleEndian::read_u32(&page[flagsoff..flagsoff + 4]);
flagsval &= !(((1 << pg_constants::MXACT_MEMBER_BITS_PER_XACT)
- 1)
<< bshift);
flagsval |= xlrec.members[i as usize].status << bshift;
LittleEndian::write_u32(
&mut page[flagsoff..flagsoff + 4],
flagsval,
);
LittleEndian::write_u32(
&mut page[memberoff..memberoff + 4],
xlrec.members[i as usize].xid,
);
}
}
} else {
// Multixact offsets SLRU
let offs = (xlrec.mid % pg_constants::MULTIXACT_OFFSETS_PER_PAGE as u32 * 4)
as usize;
LittleEndian::write_u32(&mut page[offs..offs + 4], xlrec.moff);
}
} else {
panic!();
}
} else {
panic!();
}
}
}
page.freeze()
}

View File

@@ -1,123 +0,0 @@
use std::{
convert::TryInto,
io::{ErrorKind, Read},
os::unix::prelude::AsRawFd,
time::{Duration, Instant},
};
pub fn set_nonblocking(fd: &impl AsRawFd) -> std::io::Result<()> {
use nix::fcntl::{fcntl, FcntlArg, OFlag};
let fd = fd.as_raw_fd();
let flags_bits = fcntl(fd, FcntlArg::F_GETFL).unwrap();
let mut flags = OFlag::from_bits(flags_bits).unwrap();
flags.insert(OFlag::O_NONBLOCK);
fcntl(fd, FcntlArg::F_SETFL(flags)).unwrap();
Ok(())
}
pub struct TimeoutReader<'a, R> {
pub timeout: Duration,
pub reader: &'a mut R,
}
impl<R: Read + AsRawFd> std::io::Read for TimeoutReader<'_, R> {
fn read(&mut self, buf: &mut [u8]) -> std::io::Result<usize> {
let mut start_time_opt: Option<Instant> = None;
loop {
match self.reader.read(buf) {
ok @ Ok(_) => return ok,
Err(ref e) if e.kind() == ErrorKind::WouldBlock => {}
err @ Err(_) => return err,
}
let timeout = if let Some(start_time) = start_time_opt {
let elapsed = start_time.elapsed();
match self.timeout.checked_sub(elapsed) {
Some(timeout) => timeout,
None => {
return Err(std::io::Error::new(
std::io::ErrorKind::TimedOut,
"read timed out",
));
}
}
} else {
start_time_opt = Some(Instant::now());
self.timeout
};
use nix::{
errno::Errno,
poll::{poll, PollFd, PollFlags},
};
let mut poll_fd = PollFd::new(self.reader.as_raw_fd(), PollFlags::POLLIN);
let millis: i32 = timeout.as_millis().try_into().unwrap_or(i32::MAX);
match poll(std::slice::from_mut(&mut poll_fd), millis) {
Ok(0) => {}
Ok(n) => {
debug_assert!(n == 1);
}
Err(Errno::EINTR) => {}
Err(e) => return Err(std::io::Error::from_raw_os_error(e as i32)),
}
}
}
}
pub struct TimeoutWriter<'a, W: std::io::Write + AsRawFd> {
pub timeout: Duration,
pub writer: &'a mut W,
}
impl<W: std::io::Write + AsRawFd> std::io::Write for TimeoutWriter<'_, W> {
fn write(&mut self, buf: &[u8]) -> std::io::Result<usize> {
let mut start_time_opt: Option<Instant> = None;
loop {
match self.writer.write(buf) {
ok @ Ok(_) => return ok,
Err(ref e) if e.kind() == ErrorKind::WouldBlock => {}
err @ Err(_) => return err,
}
let timeout = if let Some(start_time) = start_time_opt {
let elapsed = start_time.elapsed();
match self.timeout.checked_sub(elapsed) {
Some(timeout) => timeout,
None => {
return Err(std::io::Error::new(
std::io::ErrorKind::TimedOut,
"write timed out",
));
}
}
} else {
start_time_opt = Some(Instant::now());
self.timeout
};
use nix::{
errno::Errno,
poll::{poll, PollFd, PollFlags},
};
let mut poll_fd = PollFd::new(self.writer.as_raw_fd(), PollFlags::POLLOUT);
let millis: i32 = timeout.as_millis().try_into().unwrap_or(i32::MAX);
match poll(std::slice::from_mut(&mut poll_fd), millis) {
Ok(0) => {} // TODO want to check timeout before calling read again
Ok(n) => {
debug_assert!(n == 1);
}
Err(Errno::EINTR) => {}
Err(e) => return Err(std::io::Error::from_raw_os_error(e as i32)),
}
}
}
fn flush(&mut self) -> std::io::Result<()> {
Ok(())
}
}

View File

@@ -1,86 +0,0 @@
///! Functions for constructing messages to send to the postgres WAL redo
///! process. See vendor/postgres/src/backend/tcop/zenith_wal_redo.c for
///! explanation of the protocol.
use bytes::{BufMut, Bytes, BytesMut};
use zenith_utils::{bin_ser::BeSer, lsn::Lsn};
use crate::repository::WALRecord;
use super::BufferTag;
pub fn serialize_request(
tag: BufferTag,
base_img: &Option<Bytes>,
records: &[WALRecord],
) -> BytesMut {
let mut capacity = 1 + BEGIN_REDO_MSG_LEN;
if base_img.is_some() {
capacity += 1 + PUSH_PAGE_MSG_LEN;
}
capacity += (1 + APPLY_MSG_HEADER_LEN) * records.len();
capacity += records.iter().map(|rec| rec.rec.len()).sum::<usize>();
capacity += 1 + GET_PAGE_MSG_LEN;
let mut buf = BytesMut::with_capacity(capacity);
build_begin_redo_for_block_msg(&mut buf, tag);
if let Some(base_img) = base_img.as_ref() {
build_push_page_msg(&mut buf, tag, base_img);
}
for record in records {
build_apply_record_msg(&mut buf, record.lsn, &record.rec);
}
build_get_page_msg(&mut buf, tag);
debug_assert_eq!(capacity, buf.len());
buf
}
const TAG_LEN: usize = 4 * 4;
const PAGE_SIZE: usize = 8192;
const BEGIN_REDO_MSG_LEN: usize = 4 + 1 + TAG_LEN;
const PUSH_PAGE_MSG_LEN: usize = 4 + 1 + TAG_LEN + PAGE_SIZE;
const APPLY_MSG_HEADER_LEN: usize = 4 + 8;
const GET_PAGE_MSG_LEN: usize = 4 + 1 + TAG_LEN;
fn build_begin_redo_for_block_msg(buf: &mut BytesMut, tag: BufferTag) {
buf.put_u8(b'B');
buf.put_u32(BEGIN_REDO_MSG_LEN as u32);
// TODO tag is serialized multiple times
// let's try to serialize it just once
// or make the protocol less repetitive
tag.ser_into(&mut buf.writer())
.expect("serialize BufferTag should always succeed");
}
fn build_push_page_msg(buf: &mut BytesMut, tag: BufferTag, base_img: &Bytes) {
debug_assert_eq!(base_img.len(), PAGE_SIZE);
buf.put_u8(b'P');
buf.put_u32(PUSH_PAGE_MSG_LEN as u32);
tag.ser_into(&mut buf.writer())
.expect("serialize BufferTag should always succeed");
buf.extend(base_img);
}
fn build_apply_record_msg(buf: &mut BytesMut, endlsn: Lsn, rec: &Bytes) {
buf.put_u8(b'A');
let len = APPLY_MSG_HEADER_LEN + rec.len();
buf.put_u32(len as u32);
buf.put_u64(endlsn.0);
buf.extend(rec);
}
fn build_get_page_msg(buf: &mut BytesMut, tag: BufferTag) {
buf.put_u8(b'G');
buf.put_u32(GET_PAGE_MSG_LEN as u32);
tag.ser_into(&mut buf.writer())
.expect("serialize BufferTag should always succeed");
}

View File

@@ -18,6 +18,5 @@ tokio = "1.11"
tokio-postgres = { git = "https://github.com/zenithdb/rust-postgres.git", rev="9eb0dbfbeb6a6c1b79099b9f7ae4a8c021877858" }
clap = "2.33.0"
rustls = "0.19.1"
reqwest = { version = "0.11", features = ["blocking", "json"] }
zenith_utils = { path = "../zenith_utils" }

View File

@@ -1,14 +1,17 @@
use anyhow::{bail, Context, Result};
use anyhow::{bail, Result};
use serde::{Deserialize, Serialize};
use std::net::{SocketAddr, ToSocketAddrs};
use std::{
collections::HashMap,
net::{IpAddr, SocketAddr},
};
pub struct CPlaneApi {
auth_endpoint: &'static str,
// address: SocketAddr,
}
#[derive(Serialize, Deserialize, Debug)]
#[derive(Serialize, Deserialize)]
pub struct DatabaseInfo {
pub host: String,
pub host: IpAddr, // TODO: allow host name here too
pub port: u16,
pub dbname: String,
pub user: String,
@@ -16,13 +19,8 @@ pub struct DatabaseInfo {
}
impl DatabaseInfo {
pub fn socket_addr(&self) -> Result<SocketAddr> {
let host_port = format!("{}:{}", self.host, self.port);
host_port
.to_socket_addrs()
.with_context(|| format!("cannot resolve {} to SocketAddr", host_port))?
.next()
.ok_or_else(|| anyhow::Error::msg("cannot resolve at least one SocketAddr"))
pub fn socket_addr(&self) -> SocketAddr {
SocketAddr::new(self.host, self.port)
}
pub fn conn_string(&self) -> String {
@@ -33,35 +31,62 @@ impl DatabaseInfo {
}
}
// mock cplane api
impl CPlaneApi {
pub fn new(auth_endpoint: &'static str) -> CPlaneApi {
CPlaneApi { auth_endpoint }
pub fn new(_address: &SocketAddr) -> CPlaneApi {
CPlaneApi {
// address: address.clone(),
}
}
pub fn authenticate_proxy_request(
&self,
user: &str,
database: &str,
md5_response: &[u8],
salt: &[u8; 4],
) -> Result<DatabaseInfo> {
let mut url = reqwest::Url::parse(self.auth_endpoint)?;
url.query_pairs_mut()
.append_pair("login", user)
.append_pair("database", database)
.append_pair("md5response", std::str::from_utf8(md5_response)?)
.append_pair("salt", &hex::encode(salt));
pub fn check_auth(&self, user: &str, md5_response: &[u8], salt: &[u8; 4]) -> Result<()> {
// passwords for both is "mypass"
let auth_map: HashMap<_, &str> = vec![
("stas@zenith", "716ee6e1c4a9364d66285452c47402b1"),
("stas2@zenith", "3996f75df64c16a8bfaf01301b61d582"),
]
.into_iter()
.collect();
println!("cplane request: {}", url.as_str());
let stored_hash = auth_map
.get(&user)
.ok_or_else(|| anyhow::Error::msg("user not found"))?;
let salted_stored_hash = format!(
"md5{:x}",
md5::compute([stored_hash.as_bytes(), salt].concat())
);
let resp = reqwest::blocking::get(url)?;
let received_hash = std::str::from_utf8(md5_response)?;
if resp.status().is_success() {
let conn_info: DatabaseInfo = serde_json::from_str(resp.text()?.as_str())?;
println!("got conn info: #{:?}", conn_info);
Ok(conn_info)
println!(
"auth: {} rh={} sh={} ssh={} {:?}",
user, received_hash, stored_hash, salted_stored_hash, salt
);
if received_hash == salted_stored_hash {
Ok(())
} else {
bail!("Auth failed")
}
}
pub fn get_database_uri(&self, _user: &str, _database: &str) -> Result<DatabaseInfo> {
Ok(DatabaseInfo {
host: "127.0.0.1".parse()?,
port: 5432,
dbname: "stas".to_string(),
user: "stas".to_string(),
password: "mypass".to_string(),
})
}
// pub fn create_database(&self, _user: &String, _database: &String) -> Result<DatabaseInfo> {
// Ok(DatabaseInfo {
// host: "127.0.0.1".parse()?,
// port: 5432,
// dbname: "stas".to_string(),
// user: "stas".to_string(),
// password: "mypass".to_string(),
// })
// }
}

View File

@@ -34,7 +34,7 @@ pub struct ProxyConf {
pub redirect_uri: String,
/// control plane address where we would check auth.
pub auth_endpoint: String,
pub cplane_address: SocketAddr,
pub ssl_config: Option<Arc<ServerConfig>>,
}
@@ -56,7 +56,8 @@ fn configure_ssl(arg_matches: &ArgMatches) -> anyhow::Result<Option<Arc<ServerCo
let key = {
let key_bytes = std::fs::read(key_path).context("SSL key file")?;
let mut keys = pemfile::pkcs8_private_keys(&mut &key_bytes[..])
let mut keys = pemfile::rsa_private_keys(&mut &key_bytes[..])
.or_else(|_| pemfile::pkcs8_private_keys(&mut &key_bytes[..]))
.map_err(|_| anyhow!("couldn't read TLS keys"))?;
ensure!(keys.len() == 1, "keys.len() = {} (should be 1)", keys.len());
keys.pop().unwrap()
@@ -101,14 +102,6 @@ fn main() -> anyhow::Result<()> {
.help("redirect unauthenticated users to given uri")
.default_value("http://localhost:3000/psql_session/"),
)
.arg(
Arg::with_name("auth-endpoint")
.short("a")
.long("auth-endpoint")
.takes_value(true)
.help("redirect unauthenticated users to given uri")
.default_value("http://localhost:3000/authenticate_proxy_request/"),
)
.arg(
Arg::with_name("ssl-key")
.short("k")
@@ -129,7 +122,7 @@ fn main() -> anyhow::Result<()> {
proxy_address: arg_matches.value_of("proxy").unwrap().parse()?,
mgmt_address: arg_matches.value_of("mgmt").unwrap().parse()?,
redirect_uri: arg_matches.value_of("uri").unwrap().parse()?,
auth_endpoint: arg_matches.value_of("auth-endpoint").unwrap().parse()?,
cplane_address: "127.0.0.1:3000".parse()?,
ssl_config: configure_ssl(&arg_matches)?,
};
let state = ProxyState {

View File

@@ -5,7 +5,7 @@ use std::{
use anyhow::bail;
use bytes::Bytes;
use serde::Deserialize;
use serde::{Deserialize, Serialize};
use zenith_utils::{
postgres_backend::{self, query_from_cstring, AuthType, PostgresBackend},
pq_proto::{BeMessage, SINGLE_COL_ROWDESC},
@@ -49,7 +49,7 @@ struct MgmtHandler {
// "host": "127.0.0.1",
// "port": 5432,
// "dbname": "stas",
// "user": "stas",
// "user": "stas"
// "password": "mypass"
// }
// }
@@ -60,16 +60,13 @@ struct MgmtHandler {
// "Failure": "oops"
// }
// }
//
// // to test manually by sending a query to mgmt interface:
// psql -h 127.0.0.1 -p 9999 -c '{"session_id":"4f10dde522e14739","result":{"Success":{"host":"127.0.0.1","port":5432,"dbname":"stas","user":"stas","password":"stas"}}}'
#[derive(Deserialize)]
#[derive(Serialize, Deserialize)]
pub struct PsqlSessionResponse {
session_id: String,
result: PsqlSessionResult,
}
#[derive(Deserialize)]
#[derive(Serialize, Deserialize)]
pub enum PsqlSessionResult {
Success(DatabaseInfo),
Failure(String),
@@ -81,47 +78,34 @@ impl postgres_backend::Handler for MgmtHandler {
pgb: &mut PostgresBackend,
query_string: Bytes,
) -> anyhow::Result<()> {
let res = try_process_query(self, pgb, query_string);
// intercept and log error message
if res.is_err() {
println!("Mgmt query failed: #{:?}", res);
}
res
}
}
let query_string = query_from_cstring(query_string);
fn try_process_query(
mgmt: &mut MgmtHandler,
pgb: &mut PostgresBackend,
query_string: Bytes,
) -> anyhow::Result<()> {
let query_string = query_from_cstring(query_string);
println!("Got mgmt query: '{}'", std::str::from_utf8(&query_string)?);
println!("Got mgmt query: '{}'", std::str::from_utf8(&query_string)?);
let resp: PsqlSessionResponse = serde_json::from_slice(&query_string)?;
let resp: PsqlSessionResponse = serde_json::from_slice(&query_string)?;
let waiters = self.state.waiters.lock().unwrap();
let waiters = mgmt.state.waiters.lock().unwrap();
let sender = waiters
.get(&resp.session_id)
.ok_or_else(|| anyhow::Error::msg("psql_session_id is not found"))?;
let sender = waiters
.get(&resp.session_id)
.ok_or_else(|| anyhow::Error::msg("psql_session_id is not found"))?;
match resp.result {
PsqlSessionResult::Success(db_info) => {
sender.send(Ok(db_info))?;
match resp.result {
PsqlSessionResult::Success(db_info) => {
sender.send(Ok(db_info))?;
pgb.write_message_noflush(&SINGLE_COL_ROWDESC)?
.write_message_noflush(&BeMessage::DataRow(&[Some(b"ok")]))?
.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
pgb.flush()?;
Ok(())
}
pgb.write_message_noflush(&SINGLE_COL_ROWDESC)?
.write_message_noflush(&BeMessage::DataRow(&[Some(b"ok")]))?
.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
pgb.flush()?;
Ok(())
}
PsqlSessionResult::Failure(message) => {
sender.send(Err(anyhow::Error::msg(message.clone())))?;
PsqlSessionResult::Failure(message) => {
sender.send(Err(anyhow::Error::msg(message.clone())))?;
bail!("psql session request failed: {}", message)
bail!("psql session request failed: {}", message)
}
}
}
}

View File

@@ -57,7 +57,7 @@ pub fn proxy_conn_main(
) -> anyhow::Result<()> {
let mut conn = ProxyConnection {
state,
cplane: CPlaneApi::new(&state.conf.auth_endpoint),
cplane: CPlaneApi::new(&state.conf.cplane_address),
user: "".into(),
database: "".into(),
pgb: PostgresBackend::new(
@@ -80,8 +80,6 @@ pub fn proxy_conn_main(
conn.handle_new_user()?
};
// XXX: move that inside handle_new_user/handle_existing_user to be able to
// report wrong connection error.
proxy_pass(conn.pgb, db_info)
}
@@ -174,31 +172,21 @@ impl ProxyConnection {
.split_last()
.ok_or_else(|| anyhow::Error::msg("unexpected password message"))?;
match self.cplane.authenticate_proxy_request(
self.user.as_str(),
self.database.as_str(),
md5_response,
&self.md5_salt,
) {
Err(e) => {
self.pgb
.write_message(&BeMessage::ErrorResponse(format!("{}", e)))?;
bail!("auth failed: {}", e);
}
Ok(conn_info) => {
self.pgb
.write_message_noflush(&BeMessage::AuthenticationOk)?;
self.pgb
.write_message_noflush(&BeMessage::ParameterStatus)?;
self.pgb.write_message(&BeMessage::ReadyForQuery)?;
Ok(conn_info)
}
if let Err(e) = self.check_auth_md5(md5_response) {
self.pgb
.write_message(&BeMessage::ErrorResponse(format!("{}", e)))?;
bail!("auth failed: {}", e);
} else {
self.pgb
.write_message_noflush(&BeMessage::AuthenticationOk)?;
self.pgb
.write_message_noflush(&BeMessage::ParameterStatus)?;
self.pgb.write_message(&BeMessage::ReadyForQuery)?;
}
} else {
bail!("protocol violation");
}
// ok, we are authorized
self.cplane.get_database_uri(&self.user, &self.database)
}
fn handle_new_user(&mut self) -> anyhow::Result<DatabaseInfo> {
@@ -244,11 +232,17 @@ databases without opening the browser.
Ok(dbinfo)
}
fn check_auth_md5(&self, md5_response: &[u8]) -> anyhow::Result<()> {
assert!(self.is_existing_user());
self.cplane
.check_auth(self.user.as_str(), md5_response, &self.md5_salt)
}
}
/// Create a TCP connection to a postgres database, authenticate with it, and receive the ReadyForQuery message
async fn connect_to_db(db_info: DatabaseInfo) -> anyhow::Result<tokio::net::TcpStream> {
let mut socket = tokio::net::TcpStream::connect(db_info.socket_addr()?).await?;
let mut socket = tokio::net::TcpStream::connect(db_info.socket_addr()).await?;
let config = db_info.conn_string().parse::<tokio_postgres::Config>()?;
let _ = config.connect_raw(&mut socket, NoTls).await?;
Ok(socket)
@@ -279,9 +273,7 @@ fn proxy(
/// Proxy a client connection to a postgres database
fn proxy_pass(pgb: PostgresBackend, db_info: DatabaseInfo) -> anyhow::Result<()> {
let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()?;
let runtime = tokio::runtime::Builder::new_current_thread().build()?;
let db_stream = runtime.block_on(connect_to_db(db_info))?;
let db_stream = db_stream.into_std()?;
db_stream.set_nonblocking(false)?;

View File

@@ -46,7 +46,7 @@ def test_old_request_lsn(zenith_cli, pageserver: ZenithPageserver, postgres: Pos
from pg_settings where name = 'shared_buffers'
''')
row = cur.fetchone()
print(f'shared_buffers is {row[0]}, table size {row[1]}');
print("shared_buffers is {}, table size {}", row[0], row[1]);
assert int(row[0]) < int(row[1])
cur.execute('VACUUM foo');

View File

@@ -12,5 +12,5 @@ def test_pgbench(postgres: PostgresFactory, pg_bin, zenith_cli):
connstr = pg.connstr()
pg_bin.run_capture(['pgbench', '-i', connstr])
pg_bin.run_capture(['pgbench'] + '-c 10 -T 5 -P 1 -M prepared'.split() + [connstr])
pg_bin.run_capture(['pgbench', '-i', '-s', '100', connstr])
pg_bin.run_capture(['pgbench'] + '-c 1 -N -T 100 -P 1 -M prepared'.split() + [connstr])

View File

@@ -9,7 +9,10 @@ pytest_plugins = ("fixtures.zenith_fixtures")
#
# Test restarting and recreating a postgres instance
#
@pytest.mark.parametrize('with_wal_acceptors', [False, True])
# XXX: with_wal_acceptors=True fails now, would be fixed with
# `postgres --sync-walkeepers` patches.
#
@pytest.mark.parametrize('with_wal_acceptors', [False])
def test_restart_compute(
zenith_cli,
pageserver: ZenithPageserver,

View File

@@ -6,8 +6,8 @@ pytest_plugins = ("fixtures.zenith_fixtures")
def print_gc_result(row):
print("GC duration {elapsed} ms".format_map(row));
print(" 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))
print(" 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}".format_map(row))
print(" 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}, removed: {layer_relfiles_removed}, dropped: {layer_relfiles_dropped}".format_map(row))
print(" 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}, removed: {layer_nonrelfiles_removed}, dropped: {layer_nonrelfiles_dropped}".format_map(row))
#
@@ -58,21 +58,19 @@ def test_layerfiles_gc(zenith_cli, pageserver, postgres, pg_bin):
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.
# Insert a row.
print("Inserting one row and running GC")
cur.execute("INSERT INTO foo VALUES (1)")
pscur.execute(f"do_gc {pageserver.initial_tenant} {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_total'] == layer_relfiles_remain + 1
assert row['layer_relfiles_removed'] == 1
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.
# This should create a new layer file with the new contents, and
# remove the old one.
print("Inserting two more rows and running GC")
cur.execute("INSERT INTO foo VALUES (2)")
cur.execute("INSERT INTO foo VALUES (3)")
@@ -80,11 +78,11 @@ def test_layerfiles_gc(zenith_cli, pageserver, postgres, pg_bin):
pscur.execute(f"do_gc {pageserver.initial_tenant} {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_total'] == layer_relfiles_remain + 1
assert row['layer_relfiles_removed'] == 1
assert row['layer_relfiles_dropped'] == 0
# Do it again. Should again create two new layer files and remove old ones.
# Do it again. Should again create a new layer file and remove old one.
print("Inserting two more rows and running GC")
cur.execute("INSERT INTO foo VALUES (2)")
cur.execute("INSERT INTO foo VALUES (3)")
@@ -92,8 +90,8 @@ def test_layerfiles_gc(zenith_cli, pageserver, postgres, pg_bin):
pscur.execute(f"do_gc {pageserver.initial_tenant} {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_total'] == layer_relfiles_remain + 1
assert row['layer_relfiles_removed'] == 1
assert row['layer_relfiles_dropped'] == 0
# Run GC again, with no changes in the database. Should not remove anything.
@@ -115,19 +113,12 @@ def test_layerfiles_gc(zenith_cli, pageserver, postgres, pg_bin):
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
assert row['layer_relfiles_dropped'] == 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

View File

@@ -1,14 +1,10 @@
import pytest
import random
import time
import os
import subprocess
import uuid
from contextlib import closing
from multiprocessing import Process, Value
from fixtures.zenith_fixtures import WalAcceptorFactory, ZenithPageserver, PostgresFactory, PgBin
from fixtures.utils import lsn_to_hex, mkdir_if_needed
from fixtures.zenith_fixtures import WalAcceptorFactory, ZenithPageserver, PostgresFactory
pytest_plugins = ("fixtures.zenith_fixtures")
@@ -202,92 +198,3 @@ def test_race_conditions(zenith_cli, pageserver: ZenithPageserver, postgres: Pos
stop_value.value = 1
proc.join()
class ProposerPostgres:
"""Object for running safekeepers sync with walproposer"""
def __init__(self, pgdata_dir: str, pg_bin: PgBin, timeline_id: str, tenant_id: str):
self.pgdata_dir: str = pgdata_dir
self.pg_bin: PgBin = pg_bin
self.timeline_id: str = timeline_id
self.tenant_id: str = tenant_id
def pg_data_dir_path(self) -> str:
""" Path to data directory """
return self.pgdata_dir
def config_file_path(self) -> str:
""" Path to postgresql.conf """
return os.path.join(self.pgdata_dir, 'postgresql.conf')
def create_dir_config(self, wal_acceptors: str):
""" Create dir and config for running --sync-safekeepers """
mkdir_if_needed(self.pg_data_dir_path())
with open(self.config_file_path(), "w") as f:
f.writelines([
"synchronous_standby_names = 'walproposer'\n",
f"zenith.zenith_timeline = '{self.timeline_id}'\n",
f"zenith.zenith_tenant = '{self.tenant_id}'\n",
f"wal_acceptors = '{wal_acceptors}'\n",
])
def sync_safekeepers(self) -> str:
"""
Run 'postgres --sync-safekeepers'.
Returns execution result, which is commit_lsn after sync.
"""
command = ["postgres", "--sync-safekeepers"]
env = {
"PGDATA": self.pg_data_dir_path(),
}
basepath = self.pg_bin.run_capture(command, env)
stdout_filename = basepath + '.stdout'
with open(stdout_filename, 'r') as stdout_f:
stdout = stdout_f.read()
return stdout.strip("\n ")
# insert wal in all safekeepers and run sync on proposer
def test_sync_safekeepers(repo_dir: str, pg_bin: PgBin, wa_factory: WalAcceptorFactory):
wa_factory.start_n_new(3)
timeline_id = uuid.uuid4().hex
tenant_id = uuid.uuid4().hex
# write config for proposer
pgdata_dir = os.path.join(repo_dir, "proposer_pgdata")
pg = ProposerPostgres(pgdata_dir, pg_bin, timeline_id, tenant_id)
pg.create_dir_config(wa_factory.get_connstrs())
# valid lsn, which is not in the segment start, nor in zero segment
epoch_start_lsn = 0x16B9188 # 0/16B9188
begin_lsn = epoch_start_lsn
# append and commit WAL
lsn_after_append = []
for i in range(3):
res = wa_factory.instances[i].append_logical_message(
tenant_id,
timeline_id,
{
"lm_prefix": "prefix",
"lm_message": "message",
"set_commit_lsn": True,
"term": 2,
"begin_lsn": begin_lsn,
"epoch_start_lsn": epoch_start_lsn,
"truncate_lsn": epoch_start_lsn,
},
)
lsn_hex = lsn_to_hex(res["inserted_wal"]["end_lsn"])
lsn_after_append.append(lsn_hex)
print(f"safekeeper[{i}] lsn after append: {lsn_hex}")
# run sync safekeepers
lsn_after_sync = pg.sync_safekeepers()
print(f"lsn after sync = {lsn_after_sync}")
assert all(lsn_after_sync == lsn for lsn in lsn_after_append)

View File

@@ -118,14 +118,9 @@ async def run_restarts_under_load(pg: Postgres, acceptors: List[WalAcceptor], n_
victim = acceptors[it % len(acceptors)]
victim.stop()
# Wait till previous victim recovers so it is ready for the next
# iteration by making any writing xact.
conn = await pg.connect_async()
await conn.execute(
'UPDATE bank_accs SET amount = amount WHERE uid = 1',
timeout=120
)
await conn.close()
# wait for transactions that could have started and finished before
# victim acceptor was stopped
await asyncio.sleep(1)
stats.reset()
await asyncio.sleep(period_time)

View File

@@ -136,8 +136,7 @@ class ZenithBenchmarker:
# The metric should be an integer, as it's a number of bytes. But in general
# all prometheus metrics are floats. So to be pedantic, read it as a float
# and round to integer.
matches = re.search(r'^pageserver_disk_io_bytes{io_operation="write"} (\S+)$', all_metrics,
re.MULTILINE)
matches = re.search(r'pageserver_disk_io_bytes{io_operation="write"} (\S+)', all_metrics)
return int(round(float(matches.group(1))))
@contextmanager

View File

@@ -21,7 +21,7 @@ def mkdir_if_needed(path: str) -> None:
assert os.path.isdir(path)
def subprocess_capture(capture_dir: str, cmd: List[str], **kwargs: Any) -> str:
def subprocess_capture(capture_dir: str, cmd: List[str], **kwargs: Any) -> None:
""" Run a process and capture its output
Output will go to files named "cmd_NNN.stdout" and "cmd_NNN.stderr"
@@ -29,7 +29,6 @@ def subprocess_capture(capture_dir: str, cmd: List[str], **kwargs: Any) -> str:
counter.
If those files already exist, we will overwrite them.
Returns basepath for files with captured output.
"""
assert type(cmd) is list
base = os.path.basename(cmd[0]) + '_{}'.format(global_counter())
@@ -42,8 +41,6 @@ def subprocess_capture(capture_dir: str, cmd: List[str], **kwargs: Any) -> str:
print('(capturing output to "{}.stdout")'.format(base))
subprocess.run(cmd, **kwargs, stdout=stdout_f, stderr=stderr_f)
return basepath
_global_counter = 0
@@ -65,7 +62,3 @@ def debug_print(*args, **kwargs) -> None:
"""
if os.environ.get('TEST_DEBUG_PRINT') is not None:
print(*args, **kwargs)
def lsn_to_hex(num: int) -> str:
""" Convert lsn from int to standard hex notation. """
return "{:X}/{:X}".format(num >> 32, num & 0xffffffff)

View File

@@ -5,7 +5,6 @@ import os
import pathlib
import uuid
import jwt
import json
import psycopg2
import pytest
import shutil
@@ -71,9 +70,7 @@ def pytest_configure(config):
# This is bad; we don't want any of those processes polluting the
# result of the test.
# NOTE this shows as an internal pytest error, there might be a better way
raise Exception(
'Found interfering processes running. Stop all Zenith pageservers, nodes, WALs, as well as stand-alone Postgres.'
)
raise Exception('found interfering processes running')
def determine_scope(fixture_name: str, config: Any) -> str:
@@ -488,19 +485,17 @@ class PgBin:
def run_capture(self,
command: List[str],
env: Optional[Env] = None,
cwd: Optional[str] = None,
**kwargs: Any) -> None:
cwd: Optional[str] = None) -> None:
"""
Run one of the postgres binaries, with stderr and stdout redirected to a file.
This is just like `run`, but for chatty programs. Returns basepath for files
with captured output.
This is just like `run`, but for chatty programs.
"""
self._fixpath(command)
print('Running command "{}"'.format(' '.join(command)))
env = self._build_env(env)
return subprocess_capture(self.log_dir, command, env=env, cwd=cwd, check=True, **kwargs)
subprocess_capture(self.log_dir, command, env=env, cwd=cwd, check=True)
@zenfixture
@@ -859,27 +854,6 @@ class WalAcceptor:
pass # pidfile might be obsolete
return self
def append_logical_message(self, tenant_id: str, timeline_id: str, request: Dict[str, Any]) -> Dict[str, Any]:
"""
Send JSON_CTRL query to append LogicalMessage to WAL and modify
safekeeper state. It will construct LogicalMessage from provided
prefix and message, and then will write it to WAL.
"""
# "replication=0" hacks psycopg not to send additional queries
# on startup, see https://github.com/psycopg/psycopg2/pull/482
connstr = f"host=localhost port={self.port} replication=0 options='-c ztimelineid={timeline_id} ztenantid={tenant_id}'"
with closing(psycopg2.connect(connstr)) as conn:
# server doesn't support transactions
conn.autocommit = True
with conn.cursor() as cur:
request_json = json.dumps(request)
print(f"JSON_CTRL request on port {self.port}: {request_json}")
cur.execute("JSON_CTRL " + request_json)
all = cur.fetchall()
print(f"JSON_CTRL response: {all[0][0]}")
return json.loads(all[0][0])
class WalAcceptorFactory:
""" An object representing multiple running wal acceptors. """

View File

@@ -1,58 +0,0 @@
import timeit
import pytest
from fixtures.zenith_fixtures import (
TenantFactory,
ZenithCli,
PostgresFactory,
)
pytest_plugins = ("fixtures.benchmark_fixture")
# Run bulk tenant creation test.
#
# Collects metrics:
#
# 1. Time to create {1,10,50} tenants
# 2. Average creation time per tenant
@pytest.mark.parametrize('tenants_count', [1, 5, 10])
@pytest.mark.parametrize('use_wal_acceptors', ['with_wa', 'without_wa'])
def test_bulk_tenant_create(
zenith_cli: ZenithCli,
tenant_factory: TenantFactory,
postgres: PostgresFactory,
wa_factory,
use_wal_acceptors: str,
tenants_count: int,
zenbenchmark,
):
"""Measure tenant creation time (with and without wal acceptors)"""
time_slices = []
for i in range(tenants_count):
start = timeit.default_timer()
tenant = tenant_factory.create()
zenith_cli.run([
"branch", f"test_bulk_tenant_create_{tenants_count}_{i}_{use_wal_acceptors}", "main",
f"--tenantid={tenant}"
])
if use_wal_acceptors == 'with_wa':
wa_factory.start_n_new(3)
pg_tenant = postgres.create_start(
f"test_bulk_tenant_create_{tenants_count}_{i}_{use_wal_acceptors}",
tenant,
wal_acceptors=wa_factory.get_connstrs() if use_wal_acceptors == 'with_wa' else None,
)
end = timeit.default_timer()
time_slices.append(end - start)
pg_tenant.stop()
zenbenchmark.record('tenant_creation_time', sum(time_slices) / len(time_slices), 's')

View File

@@ -13,7 +13,6 @@ bytes = "1.0.1"
byteorder = "1.4.3"
fs2 = "0.4.3"
lazy_static = "1.4.0"
serde_json = "1"
log = "0.4.14"
clap = "2.33.0"
daemonize = "0.4.1"

View File

@@ -76,43 +76,6 @@ safekeepers.
See README_PROTO.md for a more detailed desription of the consensus
protocol. spec/ contains TLA+ specification of it.
# Q&A
Q: Why have a separate service instead of connecting Page Server directly to a
primary PostgreSQL node?
A: Page Server is a single server which can be lost. As our primary
fault-tolerant storage is S3, we do not want to wait for it before
committing a transaction. The WAL service acts as a temporary fault-tolerant
storage for recent data before it gets to the Page Server and then finally
to S3. Whenever WALs and pages are committed to S3, WAL's storage can be
trimmed.
Q: What if the compute node evicts a page, needs it back, but the page is yet
to reach the Page Server?
A: If the compute node has evicted a page, all changes from that page are
already committed, i.e. they are saved on majority of WAL safekeepers. These
WAL records will eventually reach the Page Server. The Page Server notes
that the compute note requests pages with a very recent LSN and will not
respond to the compute node until it a corresponding WAL is received from WAL
safekeepers.
Q: How long may Page Server wait for?
A: Not too long, hopefully. If a page is evicted, it probably was not used for
a while, so the WAL service have had enough time to push changes to the Page
Server. There may be issues if there is no backpressure and compute node with
WAL service run ahead of Page Server, though.
There is no backpressure right now, so you may even see some spurious
timeouts in tests.
Q: How do WAL safekeepers communicate with each other?
A: They may only send each other messages via the compute node, they never
communicate directly with each other.
Q: Why have a consensus algorithm if there is only a single compute node?
A: Actually there may be moments with multiple PostgreSQL nodes running at the
same time. E.g. we are bringing one up and one down. We would like to avoid
simultaneous writes from different nodes, so there should be a consensus on
who is the primary node.
# Terminology

View File

@@ -1,249 +0,0 @@
//!
//! This module implements JSON_CTRL protocol, which allows exchange
//! JSON messages over psql for testing purposes.
//!
//! Currently supports AppendLogicalMessage, which is used for WAL
//! modifications in tests.
//!
use anyhow::{anyhow, Result};
use bytes::{BufMut, Bytes, BytesMut};
use crc32c::crc32c_append;
use log::*;
use serde::{Deserialize, Serialize};
use crate::safekeeper::{AcceptorProposerMessage, AppendResponse};
use crate::safekeeper::{
AppendRequest, AppendRequestHeader, ProposerAcceptorMessage, ProposerGreeting,
};
use crate::safekeeper::{SafeKeeperState, Term};
use crate::send_wal::SendWalHandler;
use crate::timeline::TimelineTools;
use postgres_ffi::pg_constants;
use postgres_ffi::xlog_utils;
use postgres_ffi::{uint32, uint64, Oid, XLogRecord};
use zenith_utils::lsn::Lsn;
use zenith_utils::postgres_backend::PostgresBackend;
use zenith_utils::pq_proto::{BeMessage, RowDescriptor, TEXT_OID};
#[derive(Serialize, Deserialize, Debug)]
struct AppendLogicalMessage {
// prefix and message to build LogicalMessage
lm_prefix: String,
lm_message: String,
// if true, commit_lsn will match flush_lsn after append
set_commit_lsn: bool,
// fields from AppendRequestHeader
term: Term,
epoch_start_lsn: Lsn,
begin_lsn: Lsn,
truncate_lsn: Lsn,
}
#[derive(Serialize, Deserialize)]
struct AppendResult {
// safekeeper state after append
state: SafeKeeperState,
// info about new record in the WAL
inserted_wal: InsertedWAL,
}
/// Handles command to craft logical message WAL record with given
/// content, and then append it with specified term and lsn. This
/// function is used to test safekeepers in different scenarios.
pub fn handle_json_ctrl(
swh: &mut SendWalHandler,
pgb: &mut PostgresBackend,
cmd: &Bytes,
) -> Result<()> {
let cmd = cmd
.strip_prefix(b"JSON_CTRL")
.ok_or_else(|| anyhow!("invalid prefix"))?;
// trim zeroes in the end
let cmd = cmd.strip_suffix(&[0u8]).unwrap_or(cmd);
let append_request: AppendLogicalMessage = serde_json::from_slice(cmd)?;
info!("JSON_CTRL request: {:?}", append_request);
// need to init safekeeper state before AppendRequest
prepare_safekeeper(swh)?;
let inserted_wal = append_logical_message(swh, append_request)?;
let response = AppendResult {
state: swh.timeline.get().get_info(),
inserted_wal,
};
let response_data = serde_json::to_vec(&response)?;
pgb.write_message_noflush(&BeMessage::RowDescription(&[RowDescriptor {
name: b"json",
typoid: TEXT_OID,
typlen: -1,
..Default::default()
}]))?
.write_message_noflush(&BeMessage::DataRow(&[Some(&response_data)]))?
.write_message(&BeMessage::CommandComplete(b"JSON_CTRL"))?;
Ok(())
}
/// Prepare safekeeper to process append requests without crashes,
/// by sending ProposerGreeting with default server.wal_seg_size.
fn prepare_safekeeper(swh: &mut SendWalHandler) -> Result<()> {
let greeting_request = ProposerAcceptorMessage::Greeting(ProposerGreeting {
protocol_version: 1, // current protocol
pg_version: 0, // unknown
proposer_id: [0u8; 16],
system_id: 0,
ztli: swh.timelineid.unwrap(),
tenant_id: swh.tenantid.unwrap(),
tli: 0,
wal_seg_size: pg_constants::WAL_SEGMENT_SIZE as u32, // 16MB, default for tests
});
let response = swh.timeline.get().process_msg(&greeting_request)?;
match response {
AcceptorProposerMessage::Greeting(_) => Ok(()),
_ => anyhow::bail!("not GreetingResponse"),
}
}
#[derive(Serialize, Deserialize)]
struct InsertedWAL {
begin_lsn: Lsn,
end_lsn: Lsn,
append_response: AppendResponse,
}
/// Extend local WAL with new LogicalMessage record. To do that,
/// create AppendRequest with new WAL and pass it to safekeeper.
fn append_logical_message(
swh: &mut SendWalHandler,
msg: AppendLogicalMessage,
) -> Result<InsertedWAL> {
let wal_data = encode_logical_message(msg.lm_prefix, msg.lm_message);
let sk_state = swh.timeline.get().get_info();
let begin_lsn = msg.begin_lsn;
let end_lsn = begin_lsn + wal_data.len() as u64;
let commit_lsn = if msg.set_commit_lsn {
end_lsn
} else {
sk_state.commit_lsn
};
let append_request = ProposerAcceptorMessage::AppendRequest(AppendRequest {
h: AppendRequestHeader {
term: msg.term,
epoch_start_lsn: begin_lsn,
begin_lsn,
end_lsn,
commit_lsn,
truncate_lsn: msg.truncate_lsn,
proposer_uuid: [0u8; 16],
},
wal_data: Bytes::from(wal_data),
});
let response = swh.timeline.get().process_msg(&append_request)?;
let append_response = match response {
AcceptorProposerMessage::AppendResponse(resp) => resp,
_ => anyhow::bail!("not AppendResponse"),
};
Ok(InsertedWAL {
begin_lsn,
end_lsn,
append_response,
})
}
#[repr(C)]
#[derive(Debug, Clone, Default, Serialize, Deserialize)]
struct XlLogicalMessage {
db_id: Oid,
transactional: uint32, // bool, takes 4 bytes due to alignment in C structures
prefix_size: uint64,
message_size: uint64,
}
impl XlLogicalMessage {
pub fn encode(&self) -> Bytes {
use zenith_utils::bin_ser::LeSer;
self.ser().unwrap().into()
}
}
/// Create new WAL record for non-transactional logical message.
/// Used for creating artificial WAL for tests, as LogicalMessage
/// record is basically no-op.
fn encode_logical_message(prefix: String, message: String) -> Vec<u8> {
let mut prefix_bytes = BytesMut::with_capacity(prefix.len() + 1);
prefix_bytes.put(prefix.as_bytes());
prefix_bytes.put_u8(0);
let message_bytes = message.as_bytes();
let logical_message = XlLogicalMessage {
db_id: 0,
transactional: 0,
prefix_size: prefix_bytes.len() as u64,
message_size: message_bytes.len() as u64,
};
let mainrdata = logical_message.encode();
let mainrdata_len: usize = mainrdata.len() + prefix_bytes.len() + message_bytes.len();
// only short mainrdata is supported for now
assert!(mainrdata_len <= 255);
let mainrdata_len = mainrdata_len as u8;
let mut data: Vec<u8> = vec![pg_constants::XLR_BLOCK_ID_DATA_SHORT, mainrdata_len];
data.extend_from_slice(&mainrdata);
data.extend_from_slice(&prefix_bytes);
data.extend_from_slice(message_bytes);
let total_len = xlog_utils::XLOG_SIZE_OF_XLOG_RECORD + data.len();
let mut header = XLogRecord {
xl_tot_len: total_len as u32,
xl_xid: 0,
xl_prev: 0,
xl_info: 0,
xl_rmid: 21,
__bindgen_padding_0: [0u8; 2usize],
xl_crc: 0, // crc will be calculated later
};
let header_bytes = header.encode();
let crc = crc32c_append(0, &data);
let crc = crc32c_append(crc, &header_bytes[0..xlog_utils::XLOG_RECORD_CRC_OFFS]);
header.xl_crc = crc;
let mut wal: Vec<u8> = Vec::new();
wal.extend_from_slice(&header.encode());
wal.extend_from_slice(&data);
// WAL start position must be aligned at 8 bytes,
// this will add padding for the next WAL record.
const PADDING: usize = 8;
let padding_rem = wal.len() % PADDING;
if padding_rem != 0 {
wal.resize(wal.len() + PADDING - padding_rem, 0);
}
wal
}
#[test]
fn test_encode_logical_message() {
let expected = [
64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 21, 0, 0, 170, 34, 166, 227, 255, 38,
0, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 112, 114, 101, 102,
105, 120, 0, 109, 101, 115, 115, 97, 103, 101,
];
let actual = encode_logical_message("prefix".to_string(), "message".to_string());
assert_eq!(expected, actual[..]);
}

View File

@@ -2,7 +2,6 @@
use std::path::PathBuf;
use std::time::Duration;
pub mod json_ctrl;
pub mod receive_wal;
pub mod replication;
pub mod s3_offload;

View File

@@ -28,7 +28,7 @@ const SK_PROTOCOL_VERSION: u32 = 1;
const UNKNOWN_SERVER_VERSION: u32 = 0;
/// Consensus logical timestamp.
pub type Term = u64;
type Term = u64;
/// Unique id of proposer. Not needed for correctness, used for monitoring.
type PgUuid = [u8; 16];
@@ -111,7 +111,7 @@ impl Default for SafeKeeperState {
// protocol messages
/// Initial Proposer -> Acceptor message
#[derive(Debug, Deserialize)]
#[derive(Debug, Serialize, Deserialize)]
pub struct ProposerGreeting {
/// proposer-acceptor protocol version
pub protocol_version: u32,
@@ -128,19 +128,19 @@ pub struct ProposerGreeting {
/// Acceptor -> Proposer initial response: the highest term known to me
/// (acceptor voted for).
#[derive(Debug, Serialize)]
#[derive(Debug, Serialize, Deserialize)]
pub struct AcceptorGreeting {
term: u64,
}
/// Vote request sent from proposer to safekeepers
#[derive(Debug, Deserialize)]
#[derive(Debug, Serialize, Deserialize)]
pub struct VoteRequest {
term: Term,
}
/// Vote itself, sent from safekeeper to proposer
#[derive(Debug, Serialize)]
#[derive(Debug, Serialize, Deserialize)]
pub struct VoteResponse {
term: Term, // not really needed, just a sanity check
vote_given: u64, // fixme u64 due to padding
@@ -152,26 +152,26 @@ pub struct VoteResponse {
/// Request with WAL message sent from proposer to safekeeper. Along the way it
/// announces 1) successful election (with epoch_start_lsn); 2) commit_lsn.
#[derive(Debug)]
#[derive(Debug, Serialize, Deserialize)]
pub struct AppendRequest {
pub h: AppendRequestHeader,
pub wal_data: Bytes,
h: AppendRequestHeader,
wal_data: Bytes,
}
#[derive(Debug, Clone, Deserialize)]
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct AppendRequestHeader {
pub term: Term,
term: Term,
// LSN since the proposer appends WAL; determines epoch switch point.
pub epoch_start_lsn: Lsn,
epoch_start_lsn: Lsn,
/// start position of message in WAL
pub begin_lsn: Lsn,
begin_lsn: Lsn,
/// end position of message in WAL
pub end_lsn: Lsn,
end_lsn: Lsn,
/// LSN committed by quorum of safekeepers
pub commit_lsn: Lsn,
commit_lsn: Lsn,
/// minimal LSN which may be needed by proposer to perform recovery of some safekeeper
pub truncate_lsn: Lsn,
truncate_lsn: Lsn,
// only for logging/debugging
pub proposer_uuid: PgUuid,
proposer_uuid: PgUuid,
}
/// Report safekeeper state to proposer
@@ -275,8 +275,8 @@ impl AcceptorProposerMessage {
pub trait Storage {
/// Persist safekeeper state on disk, optionally syncing it.
fn persist(&mut self, s: &SafeKeeperState, sync: bool) -> Result<()>;
/// Write piece of wal in buf to disk and sync it.
fn write_wal(&mut self, server: &ServerInfo, startpos: Lsn, buf: &[u8]) -> Result<()>;
/// Write piece of wal in buf to disk.
fn write_wal(&mut self, s: &SafeKeeperState, startpos: Lsn, buf: &[u8]) -> Result<()>;
}
/// SafeKeeper which consumes events (messages from compute) and provides
@@ -423,7 +423,7 @@ where
let mut last_rec_lsn = Lsn(0);
if !msg.wal_data.is_empty() {
self.storage
.write_wal(&self.s.server, msg.h.begin_lsn, &msg.wal_data)?;
.write_wal(&self.s, msg.h.begin_lsn, &msg.wal_data)?;
// figure out last record's end lsn for reporting (if we got the
// whole record)
@@ -546,7 +546,7 @@ mod tests {
Ok(())
}
fn write_wal(&mut self, _server: &ServerInfo, _startpos: Lsn, _buf: &[u8]) -> Result<()> {
fn write_wal(&mut self, _s: &SafeKeeperState, _startpos: Lsn, _buf: &[u8]) -> Result<()> {
Ok(())
}
}

View File

@@ -2,7 +2,6 @@
//! pageserver/any other consumer.
//!
use crate::json_ctrl::handle_json_ctrl;
use crate::receive_wal::ReceiveWalConn;
use crate::replication::ReplicationConn;
use crate::timeline::{Timeline, TimelineTools};
@@ -50,11 +49,9 @@ impl postgres_backend::Handler for SendWalHandler {
}
fn process_query(&mut self, pgb: &mut PostgresBackend, query_string: Bytes) -> Result<()> {
// START_WAL_PUSH is the only command that initializes the timeline in production.
// There is also JSON_CTRL command, which should initialize the timeline for testing.
// START_WAL_PUSH is the only command that initializes the timeline
if self.timeline.is_none() {
if query_string.starts_with(b"START_WAL_PUSH") || query_string.starts_with(b"JSON_CTRL")
{
if query_string.starts_with(b"START_WAL_PUSH") {
self.timeline.set(
&self.conf,
self.tenantid.unwrap(),
@@ -79,9 +76,6 @@ impl postgres_backend::Handler for SendWalHandler {
} else if query_string.starts_with(b"START_WAL_PUSH") {
ReceiveWalConn::new(pgb)?.run(self)?;
Ok(())
} else if query_string.starts_with(b"JSON_CTRL") {
handle_json_ctrl(self, pgb, &query_string)?;
Ok(())
} else {
bail!("Unexpected command {:?}", query_string);
}

View File

@@ -18,8 +18,8 @@ use zenith_utils::zid::{ZTenantId, ZTimelineId};
use crate::replication::{HotStandbyFeedback, END_REPLICATION_MARKER};
use crate::safekeeper::{
AcceptorProposerMessage, ProposerAcceptorMessage, SafeKeeper, SafeKeeperState, ServerInfo,
Storage, SK_FORMAT_VERSION, SK_MAGIC,
AcceptorProposerMessage, ProposerAcceptorMessage, SafeKeeper, SafeKeeperState, Storage,
SK_FORMAT_VERSION, SK_MAGIC,
};
use crate::WalAcceptorConf;
use postgres_ffi::xlog_utils::{XLogFileName, XLOG_BLCKSZ};
@@ -32,7 +32,7 @@ struct SharedState {
sk: SafeKeeper<FileStorage>,
/// For receiving-sending wal cooperation
/// quorum commit LSN we've notified walsenders about
notified_commit_lsn: Lsn,
commit_lsn: Lsn,
/// combined hot standby feedback from all replicas
hs_feedback: HotStandbyFeedback,
}
@@ -72,7 +72,7 @@ impl SharedState {
};
Ok(Self {
notified_commit_lsn: Lsn(0),
commit_lsn: Lsn(0),
sk: SafeKeeper::new(Lsn(flush_lsn), tli, storage, state),
hs_feedback: HotStandbyFeedback {
ts: 0,
@@ -186,7 +186,7 @@ impl Timeline {
pub fn wait_for_lsn(&self, lsn: Lsn) -> Lsn {
let mut shared_state = self.mutex.lock().unwrap();
loop {
let commit_lsn = shared_state.notified_commit_lsn;
let commit_lsn = shared_state.commit_lsn;
// This must be `>`, not `>=`.
if commit_lsn > lsn {
return commit_lsn;
@@ -198,8 +198,8 @@ impl Timeline {
// Notify caught-up WAL senders about new WAL data received
pub fn notify_wal_senders(&self, commit_lsn: Lsn) {
let mut shared_state = self.mutex.lock().unwrap();
if shared_state.notified_commit_lsn < commit_lsn {
shared_state.notified_commit_lsn = commit_lsn;
if shared_state.commit_lsn < commit_lsn {
shared_state.commit_lsn = commit_lsn;
self.cond.notify_all();
}
}
@@ -337,14 +337,14 @@ impl Storage for FileStorage {
Ok(())
}
fn write_wal(&mut self, server: &ServerInfo, startpos: Lsn, buf: &[u8]) -> Result<()> {
fn write_wal(&mut self, s: &SafeKeeperState, startpos: Lsn, buf: &[u8]) -> Result<()> {
let mut bytes_left: usize = buf.len();
let mut bytes_written: usize = 0;
let mut partial;
let mut start_pos = startpos;
const ZERO_BLOCK: &[u8] = &[0u8; XLOG_BLCKSZ];
let wal_seg_size = server.wal_seg_size as usize;
let ztli = server.ztli;
let wal_seg_size = s.server.wal_seg_size as usize;
let ztli = s.server.ztli;
/* Extract WAL location for this block */
let mut xlogoff = start_pos.segment_offset(wal_seg_size) as usize;
@@ -365,7 +365,7 @@ impl Storage for FileStorage {
/* Open file */
let segno = start_pos.segment_number(wal_seg_size);
// note: we basically don't support changing pg timeline
let wal_file_name = XLogFileName(server.tli, segno, wal_seg_size);
let wal_file_name = XLogFileName(s.server.tli, segno, wal_seg_size);
let wal_file_path = self
.conf
.data_dir

View File

@@ -7,4 +7,3 @@ edition = "2018"
prometheus = {version = "0.12", default_features=false} # removes protobuf dependency
libc = "0.2"
lazy_static = "1.4"
once_cell = "1.8.0"

View File

@@ -3,7 +3,6 @@
//! Otherwise, we might not see all metrics registered via
//! a default registry.
use lazy_static::lazy_static;
use once_cell::race::OnceBox;
pub use prometheus::{exponential_buckets, linear_buckets};
pub use prometheus::{register_histogram, Histogram};
pub use prometheus::{register_histogram_vec, HistogramVec};
@@ -25,29 +24,9 @@ pub fn gather() -> Vec<prometheus::proto::MetricFamily> {
prometheus::gather()
}
static COMMON_METRICS_PREFIX: OnceBox<&str> = OnceBox::new();
/// Sets a prefix which will be used for all common metrics, typically a service
/// name like 'pageserver'. Should be executed exactly once in the beginning of
/// any executable which uses common metrics.
pub fn set_common_metrics_prefix(prefix: &'static str) {
COMMON_METRICS_PREFIX.set(prefix.into()).unwrap();
}
/// Prepends a prefix to a common metric name so they are distinguished between
/// different services, see https://github.com/zenithdb/zenith/pull/681
/// A call to set_common_metrics_prefix() is necessary prior to calling this.
pub fn new_common_metric_name(unprefixed_metric_name: &str) -> String {
format!(
"{}_{}",
COMMON_METRICS_PREFIX.get().unwrap(),
unprefixed_metric_name
)
}
lazy_static! {
static ref DISK_IO_BYTES: IntGaugeVec = register_int_gauge_vec!(
new_common_metric_name("disk_io_bytes"),
"pageserver_disk_io_bytes",
"Bytes written and read from disk, grouped by the operation (read|write)",
&["io_operation"]
)

View File

@@ -38,4 +38,3 @@ rustls-split = "0.2.1"
hex-literal = "0.3"
bytes = "1.0"
webpki = "0.21"
tempfile = "3.2"

View File

@@ -1,33 +0,0 @@
/// A helper to "accumulate" a value similar to `Iterator::reduce`, but lets you
/// feed the accumulated values by calling the 'accum' function, instead of having an
/// iterator.
///
/// For example, to calculate the smallest value among some integers:
///
/// ```
/// use zenith_utils::accum::Accum;
///
/// let values = [1, 2, 3];
///
/// let mut min_value: Accum<u32> = Accum(None);
/// for new_value in &values {
/// min_value.accum(std::cmp::min, *new_value);
/// }
///
/// assert_eq!(min_value.0.unwrap(), 1);
/// ```
pub struct Accum<T>(pub Option<T>);
impl<T: Copy> Accum<T> {
pub fn accum<F>(&mut self, func: F, new_value: T)
where
F: FnOnce(T, T) -> T,
{
// If there is no previous value, just store the new value.
// Otherwise call the function to decide which one to keep.
self.0 = Some(if let Some(accum) = self.0 {
func(accum, new_value)
} else {
new_value
});
}
}

View File

@@ -94,12 +94,9 @@ pub fn le_coder() -> impl Options {
/// Binary serialize/deserialize helper functions (Big Endian)
///
pub trait BeSer {
pub trait BeSer: Serialize + DeserializeOwned {
/// Serialize into a byte slice
fn ser_into_slice(&self, mut b: &mut [u8]) -> Result<(), SerializeError>
where
Self: Serialize,
{
fn ser_into_slice(&self, mut b: &mut [u8]) -> Result<(), SerializeError> {
// &mut [u8] implements Write, but `ser_into` needs a mutable
// reference to that. So we need the slightly awkward "mutable
// reference to a mutable reference.
@@ -110,28 +107,19 @@ pub trait BeSer {
///
/// This is useful for most `Write` types except `&mut [u8]`, which
/// can more easily use [`ser_into_slice`](Self::ser_into_slice).
fn ser_into<W: Write>(&self, w: &mut W) -> Result<(), SerializeError>
where
Self: Serialize,
{
fn ser_into<W: Write>(&self, w: &mut W) -> Result<(), SerializeError> {
be_coder().serialize_into(w, &self).map_err(|e| e.into())
}
/// Serialize into a new heap-allocated buffer
fn ser(&self) -> Result<Vec<u8>, SerializeError>
where
Self: Serialize,
{
fn ser(&self) -> Result<Vec<u8>, SerializeError> {
be_coder().serialize(&self).map_err(|e| e.into())
}
/// Deserialize from the full contents of a byte slice
///
/// See also: [`BeSer::des_prefix`]
fn des(buf: &[u8]) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des(buf: &[u8]) -> Result<Self, DeserializeError> {
be_coder()
.deserialize(buf)
.or(Err(DeserializeError::BadInput))
@@ -143,10 +131,7 @@ pub trait BeSer {
/// type, but does not guarantee that the entire slice is used.
///
/// See also: [`BeSer::des`]
fn des_prefix(buf: &[u8]) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des_prefix(buf: &[u8]) -> Result<Self, DeserializeError> {
be_coder()
.allow_trailing_bytes()
.deserialize(buf)
@@ -154,10 +139,7 @@ pub trait BeSer {
}
/// Deserialize from a reader
fn des_from<R: Read>(r: &mut R) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des_from<R: Read>(r: &mut R) -> Result<Self, DeserializeError> {
be_coder().deserialize_from(r).map_err(|e| e.into())
}
@@ -165,22 +147,16 @@ pub trait BeSer {
///
/// Note: it may be faster to serialize to a buffer and then measure the
/// buffer length, than to call `serialized_size` and then `ser_into`.
fn serialized_size(&self) -> Result<u64, SerializeError>
where
Self: Serialize,
{
fn serialized_size(&self) -> Result<u64, SerializeError> {
be_coder().serialized_size(self).map_err(|e| e.into())
}
}
/// Binary serialize/deserialize helper functions (Little Endian)
///
pub trait LeSer {
pub trait LeSer: Serialize + DeserializeOwned {
/// Serialize into a byte slice
fn ser_into_slice(&self, mut b: &mut [u8]) -> Result<(), SerializeError>
where
Self: Serialize,
{
fn ser_into_slice(&self, mut b: &mut [u8]) -> Result<(), SerializeError> {
// &mut [u8] implements Write, but `ser_into` needs a mutable
// reference to that. So we need the slightly awkward "mutable
// reference to a mutable reference.
@@ -191,28 +167,19 @@ pub trait LeSer {
///
/// This is useful for most `Write` types except `&mut [u8]`, which
/// can more easily use [`ser_into_slice`](Self::ser_into_slice).
fn ser_into<W: Write>(&self, w: &mut W) -> Result<(), SerializeError>
where
Self: Serialize,
{
fn ser_into<W: Write>(&self, w: &mut W) -> Result<(), SerializeError> {
le_coder().serialize_into(w, &self).map_err(|e| e.into())
}
/// Serialize into a new heap-allocated buffer
fn ser(&self) -> Result<Vec<u8>, SerializeError>
where
Self: Serialize,
{
fn ser(&self) -> Result<Vec<u8>, SerializeError> {
le_coder().serialize(&self).map_err(|e| e.into())
}
/// Deserialize from the full contents of a byte slice
///
/// See also: [`LeSer::des_prefix`]
fn des(buf: &[u8]) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des(buf: &[u8]) -> Result<Self, DeserializeError> {
le_coder()
.deserialize(buf)
.or(Err(DeserializeError::BadInput))
@@ -224,10 +191,7 @@ pub trait LeSer {
/// type, but does not guarantee that the entire slice is used.
///
/// See also: [`LeSer::des`]
fn des_prefix(buf: &[u8]) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des_prefix(buf: &[u8]) -> Result<Self, DeserializeError> {
le_coder()
.allow_trailing_bytes()
.deserialize(buf)
@@ -235,10 +199,7 @@ pub trait LeSer {
}
/// Deserialize from a reader
fn des_from<R: Read>(r: &mut R) -> Result<Self, DeserializeError>
where
Self: DeserializeOwned,
{
fn des_from<R: Read>(r: &mut R) -> Result<Self, DeserializeError> {
le_coder().deserialize_from(r).map_err(|e| e.into())
}
@@ -246,18 +207,14 @@ pub trait LeSer {
///
/// Note: it may be faster to serialize to a buffer and then measure the
/// buffer length, than to call `serialized_size` and then `ser_into`.
fn serialized_size(&self) -> Result<u64, SerializeError>
where
Self: Serialize,
{
fn serialized_size(&self) -> Result<u64, SerializeError> {
le_coder().serialized_size(self).map_err(|e| e.into())
}
}
// Because usage of `BeSer` or `LeSer` can be done with *either* a Serialize or
// DeserializeOwned implementation, the blanket implementation has to be for every type.
impl<T> BeSer for T {}
impl<T> LeSer for T {}
impl<T> BeSer for T where T: Serialize + DeserializeOwned {}
impl<T> LeSer for T where T: Serialize + DeserializeOwned {}
#[cfg(test)]
mod tests {

View File

@@ -1,125 +0,0 @@
use std::{
fs::{self, File},
io,
path::Path,
};
/// Similar to [`std::fs::create_dir`], except we fsync the
/// created directory and its parent.
pub fn create_dir(path: impl AsRef<Path>) -> io::Result<()> {
let path = path.as_ref();
fs::create_dir(path)?;
File::open(path)?.sync_all()?;
if let Some(parent) = path.parent() {
File::open(parent)?.sync_all()
} else {
Err(io::Error::new(
io::ErrorKind::InvalidInput,
"can't find parent",
))
}
}
/// Similar to [`std::fs::create_dir_all`], except we fsync all
/// newly created directories and the pre-existing parent.
pub fn create_dir_all(path: impl AsRef<Path>) -> io::Result<()> {
let mut path = path.as_ref();
let mut dirs_to_create = Vec::new();
// Figure out which directories we need to create.
loop {
match path.metadata() {
Ok(metadata) if metadata.is_dir() => break,
Ok(_) => {
return Err(io::Error::new(
io::ErrorKind::AlreadyExists,
format!("non-directory found in path: {:?}", path),
));
}
Err(ref e) if e.kind() == io::ErrorKind::NotFound => {}
Err(e) => return Err(e),
}
dirs_to_create.push(path);
match path.parent() {
Some(parent) => path = parent,
None => {
return Err(io::Error::new(
io::ErrorKind::InvalidInput,
"can't find parent",
))
}
}
}
// Create directories from parent to child.
for &path in dirs_to_create.iter().rev() {
fs::create_dir(path)?;
}
// Fsync the created directories from child to parent.
for &path in dirs_to_create.iter() {
File::open(path)?.sync_all()?;
}
// If we created any new directories, fsync the parent.
if !dirs_to_create.is_empty() {
File::open(path)?.sync_all()?;
}
Ok(())
}
#[cfg(test)]
mod tests {
use tempfile::tempdir;
use super::*;
#[test]
fn test_create_dir_fsyncd() {
let dir = tempdir().unwrap();
let existing_dir_path = dir.path();
let err = create_dir(existing_dir_path).unwrap_err();
assert_eq!(err.kind(), io::ErrorKind::AlreadyExists);
let child_dir = existing_dir_path.join("child");
create_dir(child_dir).unwrap();
let nested_child_dir = existing_dir_path.join("child1").join("child2");
let err = create_dir(nested_child_dir).unwrap_err();
assert_eq!(err.kind(), io::ErrorKind::NotFound);
}
#[test]
fn test_create_dir_all_fsyncd() {
let dir = tempdir().unwrap();
let existing_dir_path = dir.path();
create_dir_all(existing_dir_path).unwrap();
let child_dir = existing_dir_path.join("child");
assert!(!child_dir.exists());
create_dir_all(&child_dir).unwrap();
assert!(child_dir.exists());
let nested_child_dir = existing_dir_path.join("child1").join("child2");
assert!(!nested_child_dir.exists());
create_dir_all(&nested_child_dir).unwrap();
assert!(nested_child_dir.exists());
let file_path = existing_dir_path.join("file");
std::fs::write(&file_path, b"").unwrap();
let err = create_dir_all(&file_path).unwrap_err();
assert_eq!(err.kind(), io::ErrorKind::AlreadyExists);
let invalid_dir_path = file_path.join("folder");
create_dir_all(&invalid_dir_path).unwrap_err();
}
}

View File

@@ -9,14 +9,14 @@ use routerify::ext::RequestExt;
use routerify::RequestInfo;
use routerify::{Middleware, Router, RouterBuilder, RouterService};
use std::net::TcpListener;
use zenith_metrics::{new_common_metric_name, register_int_counter, IntCounter};
use zenith_metrics::{register_int_counter, IntCounter};
use zenith_metrics::{Encoder, TextEncoder};
use super::error::ApiError;
lazy_static! {
static ref SERVE_METRICS_COUNT: IntCounter = register_int_counter!(
new_common_metric_name("serve_metrics_count"),
"pageserver_serve_metrics_count",
"Number of metric requests made"
)
.expect("failed to define a metric");

View File

@@ -18,9 +18,6 @@ pub mod pq_proto;
// dealing with connstring parsing and handy access to it's parts
pub mod connstring;
// helper functions for creating and fsyncing directories/trees
pub mod crashsafe_dir;
// common authentication routines
pub mod auth;
@@ -34,6 +31,3 @@ pub mod sock_split;
// common log initialisation routine
pub mod logging;
// Misc
pub mod accum;

View File

@@ -192,7 +192,9 @@ impl AtomicLsn {
/// This operation will panic on overflow.
pub fn fetch_add(&self, val: u64) -> Lsn {
let prev = self.inner.fetch_add(val, Ordering::AcqRel);
assert!(prev.checked_add(val).is_some(), "AtomicLsn overflow");
if prev.checked_add(val).is_none() {
panic!("AtomicLsn overflow");
}
Lsn(prev)
}

View File

@@ -652,7 +652,7 @@ impl<'a> BeMessage<'a> {
}
BeMessage::EncryptionResponse(should_negotiate) => {
let response = if *should_negotiate { b'S' } else { b'N' };
let response = if *should_negotiate { b'Y' } else { b'N' };
buf.put_u8(response);
}

View File

@@ -78,10 +78,6 @@ macro_rules! zid_newtype {
pub fn generate() -> Self {
$t(ZId::generate())
}
pub const fn from_array(b: [u8; 16]) -> Self {
$t(ZId(b))
}
}
impl FromStr for $t {

View File

@@ -1,10 +1,10 @@
use bytes::{Buf, BytesMut};
use hex_literal::hex;
use serde::Deserialize;
use serde::{Deserialize, Serialize};
use std::io::Read;
use zenith_utils::bin_ser::LeSer;
#[derive(Debug, PartialEq, Deserialize)]
#[derive(Debug, PartialEq, Serialize, Deserialize)]
pub struct HeaderData {
magic: u16,
info: u16,

View File

@@ -43,7 +43,7 @@ fn ssl() {
client_sock.write_u32::<BigEndian>(80877103).unwrap();
let ssl_response = client_sock.read_u8().unwrap();
assert_eq!(b'S', ssl_response);
assert_eq!(b'Y', ssl_response);
let mut cfg = rustls::ClientConfig::new();
cfg.root_store.add(&CERT).unwrap();