Compare commits

...

21 Commits

Author SHA1 Message Date
Vlad Lazar
b87ac5b375 pageserver: move things around to prepare for decoding logic
We wish to have high level WAL decoding logic in `wal_decoder::decoder`
module. For this we need the `Value` and `NeonWalRecord` types
accessible there, so:
1. Move `Value` and `NeonWalRecord` to `pageserver_api::value` and
   `pageserver_api::record` respectively. I had to add a testing feature
   to `pageserver_api` to get this working due to `NeonWalRecord` test
   directives.
2. Get rid of `pageserver::repository` (follow up from (1))
3. Move PG specific WAL record types to `postgres_ffi::record`. In
   theory they could live in `wal_decoder`, but it would create a
   circular dependency between `wal_decoder` and `postgres_ffi`.
   Long term it makes sennse for those types to be PG version specific,
   so that will work out nicely.
4. Move higher level WAL record types (to be ingested by pageserver)
   into `wal_decoder::models`
2024-10-24 17:46:21 +02:00
Vlad Lazar
56101531c0 review: add a comment for the types 2024-10-24 15:51:09 +02:00
Vlad Lazar
b08901c4aa pageserver/walingest: fix failpoint name typo 2024-10-24 15:51:09 +02:00
Vlad Lazar
dd981af5b5 ci: format python code 2024-10-24 15:51:09 +02:00
Vlad Lazar
b6d2c23605 review: rename failpoint 2024-10-24 15:51:09 +02:00
Vlad Lazar
de5ecdc6b8 review: parse logical message prefix on decode 2024-10-24 15:51:09 +02:00
Vlad Lazar
31434f1cb1 review: allow for multiple types of smgr truncation 2024-10-24 15:51:09 +02:00
Vlad Lazar
36a3999c80 pageserver: rename ingest functions
The goal of this commit is to make it clearer when we are ingesting
the whole record versus when we are ingesting an action for the record.

I also merged the VM bits clearing into one function since they were
exactly the same.
2024-10-24 15:51:09 +02:00
Vlad Lazar
69f03617b0 pageserver: wrap all special records in an enum
This will give us a nice evolution path when we want to add
new actions for a record.
2024-10-24 15:51:09 +02:00
Vlad Lazar
828ecbd1ac pageserver: refactor replorigin record 2024-10-24 15:51:09 +02:00
Vlad Lazar
59d8016c5a pageserver: refactor standby record 2024-10-24 15:51:09 +02:00
Vlad Lazar
1836cb23ff pageserer: refactor logical message record 2024-10-24 15:51:09 +02:00
Vlad Lazar
d33bb53a29 pageserver: refactor xlog record
This is an odd one. It requires the current checkpoint value to decide
what to do. That can't trivially be moved to the SK. It's possible with
protocol change, but deferring decision for now. Hence, send the raw
record and let the pageserver figure it out.
2024-10-24 15:51:09 +02:00
Vlad Lazar
d3ad5dc0db pageserver: refactor relmap record 2024-10-24 15:51:09 +02:00
Vlad Lazar
80f6d3909c pageserver: refactor multixact records 2024-10-24 15:51:09 +02:00
Vlad Lazar
0ca177cbd6 pageserver: refactor xact records
This one is a bit less obvious than the previous ones.

I merged some of the logic that was previously in
`WalIngest::ingest_record` to `WalIngest::ingest_xact_record`.
2024-10-24 15:51:09 +02:00
Vlad Lazar
45e8424dba pageserver: refactor clog records 2024-10-24 15:51:09 +02:00
Vlad Lazar
7136d0a192 pageserver: refactor dbase records 2024-10-24 15:51:09 +02:00
Vlad Lazar
48947e042b pageserver: refactor smgr records 2024-10-24 15:51:09 +02:00
Vlad Lazar
d4fd4e2d9a pageserver: refactor neonrmgr records 2024-10-24 15:51:09 +02:00
Vlad Lazar
da39155590 pagesver: refactor heapam records 2024-10-24 15:51:09 +02:00
46 changed files with 1875 additions and 1312 deletions

15
Cargo.lock generated
View File

@@ -3749,6 +3749,7 @@ dependencies = [
"tracing",
"url",
"utils",
"wal_decoder",
"walkdir",
"workspace_hack",
]
@@ -4186,6 +4187,7 @@ dependencies = [
"regex",
"serde",
"thiserror",
"tracing",
"utils",
]
@@ -6954,6 +6956,19 @@ dependencies = [
"utils",
]
[[package]]
name = "wal_decoder"
version = "0.1.0"
dependencies = [
"anyhow",
"bytes",
"pageserver_api",
"postgres_ffi",
"serde",
"tracing",
"utils",
]
[[package]]
name = "walkdir"
version = "2.3.3"

View File

@@ -33,6 +33,7 @@ members = [
"libs/postgres_ffi/wal_craft",
"libs/vm_monitor",
"libs/walproposer",
"libs/wal_decoder",
]
[workspace.package]
@@ -238,6 +239,7 @@ tracing-utils = { version = "0.1", path = "./libs/tracing-utils/" }
utils = { version = "0.1", path = "./libs/utils/" }
vm_monitor = { version = "0.1", path = "./libs/vm_monitor/" }
walproposer = { version = "0.1", path = "./libs/walproposer/" }
wal_decoder = { version = "0.1", path = "./libs/wal_decoder" }
## Common library dependency
workspace_hack = { version = "0.1", path = "./workspace_hack/" }

View File

@@ -5,9 +5,11 @@ pub mod controller_api;
pub mod key;
pub mod keyspace;
pub mod models;
pub mod record;
pub mod reltag;
pub mod shard;
/// Public API types
pub mod upcall_api;
pub mod value;
pub mod config;

View File

@@ -0,0 +1,113 @@
//! This module defines the WAL record format used within the pageserver.
use bytes::Bytes;
use postgres_ffi::record::{describe_postgres_wal_record, MultiXactMember};
use postgres_ffi::{MultiXactId, MultiXactOffset, TimestampTz, TransactionId};
use serde::{Deserialize, Serialize};
use utils::bin_ser::DeserializeError;
/// Each update to a page is represented by a NeonWalRecord. It can be a wrapper
/// around a PostgreSQL WAL record, or a custom neon-specific "record".
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub enum NeonWalRecord {
/// Native PostgreSQL WAL record
Postgres { will_init: bool, rec: Bytes },
/// Clear bits in heap visibility map. ('flags' is bitmap of bits to clear)
ClearVisibilityMapFlags {
new_heap_blkno: Option<u32>,
old_heap_blkno: Option<u32>,
flags: u8,
},
/// Mark transaction IDs as committed on a CLOG page
ClogSetCommitted {
xids: Vec<TransactionId>,
timestamp: TimestampTz,
},
/// Mark transaction IDs as aborted on a CLOG page
ClogSetAborted { xids: Vec<TransactionId> },
/// Extend multixact offsets SLRU
MultixactOffsetCreate {
mid: MultiXactId,
moff: MultiXactOffset,
},
/// Extend multixact members SLRU.
MultixactMembersCreate {
moff: MultiXactOffset,
members: Vec<MultiXactMember>,
},
/// Update the map of AUX files, either writing or dropping an entry
AuxFile {
file_path: String,
content: Option<Bytes>,
},
/// A testing record for unit testing purposes. It supports append data to an existing image, or clear it.
#[cfg(feature = "testing")]
Test {
/// Append a string to the image.
append: String,
/// Clear the image before appending.
clear: bool,
/// Treat this record as an init record. `clear` should be set to true if this field is set
/// to true. This record does not need the history WALs to reconstruct. See [`NeonWalRecord::will_init`] and
/// its references in `timeline.rs`.
will_init: bool,
},
}
impl NeonWalRecord {
/// Does replaying this WAL record initialize the page from scratch, or does
/// it need to be applied over the previous image of the page?
pub fn will_init(&self) -> bool {
// If you change this function, you'll also need to change ValueBytes::will_init
match self {
NeonWalRecord::Postgres { will_init, rec: _ } => *will_init,
#[cfg(feature = "testing")]
NeonWalRecord::Test { will_init, .. } => *will_init,
// None of the special neon record types currently initialize the page
_ => false,
}
}
#[cfg(feature = "testing")]
pub fn wal_append(s: impl AsRef<str>) -> Self {
Self::Test {
append: s.as_ref().to_string(),
clear: false,
will_init: false,
}
}
#[cfg(feature = "testing")]
pub fn wal_clear() -> Self {
Self::Test {
append: "".to_string(),
clear: true,
will_init: false,
}
}
#[cfg(feature = "testing")]
pub fn wal_init() -> Self {
Self::Test {
append: "".to_string(),
clear: true,
will_init: true,
}
}
}
/// Build a human-readable string to describe a WAL record
///
/// For debugging purposes
pub fn describe_wal_record(rec: &NeonWalRecord) -> Result<String, DeserializeError> {
match rec {
NeonWalRecord::Postgres { will_init, rec } => Ok(format!(
"will_init: {}, {}",
will_init,
describe_postgres_wal_record(rec)?
)),
_ => Ok(format!("{:?}", rec)),
}
}

View File

@@ -1,13 +1,9 @@
use crate::walrecord::NeonWalRecord;
use anyhow::Result;
//! This module defines the value type used by the storage engine.
use crate::record::NeonWalRecord;
use bytes::Bytes;
use serde::{Deserialize, Serialize};
use std::ops::AddAssign;
use std::time::Duration;
pub use pageserver_api::key::{Key, KEY_SIZE};
/// A 'value' stored for a one Key.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub enum Value {
/// An Image value contains a full copy of the value
@@ -33,17 +29,17 @@ impl Value {
}
#[derive(Debug, PartialEq)]
pub(crate) enum InvalidInput {
pub enum InvalidInput {
TooShortValue,
TooShortPostgresRecord,
}
/// We could have a ValueRef where everything is `serde(borrow)`. Before implementing that, lets
/// use this type for querying if a slice looks some particular way.
pub(crate) struct ValueBytes;
pub struct ValueBytes;
impl ValueBytes {
pub(crate) fn will_init(raw: &[u8]) -> Result<bool, InvalidInput> {
pub fn will_init(raw: &[u8]) -> Result<bool, InvalidInput> {
if raw.len() < 12 {
return Err(InvalidInput::TooShortValue);
}
@@ -79,6 +75,7 @@ impl ValueBytes {
mod test {
use super::*;
use bytes::Bytes;
use utils::bin_ser::BeSer;
macro_rules! roundtrip {
@@ -229,56 +226,3 @@ mod test {
assert!(!ValueBytes::will_init(&expected).unwrap());
}
}
///
/// Result of performing GC
///
#[derive(Default, Serialize, Debug)]
pub struct GcResult {
pub layers_total: u64,
pub layers_needed_by_cutoff: u64,
pub layers_needed_by_pitr: u64,
pub layers_needed_by_branches: u64,
pub layers_needed_by_leases: u64,
pub layers_not_updated: u64,
pub layers_removed: u64, // # of layer files removed because they have been made obsolete by newer ondisk files.
#[serde(serialize_with = "serialize_duration_as_millis")]
pub elapsed: Duration,
/// The layers which were garbage collected.
///
/// Used in `/v1/tenant/:tenant_id/timeline/:timeline_id/do_gc` to wait for the layers to be
/// dropped in tests.
#[cfg(feature = "testing")]
#[serde(skip)]
pub(crate) doomed_layers: Vec<crate::tenant::storage_layer::Layer>,
}
// helper function for `GcResult`, serializing a `Duration` as an integer number of milliseconds
fn serialize_duration_as_millis<S>(d: &Duration, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
d.as_millis().serialize(serializer)
}
impl AddAssign for GcResult {
fn add_assign(&mut self, other: Self) {
self.layers_total += other.layers_total;
self.layers_needed_by_pitr += other.layers_needed_by_pitr;
self.layers_needed_by_cutoff += other.layers_needed_by_cutoff;
self.layers_needed_by_branches += other.layers_needed_by_branches;
self.layers_needed_by_leases += other.layers_needed_by_leases;
self.layers_not_updated += other.layers_not_updated;
self.layers_removed += other.layers_removed;
self.elapsed += other.elapsed;
#[cfg(feature = "testing")]
{
let mut other = other;
self.doomed_layers.append(&mut other.doomed_layers);
}
}
}

View File

@@ -15,6 +15,7 @@ memoffset.workspace = true
thiserror.workspace = true
serde.workspace = true
utils.workspace = true
tracing.workspace = true
[dev-dependencies]
env_logger.workspace = true

View File

@@ -216,6 +216,7 @@ macro_rules! enum_pgversion {
}
pub mod pg_constants;
pub mod record;
pub mod relfile_utils;
// Export some widely used datatypes that are unlikely to change across Postgres versions

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,17 @@
[package]
name = "wal_decoder"
version = "0.1.0"
edition.workspace = true
license.workspace = true
[features]
testing = []
[dependencies]
anyhow.workspace = true
bytes.workspace = true
pageserver_api.workspace = true
postgres_ffi.workspace = true
serde.workspace = true
tracing.workspace = true
utils.workspace = true

View File

View File

@@ -0,0 +1,2 @@
pub mod decoder;
pub mod models;

View File

@@ -0,0 +1,144 @@
//! This module houses types which represent decoded PG WAL records
//! ready for the pageserver to interpret. They are higher level
//! than their counterparts in [`postgres_ffi::record`].
use bytes::Bytes;
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::record::{
XlMultiXactCreate, XlMultiXactTruncate, XlRelmapUpdate, XlReploriginDrop, XlReploriginSet, XlSmgrTruncate, XlXactParsedRecord
};
use postgres_ffi::{Oid, TransactionId};
use utils::lsn::Lsn;
pub enum HeapamRecord {
ClearVmBits(ClearVmBits),
}
pub struct ClearVmBits {
pub new_heap_blkno: Option<u32>,
pub old_heap_blkno: Option<u32>,
pub vm_rel: RelTag,
pub flags: u8,
}
pub enum NeonrmgrRecord {
ClearVmBits(ClearVmBits),
}
pub enum SmgrRecord {
Create(SmgrCreate),
Truncate(XlSmgrTruncate),
}
pub struct SmgrCreate {
pub rel: RelTag,
}
pub enum DbaseRecord {
Create(DbaseCreate),
Drop(DbaseDrop),
}
pub struct DbaseCreate {
pub db_id: Oid,
pub tablespace_id: Oid,
pub src_db_id: Oid,
pub src_tablespace_id: Oid,
}
pub struct DbaseDrop {
pub db_id: Oid,
pub tablespace_ids: Vec<Oid>,
}
pub enum ClogRecord {
ZeroPage(ClogZeroPage),
Truncate(ClogTruncate),
}
pub struct ClogZeroPage {
pub segno: u32,
pub rpageno: u32,
}
pub struct ClogTruncate {
pub pageno: u32,
pub oldest_xid: TransactionId,
pub oldest_xid_db: Oid,
}
pub enum XactRecord {
Commit(XactCommon),
Abort(XactCommon),
CommitPrepared(XactCommon),
AbortPrepared(XactCommon),
Prepare(XactPrepare),
}
pub struct XactCommon {
pub parsed: XlXactParsedRecord,
pub origin_id: u16,
// Fields below are only used for logging
pub xl_xid: TransactionId,
pub lsn: Lsn,
}
pub struct XactPrepare {
pub xl_xid: TransactionId,
pub data: Bytes,
}
pub enum MultiXactRecord {
ZeroPage(MultiXactZeroPage),
Create(XlMultiXactCreate),
Truncate(XlMultiXactTruncate),
}
pub struct MultiXactZeroPage {
pub slru_kind: SlruKind,
pub segno: u32,
pub rpageno: u32,
}
pub enum RelmapRecord {
Update(RelmapUpdate),
}
pub struct RelmapUpdate {
pub update: XlRelmapUpdate,
pub buf: Bytes,
}
pub enum XlogRecord {
Raw(RawXlogRecord),
}
pub struct RawXlogRecord {
pub info: u8,
pub lsn: Lsn,
pub buf: Bytes,
}
pub enum LogicalMessageRecord {
Put(PutLogicalMessage),
#[cfg(feature = "testing")]
Failpoint,
}
pub struct PutLogicalMessage {
pub path: String,
pub buf: Bytes,
}
pub enum StandbyRecord {
RunningXacts(StandbyRunningXacts),
}
pub struct StandbyRunningXacts {
pub oldest_running_xid: TransactionId,
}
pub enum ReploriginRecord {
Set(XlReploriginSet),
Drop(XlReploriginDrop),
}

View File

@@ -8,7 +8,7 @@ license.workspace = true
default = []
# Enables test-only APIs, incuding failpoints. In particular, enables the `fail_point!` macro,
# which adds some runtime cost to run tests on outage conditions
testing = ["fail/failpoints", "pageserver_api/testing" ]
testing = ["fail/failpoints", "pageserver_api/testing", "wal_decoder/testing"]
[dependencies]
anyhow.workspace = true
@@ -83,6 +83,7 @@ enum-map.workspace = true
enumset = { workspace = true, features = ["serde"]}
strum.workspace = true
strum_macros.workspace = true
wal_decoder.workspace = true
[target.'cfg(target_os = "linux")'.dependencies]
procfs.workspace = true
@@ -92,6 +93,7 @@ criterion.workspace = true
hex-literal.workspace = true
tokio = { workspace = true, features = ["process", "sync", "fs", "rt", "io-util", "time", "test-util"] }
indoc.workspace = true
# pageserver_api = { workspace = true, features = ["testing"] }
[[bench]]
name = "bench_layer_map"

View File

@@ -6,15 +6,15 @@ use criterion::{criterion_group, criterion_main, Criterion};
use pageserver::{
config::PageServerConf,
context::{DownloadBehavior, RequestContext},
gc_result::Value,
l0_flush::{L0FlushConfig, L0FlushGlobalState},
page_cache,
repository::Value,
task_mgr::TaskKind,
tenant::storage_layer::inmemory_layer::SerializedBatch,
tenant::storage_layer::InMemoryLayer,
virtual_file,
};
use pageserver_api::{key::Key, shard::TenantShardId};
use pageserver_api::{key::Key, shard::TenantShardId, value::Value};
use utils::{
bin_ser::BeSer,
id::{TenantId, TimelineId},

View File

@@ -1,9 +1,9 @@
use criterion::measurement::WallTime;
use pageserver::keyspace::{KeyPartitioning, KeySpace};
use pageserver::repository::Key;
use pageserver::tenant::layer_map::LayerMap;
use pageserver::tenant::storage_layer::LayerName;
use pageserver::tenant::storage_layer::PersistentLayerDesc;
use pageserver_api::key::Key;
use pageserver_api::shard::TenantShardId;
use rand::prelude::{SeedableRng, SliceRandom, StdRng};
use std::cmp::{max, min};

View File

@@ -60,7 +60,8 @@ use anyhow::Context;
use bytes::{Buf, Bytes};
use criterion::{BenchmarkId, Criterion};
use once_cell::sync::Lazy;
use pageserver::{config::PageServerConf, walrecord::NeonWalRecord, walredo::PostgresRedoManager};
use pageserver::{config::PageServerConf, walredo::PostgresRedoManager};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::{key::Key, shard::TenantShardId};
use std::{
future::Future,

View File

@@ -51,7 +51,7 @@
//!
use anyhow::{Context, Result};
use pageserver::repository::Key;
use pageserver_api::key::Key;
use std::cmp::Ordering;
use std::io::{self, BufRead};
use std::path::PathBuf;

View File

@@ -14,12 +14,12 @@ use std::ops::Range;
use std::{fs, str};
use pageserver::page_cache::{self, PAGE_SZ};
use pageserver::repository::{Key, KEY_SIZE};
use pageserver::tenant::block_io::FileBlockReader;
use pageserver::tenant::disk_btree::{DiskBtreeReader, VisitDirection};
use pageserver::tenant::storage_layer::delta_layer::{Summary, DELTA_KEY_SIZE};
use pageserver::tenant::storage_layer::range_overlaps;
use pageserver::virtual_file::{self, VirtualFile};
use pageserver_api::key::{Key, KEY_SIZE};
use utils::{bin_ser::BeSer, lsn::Lsn};

View File

@@ -12,15 +12,15 @@ use pageserver::tenant::storage_layer::{delta_layer, image_layer};
use pageserver::tenant::storage_layer::{DeltaLayer, ImageLayer};
use pageserver::tenant::{TENANTS_SEGMENT_NAME, TIMELINES_SEGMENT_NAME};
use pageserver::virtual_file::api::IoMode;
use pageserver::{page_cache, virtual_file};
use pageserver::{
repository::{Key, KEY_SIZE},
tenant::{
block_io::FileBlockReader, disk_btree::VisitDirection,
storage_layer::delta_layer::DELTA_KEY_SIZE,
},
virtual_file::VirtualFile,
};
use pageserver_api::key::{Key, KEY_SIZE};
use pageserver::{page_cache, virtual_file};
use std::fs;
use utils::bin_ser::BeSer;
use utils::id::{TenantId, TimelineId};

View File

@@ -696,7 +696,7 @@ impl DeletionQueue {
mod test {
use camino::Utf8Path;
use hex_literal::hex;
use pageserver_api::{shard::ShardIndex, upcall_api::ReAttachResponseTenant};
use pageserver_api::{key::Key, shard::ShardIndex, upcall_api::ReAttachResponseTenant};
use std::{io::ErrorKind, time::Duration};
use tracing::info;
@@ -705,7 +705,6 @@ mod test {
use crate::{
controller_upcall_client::RetryForeverError,
repository::Key,
tenant::{harness::TenantHarness, storage_layer::DeltaLayerName},
};

View File

@@ -0,0 +1,57 @@
use anyhow::Result;
use serde::Serialize;
use std::ops::AddAssign;
use std::time::Duration;
///
/// Result of performing GC
///
#[derive(Default, Serialize, Debug)]
pub struct GcResult {
pub layers_total: u64,
pub layers_needed_by_cutoff: u64,
pub layers_needed_by_pitr: u64,
pub layers_needed_by_branches: u64,
pub layers_needed_by_leases: u64,
pub layers_not_updated: u64,
pub layers_removed: u64, // # of layer files removed because they have been made obsolete by newer ondisk files.
#[serde(serialize_with = "serialize_duration_as_millis")]
pub elapsed: Duration,
/// The layers which were garbage collected.
///
/// Used in `/v1/tenant/:tenant_id/timeline/:timeline_id/do_gc` to wait for the layers to be
/// dropped in tests.
#[cfg(feature = "testing")]
#[serde(skip)]
pub(crate) doomed_layers: Vec<crate::tenant::storage_layer::Layer>,
}
// helper function for `GcResult`, serializing a `Duration` as an integer number of milliseconds
fn serialize_duration_as_millis<S>(d: &Duration, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
d.as_millis().serialize(serializer)
}
impl AddAssign for GcResult {
fn add_assign(&mut self, other: Self) {
self.layers_total += other.layers_total;
self.layers_needed_by_pitr += other.layers_needed_by_pitr;
self.layers_needed_by_cutoff += other.layers_needed_by_cutoff;
self.layers_needed_by_branches += other.layers_needed_by_branches;
self.layers_needed_by_leases += other.layers_needed_by_leases;
self.layers_not_updated += other.layers_not_updated;
self.layers_removed += other.layers_removed;
self.elapsed += other.elapsed;
#[cfg(feature = "testing")]
{
let mut other = other;
self.doomed_layers.append(&mut other.doomed_layers);
}
}
}

View File

@@ -2129,13 +2129,13 @@ async fn getpage_at_lsn_handler(
check_permission(&request, Some(tenant_shard_id.tenant_id))?;
let state = get_state(&request);
struct Key(crate::repository::Key);
struct Key(pageserver_api::key::Key);
impl std::str::FromStr for Key {
type Err = anyhow::Error;
fn from_str(s: &str) -> std::result::Result<Self, Self::Err> {
crate::repository::Key::from_hex(s).map(Key)
pageserver_api::key::Key::from_hex(s).map(Key)
}
}

View File

@@ -19,10 +19,9 @@ use crate::metrics::WAL_INGEST;
use crate::pgdatadir_mapping::*;
use crate::tenant::Timeline;
use crate::walingest::WalIngest;
use crate::walrecord::decode_wal_record;
use crate::walrecord::DecodedWALRecord;
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::pg_constants;
use postgres_ffi::record::{decode_wal_record, DecodedWALRecord};
use postgres_ffi::relfile_utils::*;
use postgres_ffi::waldecoder::WalStreamDecoder;
use postgres_ffi::ControlFileData;
@@ -456,6 +455,7 @@ pub async fn import_wal_from_tar(
if let Some((lsn, recdata)) = waldecoder.poll_decode()? {
let mut decoded = DecodedWALRecord::default();
decode_wal_record(recdata, &mut decoded, tline.pg_version)?;
// let (ephemeral_file_ready_buf, special_records) = decode_wal_record(recdata, tline.pg_version);
walingest
.ingest_record(decoded, lsn, &mut modification, ctx)
.await?;

View File

@@ -20,11 +20,11 @@ pub use pageserver_api::keyspace;
use tokio_util::sync::CancellationToken;
mod assert_u64_eq_usize;
pub mod aux_file;
pub mod gc_result;
pub mod metrics;
pub mod page_cache;
pub mod page_service;
pub mod pgdatadir_mapping;
pub mod repository;
pub mod span;
pub(crate) mod statvfs;
pub mod task_mgr;
@@ -32,7 +32,6 @@ pub mod tenant;
pub mod utilization;
pub mod virtual_file;
pub mod walingest;
pub mod walrecord;
pub mod walredo;
use camino::Utf8Path;

View File

@@ -7,11 +7,10 @@
//! Clarify that)
//!
use super::tenant::{PageReconstructError, Timeline};
use crate::aux_file;
use crate::context::RequestContext;
use crate::keyspace::{KeySpace, KeySpaceAccum};
use crate::span::debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id;
use crate::walrecord::NeonWalRecord;
use crate::{aux_file, repository::*};
use anyhow::{ensure, Context};
use bytes::{Buf, Bytes, BytesMut};
use enum_map::Enum;
@@ -22,7 +21,10 @@ use pageserver_api::key::{
CompactKey, AUX_FILES_KEY, CHECKPOINT_KEY, CONTROLFILE_KEY, DBDIR_KEY, TWOPHASEDIR_KEY,
};
use pageserver_api::keyspace::SparseKeySpace;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::{BlockNumber, RelTag, SlruKind};
use pageserver_api::value::Value;
use pageserver_api::key::Key;
use postgres_ffi::relfile_utils::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi::BLCKSZ;
use postgres_ffi::{Oid, RepOriginId, TimestampTz, TransactionId};

View File

@@ -84,6 +84,7 @@ use crate::config::PageServerConf;
use crate::context::{DownloadBehavior, RequestContext};
use crate::deletion_queue::DeletionQueueClient;
use crate::deletion_queue::DeletionQueueError;
use crate::gc_result::GcResult;
use crate::import_datadir;
use crate::is_uninit_mark;
use crate::l0_flush::L0FlushGlobalState;
@@ -92,7 +93,6 @@ use crate::metrics::{
remove_tenant_metrics, BROKEN_TENANTS_SET, CIRCUIT_BREAKERS_BROKEN, CIRCUIT_BREAKERS_UNBROKEN,
TENANT_STATE_METRIC, TENANT_SYNTHETIC_SIZE_METRIC,
};
use crate::repository::GcResult;
use crate::task_mgr;
use crate::task_mgr::TaskKind;
use crate::tenant::config::LocationMode;
@@ -467,10 +467,10 @@ impl WalRedoManager {
/// This method is cancellation-safe.
pub async fn request_redo(
&self,
key: crate::repository::Key,
key: pageserver_api::key::Key,
lsn: Lsn,
base_img: Option<(Lsn, bytes::Bytes)>,
records: Vec<(Lsn, crate::walrecord::NeonWalRecord)>,
records: Vec<(Lsn, pageserver_api::record::NeonWalRecord)>,
pg_version: u32,
) -> Result<bytes::Bytes, walredo::Error> {
match self {
@@ -4528,7 +4528,8 @@ pub(crate) mod harness {
use crate::deletion_queue::mock::MockDeletionQueue;
use crate::l0_flush::L0FlushConfig;
use crate::walredo::apply_neon;
use crate::{repository::Key, walrecord::NeonWalRecord};
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use super::*;
use hex_literal::hex;
@@ -4798,17 +4799,17 @@ mod tests {
use super::*;
use crate::keyspace::KeySpaceAccum;
use crate::repository::{Key, Value};
use crate::tenant::harness::*;
use crate::tenant::timeline::CompactFlags;
use crate::walrecord::NeonWalRecord;
use crate::DEFAULT_PG_VERSION;
use bytes::{Bytes, BytesMut};
use hex_literal::hex;
use itertools::Itertools;
use pageserver_api::key::{AUX_KEY_PREFIX, NON_INHERITED_RANGE};
use pageserver_api::key::{Key, AUX_KEY_PREFIX, NON_INHERITED_RANGE};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::{CompactionAlgorithm, CompactionAlgorithmSettings};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use rand::{thread_rng, Rng};
use storage_layer::PersistentLayerKey;
use tests::storage_layer::ValuesReconstructState;

View File

@@ -48,9 +48,9 @@ mod layer_coverage;
use crate::context::RequestContext;
use crate::keyspace::KeyPartitioning;
use crate::repository::Key;
use crate::tenant::storage_layer::InMemoryLayer;
use anyhow::Result;
use pageserver_api::key::Key;
use pageserver_api::keyspace::{KeySpace, KeySpaceAccum};
use range_set_blaze::{CheckSortedDisjoint, RangeSetBlaze};
use std::collections::{HashMap, VecDeque};

View File

@@ -2811,7 +2811,7 @@ where
}
use {
crate::repository::GcResult, pageserver_api::models::TimelineGcRequest,
crate::gc_result::GcResult, pageserver_api::models::TimelineGcRequest,
utils::http::error::ApiError,
};

View File

@@ -11,11 +11,11 @@ pub mod merge_iterator;
pub mod split_writer;
use crate::context::{AccessStatsBehavior, RequestContext};
use crate::repository::Value;
use crate::walrecord::NeonWalRecord;
use bytes::Bytes;
use pageserver_api::key::Key;
use pageserver_api::keyspace::{KeySpace, KeySpaceRandomAccum};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use std::cmp::{Ordering, Reverse};
use std::collections::hash_map::Entry;
use std::collections::{BinaryHeap, HashMap};

View File

@@ -30,7 +30,6 @@
use crate::config::PageServerConf;
use crate::context::{PageContentKind, RequestContext, RequestContextBuilder};
use crate::page_cache::{self, FileId, PAGE_SZ};
use crate::repository::{Key, Value, KEY_SIZE};
use crate::tenant::blob_io::BlobWriter;
use crate::tenant::block_io::{BlockBuf, BlockCursor, BlockLease, BlockReader, FileBlockReader};
use crate::tenant::disk_btree::{
@@ -46,7 +45,7 @@ use crate::tenant::PageReconstructError;
use crate::virtual_file::owned_buffers_io::io_buf_ext::{FullSlice, IoBufExt};
use crate::virtual_file::IoBufferMut;
use crate::virtual_file::{self, MaybeFatalIo, VirtualFile};
use crate::{walrecord, TEMP_FILE_SUFFIX};
use crate::TEMP_FILE_SUFFIX;
use crate::{DELTA_FILE_MAGIC, STORAGE_FORMAT_VERSION};
use anyhow::{anyhow, bail, ensure, Context, Result};
use camino::{Utf8Path, Utf8PathBuf};
@@ -54,9 +53,11 @@ use futures::StreamExt;
use itertools::Itertools;
use pageserver_api::config::MaxVectoredReadBytes;
use pageserver_api::key::DBDIR_KEY;
use pageserver_api::key::{Key, KEY_SIZE};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::ImageCompressionAlgorithm;
use pageserver_api::shard::TenantShardId;
use pageserver_api::value::Value;
use rand::{distributions::Alphanumeric, Rng};
use serde::{Deserialize, Serialize};
use std::collections::VecDeque;
@@ -1293,7 +1294,7 @@ impl DeltaLayerInner {
// is it an image or will_init walrecord?
// FIXME: this could be handled by threading the BlobRef to the
// VectoredReadBuilder
let will_init = crate::repository::ValueBytes::will_init(&data)
let will_init = pageserver_api::value::ValueBytes::will_init(&data)
.inspect_err(|_e| {
#[cfg(feature = "testing")]
tracing::error!(data=?utils::Hex(&data), err=?_e, %key, %lsn, "failed to parse will_init out of serialized value");
@@ -1356,7 +1357,7 @@ impl DeltaLayerInner {
format!(" img {} bytes", img.len())
}
Value::WalRecord(rec) => {
let wal_desc = walrecord::describe_wal_record(&rec)?;
let wal_desc = pageserver_api::record::describe_wal_record(&rec)?;
format!(
" rec {} bytes will_init: {} {}",
buf.len(),
@@ -1600,7 +1601,6 @@ pub(crate) mod test {
use rand::RngCore;
use super::*;
use crate::repository::Value;
use crate::tenant::harness::TIMELINE_ID;
use crate::tenant::storage_layer::{Layer, ResidentLayer};
use crate::tenant::vectored_blob_io::StreamingVectoredReadPlanner;
@@ -1612,6 +1612,7 @@ pub(crate) mod test {
DEFAULT_PG_VERSION,
};
use bytes::Bytes;
use pageserver_api::value::Value;
/// Construct an index for a fictional delta layer and and then
/// traverse in order to plan vectored reads for a query. Finally,
@@ -1964,8 +1965,8 @@ pub(crate) mod test {
#[tokio::test]
async fn copy_delta_prefix_smoke() {
use crate::walrecord::NeonWalRecord;
use bytes::Bytes;
use pageserver_api::record::NeonWalRecord;
let h = crate::tenant::harness::TenantHarness::create("truncate_delta_smoke")
.await

View File

@@ -7,7 +7,7 @@ use pageserver_api::{
};
use utils::lsn::Lsn;
use crate::repository::Value;
use pageserver_api::value::Value;
use super::merge_iterator::MergeIterator;
@@ -121,8 +121,8 @@ mod tests {
#[tokio::test]
async fn filter_keyspace_iterator() {
use crate::repository::Value;
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("filter_iterator_filter_keyspace_iterator")
.await

View File

@@ -28,7 +28,6 @@
use crate::config::PageServerConf;
use crate::context::{PageContentKind, RequestContext, RequestContextBuilder};
use crate::page_cache::{self, FileId, PAGE_SZ};
use crate::repository::{Key, Value, KEY_SIZE};
use crate::tenant::blob_io::BlobWriter;
use crate::tenant::block_io::{BlockBuf, FileBlockReader};
use crate::tenant::disk_btree::{
@@ -51,8 +50,10 @@ use hex;
use itertools::Itertools;
use pageserver_api::config::MaxVectoredReadBytes;
use pageserver_api::key::DBDIR_KEY;
use pageserver_api::key::{Key, KEY_SIZE};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::shard::{ShardIdentity, TenantShardId};
use pageserver_api::value::Value;
use rand::{distributions::Alphanumeric, Rng};
use serde::{Deserialize, Serialize};
use std::collections::VecDeque;
@@ -1109,6 +1110,7 @@ mod test {
use itertools::Itertools;
use pageserver_api::{
key::Key,
value::Value,
shard::{ShardCount, ShardIdentity, ShardNumber, ShardStripeSize},
};
use utils::{
@@ -1119,7 +1121,6 @@ mod test {
use crate::{
context::RequestContext,
repository::Value,
tenant::{
config::TenantConf,
harness::{TenantHarness, TIMELINE_ID},

View File

@@ -7,7 +7,6 @@
use crate::assert_u64_eq_usize::{u64_to_usize, U64IsUsize, UsizeIsU64};
use crate::config::PageServerConf;
use crate::context::{PageContentKind, RequestContext, RequestContextBuilder};
use crate::repository::{Key, Value};
use crate::tenant::ephemeral_file::EphemeralFile;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::PageReconstructError;
@@ -16,9 +15,11 @@ use crate::{l0_flush, page_cache};
use anyhow::{anyhow, Context, Result};
use camino::Utf8PathBuf;
use pageserver_api::key::CompactKey;
use pageserver_api::key::Key;
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::InMemoryLayerInfo;
use pageserver_api::shard::TenantShardId;
use pageserver_api::value::Value;
use std::collections::{BTreeMap, HashMap};
use std::sync::{Arc, OnceLock};
use std::time::Instant;

View File

@@ -760,8 +760,8 @@ async fn evict_and_wait_does_not_wait_for_download() {
/// Also checks that the same does not happen on a non-evicted layer (regression test).
#[tokio::test(start_paused = true)]
async fn eviction_cancellation_on_drop() {
use crate::repository::Value;
use bytes::Bytes;
use pageserver_api::value::Value;
// this is the runtime on which Layer spawns the blocking tasks on
let handle = tokio::runtime::Handle::current();
@@ -782,7 +782,7 @@ async fn eviction_cancellation_on_drop() {
let mut writer = timeline.writer().await;
writer
.put(
crate::repository::Key::from_i128(5),
pageserver_api::key::Key::from_i128(5),
Lsn(0x20),
&Value::Image(Bytes::from_static(b"this does not matter either")),
&ctx,

View File

@@ -3,7 +3,7 @@ use pageserver_api::shard::TenantShardId;
use std::ops::Range;
use utils::{id::TimelineId, lsn::Lsn};
use crate::repository::Key;
use pageserver_api::key::Key;
use super::{DeltaLayerName, ImageLayerName, LayerName};

View File

@@ -1,7 +1,7 @@
//!
//! Helper functions for dealing with filenames of the image and delta layer files.
//!
use crate::repository::Key;
use pageserver_api::key::Key;
use std::borrow::Cow;
use std::cmp::Ordering;
use std::fmt;

View File

@@ -7,7 +7,8 @@ use anyhow::bail;
use pageserver_api::key::Key;
use utils::lsn::Lsn;
use crate::{context::RequestContext, repository::Value};
use crate::context::RequestContext;
use pageserver_api::value::Value;
use super::{
delta_layer::{DeltaLayerInner, DeltaLayerIterator},
@@ -293,9 +294,9 @@ mod tests {
harness::{TenantHarness, TIMELINE_ID},
storage_layer::delta_layer::test::{produce_delta_layer, sort_delta, sort_delta_value},
},
walrecord::NeonWalRecord,
DEFAULT_PG_VERSION,
};
use pageserver_api::record::NeonWalRecord;
async fn assert_merge_iter_equal(
merge_iter: &mut MergeIterator<'_>,
@@ -319,8 +320,8 @@ mod tests {
#[tokio::test]
async fn merge_in_between() {
use crate::repository::Value;
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_merge_in_between")
.await
@@ -384,8 +385,8 @@ mod tests {
#[tokio::test]
async fn delta_merge() {
use crate::repository::Value;
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_delta_merge")
.await
@@ -460,8 +461,8 @@ mod tests {
#[tokio::test]
async fn delta_image_mixed_merge() {
use crate::repository::Value;
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_delta_image_mixed_merge")
.await

View File

@@ -5,7 +5,8 @@ use pageserver_api::key::{Key, KEY_SIZE};
use utils::{id::TimelineId, lsn::Lsn, shard::TenantShardId};
use crate::tenant::storage_layer::Layer;
use crate::{config::PageServerConf, context::RequestContext, repository::Value, tenant::Timeline};
use crate::{config::PageServerConf, context::RequestContext, tenant::Timeline};
use pageserver_api::value::Value;
use super::layer::S3_UPLOAD_LIMIT;
use super::{

View File

@@ -125,11 +125,12 @@ use utils::{
simple_rcu::{Rcu, RcuReadGuard},
};
use crate::repository::GcResult;
use crate::repository::{Key, Value};
use crate::gc_result::GcResult;
use crate::task_mgr;
use crate::task_mgr::TaskKind;
use crate::ZERO_PAGE;
use pageserver_api::key::Key;
use pageserver_api::value::Value;
use self::delete::DeleteTimelineFlow;
pub(super) use self::eviction_task::EvictionTaskTenantState;
@@ -5816,17 +5817,15 @@ fn is_send() {
#[cfg(test)]
mod tests {
use pageserver_api::key::Key;
use pageserver_api::value::Value;
use utils::{id::TimelineId, lsn::Lsn};
use crate::{
repository::Value,
tenant::{
harness::{test_img, TenantHarness},
layer_map::LayerMap,
storage_layer::{Layer, LayerName},
timeline::{DeltaLayerTestDesc, EvictionError},
Timeline,
},
use crate::tenant::{
harness::{test_img, TenantHarness},
layer_map::LayerMap,
storage_layer::{Layer, LayerName},
timeline::{DeltaLayerTestDesc, EvictionError},
Timeline,
};
#[tokio::test]

View File

@@ -49,9 +49,10 @@ use pageserver_api::config::tenant_conf_defaults::{
DEFAULT_CHECKPOINT_DISTANCE, DEFAULT_COMPACTION_THRESHOLD,
};
use crate::keyspace::KeySpace;
use crate::repository::{Key, Value};
use crate::walrecord::NeonWalRecord;
use pageserver_api::key::Key;
use pageserver_api::keyspace::KeySpace;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use utils::lsn::Lsn;
@@ -2143,7 +2144,7 @@ struct ResidentDeltaLayer(ResidentLayer);
struct ResidentImageLayer(ResidentLayer);
impl CompactionJobExecutor for TimelineAdaptor {
type Key = crate::repository::Key;
type Key = pageserver_api::key::Key;
type Layer = OwnArc<PersistentLayerDesc>;
type DeltaLayer = ResidentDeltaLayer;

View File

@@ -31,10 +31,10 @@ use crate::{
task_mgr::{TaskKind, WALRECEIVER_RUNTIME},
tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline, WalReceiverInfo},
walingest::WalIngest,
walrecord::{decode_wal_record, DecodedWALRecord},
};
use postgres_backend::is_expected_io_error;
use postgres_connection::PgConnectionConfig;
use postgres_ffi::record::{decode_wal_record, DecodedWALRecord};
use postgres_ffi::waldecoder::WalStreamDecoder;
use utils::{id::NodeId, lsn::Lsn};
use utils::{pageserver_feedback::PageserverFeedback, sync::gate::GateError};
@@ -343,6 +343,7 @@ pub(super) async fn handle_walreceiver_connection(
let mut decoded = DecodedWALRecord::default();
decode_wal_record(recdata, &mut decoded, modification.tline.pg_version)?;
// TODO: Handle this. Probably flush buf + data modifications early.
if decoded.is_dbase_create_copy(timeline.pg_version)
&& uncommitted_records > 0
{

File diff suppressed because it is too large Load Diff

View File

@@ -29,11 +29,11 @@ use crate::metrics::{
WAL_REDO_BYTES_HISTOGRAM, WAL_REDO_PROCESS_LAUNCH_DURATION_HISTOGRAM,
WAL_REDO_RECORDS_HISTOGRAM, WAL_REDO_TIME,
};
use crate::repository::Key;
use crate::walrecord::NeonWalRecord;
use anyhow::Context;
use bytes::{Bytes, BytesMut};
use pageserver_api::key::Key;
use pageserver_api::models::{WalRedoManagerProcessStatus, WalRedoManagerStatus};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::TenantShardId;
use std::future::Future;
use std::sync::Arc;
@@ -548,9 +548,10 @@ impl PostgresRedoManager {
#[cfg(test)]
mod tests {
use super::PostgresRedoManager;
use crate::repository::Key;
use crate::{config::PageServerConf, walrecord::NeonWalRecord};
use crate::config::PageServerConf;
use bytes::Bytes;
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::TenantShardId;
use std::str::FromStr;
use tracing::Instrument;

View File

@@ -1,8 +1,8 @@
use crate::walrecord::NeonWalRecord;
use anyhow::Context;
use byteorder::{ByteOrder, LittleEndian};
use bytes::BytesMut;
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::SlruKind;
use postgres_ffi::pg_constants;
use postgres_ffi::relfile_utils::VISIBILITYMAP_FORKNUM;
@@ -238,7 +238,7 @@ pub(crate) fn apply_in_neon(
// No-op: this record will never be created in aux v2.
warn!("AuxFile record should not be created in aux v2");
}
#[cfg(test)]
#[cfg(feature = "testing")]
NeonWalRecord::Test {
append,
clear,

View File

@@ -8,10 +8,10 @@ use crate::{
metrics::{WalRedoKillCause, WAL_REDO_PROCESS_COUNTERS, WAL_REDO_RECORD_COUNTER},
page_cache::PAGE_SZ,
span::debug_assert_current_span_has_tenant_id,
walrecord::NeonWalRecord,
};
use anyhow::Context;
use bytes::Bytes;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::{reltag::RelTag, shard::TenantShardId};
use postgres_ffi::BLCKSZ;
#[cfg(feature = "testing")]

View File

@@ -435,7 +435,9 @@ def test_emergency_relocate_with_branches_slow_replay(
# This fail point will pause the WAL ingestion on the main branch, after the
# the first insert
pageserver_http.configure_failpoints([("wal-ingest-logical-message-sleep", "return(5000)")])
pageserver_http.configure_failpoints(
[("pageserver-wal-ingest-logical-message-sleep", "return(5000)")]
)
# Attach and wait a few seconds to give it time to load the tenants, attach to the
# safekeepers, and to stream and ingest the WAL up to the pause-point.
@@ -453,11 +455,13 @@ def test_emergency_relocate_with_branches_slow_replay(
assert cur.fetchall() == [("before pause",), ("after pause",)]
# Sanity check that the failpoint was reached
env.pageserver.assert_log_contains('failpoint "wal-ingest-logical-message-sleep": sleep done')
env.pageserver.assert_log_contains(
'failpoint "pageserver-wal-ingest-logical-message-sleep": sleep done'
)
assert time.time() - before_attach_time > 5
# Clean up
pageserver_http.configure_failpoints(("wal-ingest-logical-message-sleep", "off"))
pageserver_http.configure_failpoints(("pageserver-wal-ingest-logical-message-sleep", "off"))
# Simulate hard crash of pageserver and re-attach a tenant with a branch
@@ -581,7 +585,9 @@ def test_emergency_relocate_with_branches_createdb(
# bug reproduced easily even without this, as there is always some delay between
# loading the timeline and establishing the connection to the safekeeper to stream and
# ingest the WAL, but let's make this less dependent on accidental timing.
pageserver_http.configure_failpoints([("wal-ingest-logical-message-sleep", "return(5000)")])
pageserver_http.configure_failpoints(
[("pageserver-wal-ingest-logical-message-sleep", "return(5000)")]
)
before_attach_time = time.time()
env.pageserver.tenant_attach(tenant_id)
@@ -590,8 +596,10 @@ def test_emergency_relocate_with_branches_createdb(
assert query_scalar(cur, "SELECT count(*) FROM test_migrate_one") == 200
# Sanity check that the failpoint was reached
env.pageserver.assert_log_contains('failpoint "wal-ingest-logical-message-sleep": sleep done')
env.pageserver.assert_log_contains(
'failpoint "pageserver-wal-ingest-logical-message-sleep": sleep done'
)
assert time.time() - before_attach_time > 5
# Clean up
pageserver_http.configure_failpoints(("wal-ingest-logical-message-sleep", "off"))
pageserver_http.configure_failpoints(("pageserver-wal-ingest-logical-message-sleep", "off"))