safekeeper: use protobuf for sending compressed records to pageserver (#9821)

## Problem

https://github.com/neondatabase/neon/pull/9746 lifted decoding and
interpretation of WAL to the safekeeper.
This reduced the ingested amount on the pageservers by around 10x for a
tenant with 8 shards, but doubled
the ingested amount for single sharded tenants.

Also, https://github.com/neondatabase/neon/pull/9746 uses bincode which
doesn't support schema evolution.
Technically the schema can be evolved, but it's very cumbersome.

## Summary of changes

This patch set addresses both problems by adding protobuf support for
the interpreted wal records and adding compression support. Compressed
protobuf reduced the ingested amount by 100x on the 32 shards
`test_sharded_ingest` case (compared to non-interpreted proto). For the
1 shard case the reduction is 5x.

Sister change to `rust-postgres` is
[here](https://github.com/neondatabase/rust-postgres/pull/33).

## Links

Related: https://github.com/neondatabase/neon/issues/9336
Epic: https://github.com/neondatabase/neon/issues/9329
This commit is contained in:
Vlad Lazar
2024-11-27 12:12:21 +00:00
committed by GitHub
parent 7b41ee872e
commit 9e0148de11
21 changed files with 702 additions and 106 deletions

View File

@@ -8,11 +8,19 @@ license.workspace = true
testing = ["pageserver_api/testing"]
[dependencies]
async-compression.workspace = true
anyhow.workspace = true
bytes.workspace = true
pageserver_api.workspace = true
prost.workspace = true
postgres_ffi.workspace = true
serde.workspace = true
thiserror.workspace = true
tokio = { workspace = true, features = ["io-util"] }
tonic.workspace = true
tracing.workspace = true
utils.workspace = true
workspace_hack = { version = "0.1", path = "../../workspace_hack" }
[build-dependencies]
tonic-build.workspace = true

11
libs/wal_decoder/build.rs Normal file
View File

@@ -0,0 +1,11 @@
fn main() -> Result<(), Box<dyn std::error::Error>> {
// Generate rust code from .proto protobuf.
//
// Note: we previously tried to use deterministic location at proto/ for
// easy location, but apparently interference with cachepot sometimes fails
// the build then. Anyway, per cargo docs build script shouldn't output to
// anywhere but $OUT_DIR.
tonic_build::compile_protos("proto/interpreted_wal.proto")
.unwrap_or_else(|e| panic!("failed to compile protos {:?}", e));
Ok(())
}

View File

@@ -0,0 +1,43 @@
syntax = "proto3";
package interpreted_wal;
message InterpretedWalRecords {
repeated InterpretedWalRecord records = 1;
optional uint64 next_record_lsn = 2;
}
message InterpretedWalRecord {
optional bytes metadata_record = 1;
SerializedValueBatch batch = 2;
uint64 next_record_lsn = 3;
bool flush_uncommitted = 4;
uint32 xid = 5;
}
message SerializedValueBatch {
bytes raw = 1;
repeated ValueMeta metadata = 2;
uint64 max_lsn = 3;
uint64 len = 4;
}
enum ValueMetaType {
Serialized = 0;
Observed = 1;
}
message ValueMeta {
ValueMetaType type = 1;
CompactKey key = 2;
uint64 lsn = 3;
optional uint64 batch_offset = 4;
optional uint64 len = 5;
optional bool will_init = 6;
}
message CompactKey {
int64 high = 1;
int64 low = 2;
}

View File

@@ -1,3 +1,4 @@
pub mod decoder;
pub mod models;
pub mod serialized_batch;
pub mod wire_format;

View File

@@ -37,12 +37,32 @@ use utils::lsn::Lsn;
use crate::serialized_batch::SerializedValueBatch;
// Code generated by protobuf.
pub mod proto {
// Tonic does derives as `#[derive(Clone, PartialEq, ::prost::Message)]`
// we don't use these types for anything but broker data transmission,
// so it's ok to ignore this one.
#![allow(clippy::derive_partial_eq_without_eq)]
// The generated ValueMeta has a `len` method generate for its `len` field.
#![allow(clippy::len_without_is_empty)]
tonic::include_proto!("interpreted_wal");
}
#[derive(Serialize, Deserialize)]
pub enum FlushUncommittedRecords {
Yes,
No,
}
/// A batch of interpreted WAL records
#[derive(Serialize, Deserialize)]
pub struct InterpretedWalRecords {
pub records: Vec<InterpretedWalRecord>,
// Start LSN of the next record after the batch.
// Note that said record may not belong to the current shard.
pub next_record_lsn: Option<Lsn>,
}
/// An interpreted Postgres WAL record, ready to be handled by the pageserver
#[derive(Serialize, Deserialize)]
pub struct InterpretedWalRecord {

View File

@@ -0,0 +1,356 @@
use bytes::{BufMut, Bytes, BytesMut};
use pageserver_api::key::CompactKey;
use prost::{DecodeError, EncodeError, Message};
use tokio::io::AsyncWriteExt;
use utils::bin_ser::{BeSer, DeserializeError, SerializeError};
use utils::lsn::Lsn;
use utils::postgres_client::{Compression, InterpretedFormat};
use crate::models::{
FlushUncommittedRecords, InterpretedWalRecord, InterpretedWalRecords, MetadataRecord,
};
use crate::serialized_batch::{
ObservedValueMeta, SerializedValueBatch, SerializedValueMeta, ValueMeta,
};
use crate::models::proto;
#[derive(Debug, thiserror::Error)]
pub enum ToWireFormatError {
#[error("{0}")]
Bincode(#[from] SerializeError),
#[error("{0}")]
Protobuf(#[from] ProtobufSerializeError),
#[error("{0}")]
Compression(#[from] std::io::Error),
}
#[derive(Debug, thiserror::Error)]
pub enum ProtobufSerializeError {
#[error("{0}")]
MetadataRecord(#[from] SerializeError),
#[error("{0}")]
Encode(#[from] EncodeError),
}
#[derive(Debug, thiserror::Error)]
pub enum FromWireFormatError {
#[error("{0}")]
Bincode(#[from] DeserializeError),
#[error("{0}")]
Protobuf(#[from] ProtobufDeserializeError),
#[error("{0}")]
Decompress(#[from] std::io::Error),
}
#[derive(Debug, thiserror::Error)]
pub enum ProtobufDeserializeError {
#[error("{0}")]
Transcode(#[from] TranscodeError),
#[error("{0}")]
Decode(#[from] DecodeError),
}
#[derive(Debug, thiserror::Error)]
pub enum TranscodeError {
#[error("{0}")]
BadInput(String),
#[error("{0}")]
MetadataRecord(#[from] DeserializeError),
}
pub trait ToWireFormat {
fn to_wire(
self,
format: InterpretedFormat,
compression: Option<Compression>,
) -> impl std::future::Future<Output = Result<Bytes, ToWireFormatError>> + Send;
}
pub trait FromWireFormat {
type T;
fn from_wire(
buf: &Bytes,
format: InterpretedFormat,
compression: Option<Compression>,
) -> impl std::future::Future<Output = Result<Self::T, FromWireFormatError>> + Send;
}
impl ToWireFormat for InterpretedWalRecords {
async fn to_wire(
self,
format: InterpretedFormat,
compression: Option<Compression>,
) -> Result<Bytes, ToWireFormatError> {
use async_compression::tokio::write::ZstdEncoder;
use async_compression::Level;
let encode_res: Result<Bytes, ToWireFormatError> = match format {
InterpretedFormat::Bincode => {
let buf = BytesMut::new();
let mut buf = buf.writer();
self.ser_into(&mut buf)?;
Ok(buf.into_inner().freeze())
}
InterpretedFormat::Protobuf => {
let proto: proto::InterpretedWalRecords = self.try_into()?;
let mut buf = BytesMut::new();
proto
.encode(&mut buf)
.map_err(|e| ToWireFormatError::Protobuf(e.into()))?;
Ok(buf.freeze())
}
};
let buf = encode_res?;
let compressed_buf = match compression {
Some(Compression::Zstd { level }) => {
let mut encoder = ZstdEncoder::with_quality(
Vec::with_capacity(buf.len() / 4),
Level::Precise(level as i32),
);
encoder.write_all(&buf).await?;
encoder.shutdown().await?;
Bytes::from(encoder.into_inner())
}
None => buf,
};
Ok(compressed_buf)
}
}
impl FromWireFormat for InterpretedWalRecords {
type T = Self;
async fn from_wire(
buf: &Bytes,
format: InterpretedFormat,
compression: Option<Compression>,
) -> Result<Self, FromWireFormatError> {
let decompressed_buf = match compression {
Some(Compression::Zstd { .. }) => {
use async_compression::tokio::write::ZstdDecoder;
let mut decoded_buf = Vec::with_capacity(buf.len());
let mut decoder = ZstdDecoder::new(&mut decoded_buf);
decoder.write_all(buf).await?;
decoder.flush().await?;
Bytes::from(decoded_buf)
}
None => buf.clone(),
};
match format {
InterpretedFormat::Bincode => {
InterpretedWalRecords::des(&decompressed_buf).map_err(FromWireFormatError::Bincode)
}
InterpretedFormat::Protobuf => {
let proto = proto::InterpretedWalRecords::decode(decompressed_buf)
.map_err(|e| FromWireFormatError::Protobuf(e.into()))?;
InterpretedWalRecords::try_from(proto)
.map_err(|e| FromWireFormatError::Protobuf(e.into()))
}
}
}
}
impl TryFrom<InterpretedWalRecords> for proto::InterpretedWalRecords {
type Error = SerializeError;
fn try_from(value: InterpretedWalRecords) -> Result<Self, Self::Error> {
let records = value
.records
.into_iter()
.map(proto::InterpretedWalRecord::try_from)
.collect::<Result<Vec<_>, _>>()?;
Ok(proto::InterpretedWalRecords {
records,
next_record_lsn: value.next_record_lsn.map(|l| l.0),
})
}
}
impl TryFrom<InterpretedWalRecord> for proto::InterpretedWalRecord {
type Error = SerializeError;
fn try_from(value: InterpretedWalRecord) -> Result<Self, Self::Error> {
let metadata_record = value
.metadata_record
.map(|meta_rec| -> Result<Vec<u8>, Self::Error> {
let mut buf = Vec::new();
meta_rec.ser_into(&mut buf)?;
Ok(buf)
})
.transpose()?;
Ok(proto::InterpretedWalRecord {
metadata_record,
batch: Some(proto::SerializedValueBatch::from(value.batch)),
next_record_lsn: value.next_record_lsn.0,
flush_uncommitted: matches!(value.flush_uncommitted, FlushUncommittedRecords::Yes),
xid: value.xid,
})
}
}
impl From<SerializedValueBatch> for proto::SerializedValueBatch {
fn from(value: SerializedValueBatch) -> Self {
proto::SerializedValueBatch {
raw: value.raw,
metadata: value
.metadata
.into_iter()
.map(proto::ValueMeta::from)
.collect(),
max_lsn: value.max_lsn.0,
len: value.len as u64,
}
}
}
impl From<ValueMeta> for proto::ValueMeta {
fn from(value: ValueMeta) -> Self {
match value {
ValueMeta::Observed(obs) => proto::ValueMeta {
r#type: proto::ValueMetaType::Observed.into(),
key: Some(proto::CompactKey::from(obs.key)),
lsn: obs.lsn.0,
batch_offset: None,
len: None,
will_init: None,
},
ValueMeta::Serialized(ser) => proto::ValueMeta {
r#type: proto::ValueMetaType::Serialized.into(),
key: Some(proto::CompactKey::from(ser.key)),
lsn: ser.lsn.0,
batch_offset: Some(ser.batch_offset),
len: Some(ser.len as u64),
will_init: Some(ser.will_init),
},
}
}
}
impl From<CompactKey> for proto::CompactKey {
fn from(value: CompactKey) -> Self {
proto::CompactKey {
high: (value.raw() >> 64) as i64,
low: value.raw() as i64,
}
}
}
impl TryFrom<proto::InterpretedWalRecords> for InterpretedWalRecords {
type Error = TranscodeError;
fn try_from(value: proto::InterpretedWalRecords) -> Result<Self, Self::Error> {
let records = value
.records
.into_iter()
.map(InterpretedWalRecord::try_from)
.collect::<Result<_, _>>()?;
Ok(InterpretedWalRecords {
records,
next_record_lsn: value.next_record_lsn.map(Lsn::from),
})
}
}
impl TryFrom<proto::InterpretedWalRecord> for InterpretedWalRecord {
type Error = TranscodeError;
fn try_from(value: proto::InterpretedWalRecord) -> Result<Self, Self::Error> {
let metadata_record = value
.metadata_record
.map(|mrec| -> Result<_, DeserializeError> { MetadataRecord::des(&mrec) })
.transpose()?;
let batch = {
let batch = value.batch.ok_or_else(|| {
TranscodeError::BadInput("InterpretedWalRecord::batch missing".to_string())
})?;
SerializedValueBatch::try_from(batch)?
};
Ok(InterpretedWalRecord {
metadata_record,
batch,
next_record_lsn: Lsn(value.next_record_lsn),
flush_uncommitted: if value.flush_uncommitted {
FlushUncommittedRecords::Yes
} else {
FlushUncommittedRecords::No
},
xid: value.xid,
})
}
}
impl TryFrom<proto::SerializedValueBatch> for SerializedValueBatch {
type Error = TranscodeError;
fn try_from(value: proto::SerializedValueBatch) -> Result<Self, Self::Error> {
let metadata = value
.metadata
.into_iter()
.map(ValueMeta::try_from)
.collect::<Result<Vec<_>, _>>()?;
Ok(SerializedValueBatch {
raw: value.raw,
metadata,
max_lsn: Lsn(value.max_lsn),
len: value.len as usize,
})
}
}
impl TryFrom<proto::ValueMeta> for ValueMeta {
type Error = TranscodeError;
fn try_from(value: proto::ValueMeta) -> Result<Self, Self::Error> {
match proto::ValueMetaType::try_from(value.r#type) {
Ok(proto::ValueMetaType::Serialized) => {
Ok(ValueMeta::Serialized(SerializedValueMeta {
key: value
.key
.ok_or_else(|| {
TranscodeError::BadInput("ValueMeta::key missing".to_string())
})?
.into(),
lsn: Lsn(value.lsn),
batch_offset: value.batch_offset.ok_or_else(|| {
TranscodeError::BadInput("ValueMeta::batch_offset missing".to_string())
})?,
len: value.len.ok_or_else(|| {
TranscodeError::BadInput("ValueMeta::len missing".to_string())
})? as usize,
will_init: value.will_init.ok_or_else(|| {
TranscodeError::BadInput("ValueMeta::will_init missing".to_string())
})?,
}))
}
Ok(proto::ValueMetaType::Observed) => Ok(ValueMeta::Observed(ObservedValueMeta {
key: value
.key
.ok_or_else(|| TranscodeError::BadInput("ValueMeta::key missing".to_string()))?
.into(),
lsn: Lsn(value.lsn),
})),
Err(_) => Err(TranscodeError::BadInput(format!(
"Unexpected ValueMeta::type {}",
value.r#type
))),
}
}
}
impl From<proto::CompactKey> for CompactKey {
fn from(value: proto::CompactKey) -> Self {
(((value.high as i128) << 64) | (value.low as i128)).into()
}
}