Skip to content

Commit

Permalink
pageserver: remove top levels of legacy read path
Browse files Browse the repository at this point in the history
  • Loading branch information
VladLazar committed Aug 5, 2024
1 parent 1934847 commit 2b6dbb6
Show file tree
Hide file tree
Showing 6 changed files with 7 additions and 558 deletions.
91 changes: 1 addition & 90 deletions pageserver/src/tenant/storage_layer/delta_layer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ use crate::tenant::block_io::{BlockBuf, BlockCursor, BlockLease, BlockReader, Fi
use crate::tenant::disk_btree::{
DiskBtreeBuilder, DiskBtreeIterator, DiskBtreeReader, VisitDirection,
};
use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState};
use crate::tenant::storage_layer::Layer;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::vectored_blob_io::{
BlobFlag, MaxVectoredReadBytes, StreamingVectoredReadPlanner, VectoredBlobReader, VectoredRead,
Expand Down Expand Up @@ -808,95 +808,6 @@ impl DeltaLayerInner {
})
}

pub(super) async fn get_value_reconstruct_data(
&self,
key: Key,
lsn_range: Range<Lsn>,
reconstruct_state: &mut ValueReconstructState,
ctx: &RequestContext,
) -> anyhow::Result<ValueReconstructResult> {
let mut need_image = true;
// Scan the page versions backwards, starting from `lsn`.
let block_reader = FileBlockReader::new(&self.file, self.file_id);
let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new(
self.index_start_blk,
self.index_root_blk,
&block_reader,
);
let search_key = DeltaKey::from_key_lsn(&key, Lsn(lsn_range.end.0 - 1));

let mut offsets: Vec<(Lsn, u64)> = Vec::new();

tree_reader
.visit(
&search_key.0,
VisitDirection::Backwards,
|key, value| {
let blob_ref = BlobRef(value);
if key[..KEY_SIZE] != search_key.0[..KEY_SIZE] {
return false;
}
let entry_lsn = DeltaKey::extract_lsn_from_buf(key);
if entry_lsn < lsn_range.start {
return false;
}
offsets.push((entry_lsn, blob_ref.pos()));

!blob_ref.will_init()
},
&RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::DeltaLayerBtreeNode)
.build(),
)
.await?;

let ctx = &RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::DeltaLayerValue)
.build();

// Ok, 'offsets' now contains the offsets of all the entries we need to read
let cursor = block_reader.block_cursor();
let mut buf = Vec::new();
for (entry_lsn, pos) in offsets {
cursor
.read_blob_into_buf(pos, &mut buf, ctx)
.await
.with_context(|| {
format!("Failed to read blob from virtual file {}", self.file.path)
})?;
let val = Value::des(&buf).with_context(|| {
format!(
"Failed to deserialize file blob from virtual file {}",
self.file.path
)
})?;
match val {
Value::Image(img) => {
reconstruct_state.img = Some((entry_lsn, img));
need_image = false;
break;
}
Value::WalRecord(rec) => {
let will_init = rec.will_init();
reconstruct_state.records.push((entry_lsn, rec));
if will_init {
// This WAL record initializes the page, so no need to go further back
need_image = false;
break;
}
}
}
}

// If an older page image is needed to reconstruct the page, let the
// caller know.
if need_image {
Ok(ValueReconstructResult::Continue)
} else {
Ok(ValueReconstructResult::Complete)
}
}

// Look up the keys in the provided keyspace and update
// the reconstruct state with whatever is found.
//
Expand Down
44 changes: 1 addition & 43 deletions pageserver/src/tenant/storage_layer/image_layer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,7 @@ use crate::tenant::block_io::{BlockBuf, BlockReader, FileBlockReader};
use crate::tenant::disk_btree::{
DiskBtreeBuilder, DiskBtreeIterator, DiskBtreeReader, VisitDirection,
};
use crate::tenant::storage_layer::{
LayerAccessStats, ValueReconstructResult, ValueReconstructState,
};
use crate::tenant::storage_layer::LayerAccessStats;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::vectored_blob_io::{
BlobFlag, MaxVectoredReadBytes, StreamingVectoredReadPlanner, VectoredBlobReader, VectoredRead,
Expand Down Expand Up @@ -429,46 +427,6 @@ impl ImageLayerInner {
})
}

pub(super) async fn get_value_reconstruct_data(
&self,
key: Key,
reconstruct_state: &mut ValueReconstructState,
ctx: &RequestContext,
) -> anyhow::Result<ValueReconstructResult> {
let block_reader = FileBlockReader::new(&self.file, self.file_id);
let tree_reader =
DiskBtreeReader::new(self.index_start_blk, self.index_root_blk, &block_reader);

let mut keybuf: [u8; KEY_SIZE] = [0u8; KEY_SIZE];
key.write_to_byte_slice(&mut keybuf);
if let Some(offset) = tree_reader
.get(
&keybuf,
&RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::ImageLayerBtreeNode)
.build(),
)
.await?
{
let blob = block_reader
.block_cursor()
.read_blob(
offset,
&RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::ImageLayerValue)
.build(),
)
.await
.with_context(|| format!("failed to read value from offset {}", offset))?;
let value = Bytes::from(blob);

reconstruct_state.img = Some((self.lsn, value));
Ok(ValueReconstructResult::Complete)
} else {
Ok(ValueReconstructResult::Missing)
}
}

// Look up the keys in the provided keyspace and update
// the reconstruct state with whatever is found.
pub(super) async fn get_values_reconstruct_data(
Expand Down
76 changes: 2 additions & 74 deletions pageserver/src/tenant/storage_layer/inmemory_layer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,11 +10,10 @@ use crate::page_cache::PAGE_SZ;
use crate::repository::{Key, Value};
use crate::tenant::block_io::{BlockCursor, BlockReader, BlockReaderRef};
use crate::tenant::ephemeral_file::EphemeralFile;
use crate::tenant::storage_layer::ValueReconstructResult;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::{PageReconstructError, Timeline};
use crate::{l0_flush, page_cache, walrecord};
use anyhow::{anyhow, ensure, Result};
use anyhow::{anyhow, Result};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::InMemoryLayerInfo;
use pageserver_api::shard::TenantShardId;
Expand All @@ -33,10 +32,7 @@ use std::sync::atomic::Ordering as AtomicOrdering;
use std::sync::atomic::{AtomicU64, AtomicUsize};
use tokio::sync::{RwLock, RwLockWriteGuard};

use super::{
DeltaLayerWriter, ResidentLayer, ValueReconstructSituation, ValueReconstructState,
ValuesReconstructState,
};
use super::{DeltaLayerWriter, ResidentLayer, ValueReconstructSituation, ValuesReconstructState};

#[derive(Debug, PartialEq, Eq, Clone, Copy, Hash)]
pub(crate) struct InMemoryLayerFileId(page_cache::FileId);
Expand All @@ -55,9 +51,6 @@ pub struct InMemoryLayer {
/// Writes are only allowed when this is `None`.
pub(crate) end_lsn: OnceLock<Lsn>,

/// Used for traversal path. Cached representation of the in-memory layer before frozen.
local_path_str: Arc<str>,

/// Used for traversal path. Cached representation of the in-memory layer after frozen.
frozen_local_path_str: OnceLock<Arc<str>>,

Expand Down Expand Up @@ -248,12 +241,6 @@ impl InMemoryLayer {
self.start_lsn..self.end_lsn_or_max()
}

pub(crate) fn local_path_str(&self) -> &Arc<str> {
self.frozen_local_path_str
.get()
.unwrap_or(&self.local_path_str)
}

/// debugging function to print out the contents of the layer
///
/// this is likely completly unused
Expand Down Expand Up @@ -303,60 +290,6 @@ impl InMemoryLayer {
Ok(())
}

/// Look up given value in the layer.
pub(crate) async fn get_value_reconstruct_data(
&self,
key: Key,
lsn_range: Range<Lsn>,
reconstruct_state: &mut ValueReconstructState,
ctx: &RequestContext,
) -> anyhow::Result<ValueReconstructResult> {
ensure!(lsn_range.start >= self.start_lsn);
let mut need_image = true;

let ctx = RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::InMemoryLayer)
.build();

let inner = self.inner.read().await;

let reader = inner.file.block_cursor();

// Scan the page versions backwards, starting from `lsn`.
if let Some(vec_map) = inner.index.get(&key) {
let slice = vec_map.slice_range(lsn_range);
for (entry_lsn, pos) in slice.iter().rev() {
let buf = reader.read_blob(*pos, &ctx).await?;
let value = Value::des(&buf)?;
match value {
Value::Image(img) => {
reconstruct_state.img = Some((*entry_lsn, img));
return Ok(ValueReconstructResult::Complete);
}
Value::WalRecord(rec) => {
let will_init = rec.will_init();
reconstruct_state.records.push((*entry_lsn, rec));
if will_init {
// This WAL record initializes the page, so no need to go further back
need_image = false;
break;
}
}
}
}
}

// release lock on 'inner'

// If an older page image is needed to reconstruct the page, let the
// caller know.
if need_image {
Ok(ValueReconstructResult::Continue)
} else {
Ok(ValueReconstructResult::Complete)
}
}

// Look up the keys in the provided keyspace and update
// the reconstruct state with whatever is found.
//
Expand Down Expand Up @@ -458,11 +391,6 @@ impl InMemoryLayer {

Ok(InMemoryLayer {
file_id: key,
local_path_str: {
let mut buf = String::new();
inmem_layer_log_display(&mut buf, timeline_id, start_lsn, Lsn::MAX).unwrap();
buf.into()
},
frozen_local_path_str: OnceLock::new(),
conf,
timeline_id,
Expand Down
71 changes: 1 addition & 70 deletions pageserver/src/tenant/storage_layer/layer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,7 @@ use super::delta_layer::{self, DeltaEntry};
use super::image_layer::{self};
use super::{
AsLayerDesc, ImageLayerWriter, LayerAccessStats, LayerAccessStatsReset, LayerName,
LayerVisibilityHint, PersistentLayerDesc, ValueReconstructResult, ValueReconstructState,
ValuesReconstructState,
LayerVisibilityHint, PersistentLayerDesc, ValuesReconstructState,
};

use utils::generation::Generation;
Expand Down Expand Up @@ -301,42 +300,6 @@ impl Layer {
self.0.delete_on_drop();
}

/// Return data needed to reconstruct given page at LSN.
///
/// It is up to the caller to collect more data from the previous layer and
/// perform WAL redo, if necessary.
///
/// # Cancellation-Safety
///
/// This method is cancellation-safe.
pub(crate) async fn get_value_reconstruct_data(
&self,
key: Key,
lsn_range: Range<Lsn>,
reconstruct_data: &mut ValueReconstructState,
ctx: &RequestContext,
) -> anyhow::Result<ValueReconstructResult> {
use anyhow::ensure;

let layer = self.0.get_or_maybe_download(true, Some(ctx)).await?;
self.0.access_stats.record_access(ctx);

if self.layer_desc().is_delta {
ensure!(lsn_range.start >= self.layer_desc().lsn_range.start);
ensure!(self.layer_desc().key_range.contains(&key));
} else {
ensure!(self.layer_desc().key_range.contains(&key));
ensure!(lsn_range.start >= self.layer_desc().image_layer_lsn());
ensure!(lsn_range.end >= self.layer_desc().image_layer_lsn());
}

layer
.get_value_reconstruct_data(key, lsn_range, reconstruct_data, &self.0, ctx)
.instrument(tracing::debug_span!("get_value_reconstruct_data", layer=%self))
.await
.with_context(|| format!("get_value_reconstruct_data for layer {self}"))
}

pub(crate) async fn get_values_reconstruct_data(
&self,
keyspace: KeySpace,
Expand Down Expand Up @@ -441,10 +404,6 @@ impl Layer {
&self.0.path
}

pub(crate) fn debug_str(&self) -> &Arc<str> {
&self.0.debug_str
}

pub(crate) fn metadata(&self) -> LayerFileMetadata {
self.0.metadata()
}
Expand Down Expand Up @@ -600,9 +559,6 @@ struct LayerInner {
/// Full path to the file; unclear if this should exist anymore.
path: Utf8PathBuf,

/// String representation of the layer, used for traversal id.
debug_str: Arc<str>,

desc: PersistentLayerDesc,

/// Timeline access is needed for remote timeline client and metrics.
Expand Down Expand Up @@ -836,9 +792,6 @@ impl LayerInner {

LayerInner {
conf,
debug_str: {
format!("timelines/{}/{}", timeline.timeline_id, desc.layer_name()).into()
},
path: local_path,
desc,
timeline: Arc::downgrade(timeline),
Expand Down Expand Up @@ -1759,28 +1712,6 @@ impl DownloadedLayer {
.map_err(|e| anyhow::anyhow!("layer load failed earlier: {e}"))
}

async fn get_value_reconstruct_data(
&self,
key: Key,
lsn_range: Range<Lsn>,
reconstruct_data: &mut ValueReconstructState,
owner: &Arc<LayerInner>,
ctx: &RequestContext,
) -> anyhow::Result<ValueReconstructResult> {
use LayerKind::*;

match self.get(owner, ctx).await? {
Delta(d) => {
d.get_value_reconstruct_data(key, lsn_range, reconstruct_data, ctx)
.await
}
Image(i) => {
i.get_value_reconstruct_data(key, reconstruct_data, ctx)
.await
}
}
}

async fn get_values_reconstruct_data(
&self,
keyspace: KeySpace,
Expand Down
Loading

0 comments on commit 2b6dbb6

Please sign in to comment.