From 214567bf8fafed56cd867698d9e54fafc7001b45 Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Thu, 7 Apr 2022 20:50:16 +0300 Subject: [PATCH] Use B-tree for the index in image and delta layers. We now use a page cache for those, instead of slurping the whole index into memory. Fixes https://github.com/zenithdb/zenith/issues/1356 This is a backwards-incompatible change to the storage format, so bump STORAGE_FORMAT_VERSION. --- Cargo.lock | 1 + pageserver/Cargo.toml | 1 + pageserver/src/layered_repository.rs | 10 +- pageserver/src/layered_repository/block_io.rs | 45 + .../src/layered_repository/delta_layer.rs | 290 ++- .../src/layered_repository/disk_btree.rs | 979 ++++++++ .../disk_btree_test_data.rs | 2013 +++++++++++++++++ .../src/layered_repository/image_layer.rs | 144 +- .../src/layered_repository/inmemory_layer.rs | 7 - .../src/layered_repository/storage_layer.rs | 4 - pageserver/src/lib.rs | 2 +- pageserver/src/repository.rs | 16 +- 12 files changed, 3287 insertions(+), 225 deletions(-) create mode 100644 pageserver/src/layered_repository/disk_btree.rs create mode 100644 pageserver/src/layered_repository/disk_btree_test_data.rs diff --git a/Cargo.lock b/Cargo.lock index e0b6288f63..19ccd18a10 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1499,6 +1499,7 @@ dependencies = [ "daemonize", "fail", "futures", + "hex", "hex-literal", "humantime", "hyper", diff --git a/pageserver/Cargo.toml b/pageserver/Cargo.toml index a5283cb331..4d79811bfb 100644 --- a/pageserver/Cargo.toml +++ b/pageserver/Cargo.toml @@ -10,6 +10,7 @@ regex = "1.4.5" bytes = { version = "1.0.1", features = ['serde'] } byteorder = "1.4.3" futures = "0.3.13" +hex = "0.4.3" hyper = "0.14" itertools = "0.10.3" lazy_static = "1.4.0" diff --git a/pageserver/src/layered_repository.rs b/pageserver/src/layered_repository.rs index 5adf4a89ff..d7a250f31e 100644 --- a/pageserver/src/layered_repository.rs +++ b/pageserver/src/layered_repository.rs @@ -58,6 +58,7 @@ use zenith_utils::seqwait::SeqWait; mod blob_io; pub mod block_io; mod delta_layer; +mod disk_btree; pub(crate) mod ephemeral_file; mod filename; mod image_layer; @@ -1602,15 +1603,6 @@ impl LayeredTimeline { debug!("Could not compact because no partitioning specified yet"); } - // Call unload() on all frozen layers, to release memory. - // This shouldn't be much memory, as only metadata is slurped - // into memory. - let layers = self.layers.lock().unwrap(); - for layer in layers.iter_historic_layers() { - layer.unload()?; - } - drop(layers); - Ok(()) } diff --git a/pageserver/src/layered_repository/block_io.rs b/pageserver/src/layered_repository/block_io.rs index a8992a6cb5..2eba0aa403 100644 --- a/pageserver/src/layered_repository/block_io.rs +++ b/pageserver/src/layered_repository/block_io.rs @@ -4,6 +4,7 @@ use crate::page_cache; use crate::page_cache::{ReadBufResult, PAGE_SZ}; +use bytes::Bytes; use lazy_static::lazy_static; use std::ops::{Deref, DerefMut}; use std::os::unix::fs::FileExt; @@ -172,3 +173,47 @@ where } } } + +/// +/// Trait for block-oriented output +/// +pub trait BlockWriter { + /// + /// Write a page to the underlying storage. + /// + /// 'buf' must be of size PAGE_SZ. Returns the block number the page was + /// written to. + /// + fn write_blk(&mut self, buf: Bytes) -> Result; +} + +/// +/// A simple in-memory buffer of blocks. +/// +pub struct BlockBuf { + pub blocks: Vec, +} +impl BlockWriter for BlockBuf { + fn write_blk(&mut self, buf: Bytes) -> Result { + assert!(buf.len() == PAGE_SZ); + let blknum = self.blocks.len(); + self.blocks.push(buf); + tracing::info!("buffered block {}", blknum); + Ok(blknum as u32) + } +} + +impl BlockBuf { + pub fn new() -> Self { + BlockBuf { blocks: Vec::new() } + } + + pub fn size(&self) -> u64 { + (self.blocks.len() * PAGE_SZ) as u64 + } +} +impl Default for BlockBuf { + fn default() -> Self { + Self::new() + } +} diff --git a/pageserver/src/layered_repository/delta_layer.rs b/pageserver/src/layered_repository/delta_layer.rs index 43122fd99d..dd6b5d3afa 100644 --- a/pageserver/src/layered_repository/delta_layer.rs +++ b/pageserver/src/layered_repository/delta_layer.rs @@ -7,14 +7,8 @@ //! must be page images or WAL records with the 'will_init' flag set, so that //! they can be replayed without referring to an older page version. //! -//! When a delta file needs to be accessed, we slurp the 'index' metadata -//! into memory, into the DeltaLayerInner struct. See load() and unload() functions. -//! To access a particular value, we search `index` for the given key. -//! The byte offset in the index can be used to find the value in -//! VALUES_CHAPTER. -//! -//! On disk, the delta files are stored in timelines/ directory. -//! Currently, there are no subdirectories, and each delta file is named like this: +//! The delta files are stored in timelines/ directory. Currently, +//! there are no subdirectories, and each delta file is named like this: //! //! -__- for Summary { @@ -89,6 +89,7 @@ impl From<&DeltaLayer> for Summary { lsn_range: layer.lsn_range.clone(), index_start_blk: 0, + index_root_blk: 0, } } } @@ -123,6 +124,46 @@ impl BlobRef { } } +const DELTA_KEY_SIZE: usize = KEY_SIZE + 8; +struct DeltaKey([u8; DELTA_KEY_SIZE]); + +/// +/// This is the key of the B-tree index stored in the delta layer. It consists +/// of the serialized representation of a Key and LSN. +/// +impl DeltaKey { + fn from_slice(buf: &[u8]) -> Self { + let mut bytes: [u8; DELTA_KEY_SIZE] = [0u8; DELTA_KEY_SIZE]; + bytes.copy_from_slice(buf); + DeltaKey(bytes) + } + + fn from_key_lsn(key: &Key, lsn: Lsn) -> Self { + let mut bytes: [u8; DELTA_KEY_SIZE] = [0u8; DELTA_KEY_SIZE]; + key.write_to_byte_slice(&mut bytes[0..KEY_SIZE]); + bytes[KEY_SIZE..].copy_from_slice(&u64::to_be_bytes(lsn.0)); + DeltaKey(bytes) + } + + fn key(&self) -> Key { + Key::from_slice(&self.0) + } + + fn lsn(&self) -> Lsn { + Lsn(u64::from_be_bytes(self.0[KEY_SIZE..].try_into().unwrap())) + } + + fn extract_key_from_buf(buf: &[u8]) -> Key { + Key::from_slice(&buf[..KEY_SIZE]) + } + + fn extract_lsn_from_buf(buf: &[u8]) -> Lsn { + let mut lsn_buf = [0u8; 8]; + lsn_buf.copy_from_slice(&buf[KEY_SIZE..]); + Lsn(u64::from_be_bytes(lsn_buf)) + } +} + /// /// DeltaLayer is the in-memory data structure associated with an /// on-disk delta file. We keep a DeltaLayer in memory for each @@ -143,18 +184,12 @@ pub struct DeltaLayer { } pub struct DeltaLayerInner { - /// If false, the 'index' has not been loaded into memory yet. + /// If false, the fields below have not been loaded into memory yet. loaded: bool, - /// - /// All versions of all pages in the layer are kept here. - /// Indexed by block number and LSN. The value is an offset into the - /// chapter where the page version is stored. - /// - index: HashMap>, - // values copied from summary index_start_blk: u32, + index_root_blk: u32, /// Reader object for reading blocks from the file. (None if not loaded yet) file: Option>, @@ -196,27 +231,46 @@ impl Layer for DeltaLayer { let inner = self.load()?; // Scan the page versions backwards, starting from `lsn`. - if let Some(vec_map) = inner.index.get(&key) { - let mut reader = inner.file.as_ref().unwrap().block_cursor(); - let slice = vec_map.slice_range(lsn_range); - for (entry_lsn, blob_ref) in slice.iter().rev() { - let buf = reader.read_blob(blob_ref.pos())?; - let val = Value::des(&buf)?; - match val { - Value::Image(img) => { - reconstruct_state.img = Some((*entry_lsn, img)); + let file = inner.file.as_ref().unwrap(); + let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( + inner.index_start_blk, + inner.index_root_blk, + file, + ); + 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); + offsets.push((entry_lsn, blob_ref.pos())); + + !blob_ref.will_init() + })?; + + // Ok, 'offsets' now contains the offsets of all the entries we need to read + let mut cursor = file.block_cursor(); + for (entry_lsn, pos) in offsets { + let buf = cursor.read_blob(pos)?; + let val = Value::des(&buf)?; + 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; } - 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; - } - } } } } @@ -241,36 +295,6 @@ impl Layer for DeltaLayer { } } - /// - /// Release most of the memory used by this layer. If it's accessed again later, - /// it will need to be loaded back. - /// - fn unload(&self) -> Result<()> { - // FIXME: In debug mode, loading and unloading the index slows - // things down so much that you get timeout errors. At least - // with the test_parallel_copy test. So as an even more ad hoc - // stopgap fix for that, only unload every on average 10 - // checkpoint cycles. - use rand::RngCore; - if rand::thread_rng().next_u32() > (u32::MAX / 10) { - return Ok(()); - } - - let mut inner = match self.inner.try_write() { - Ok(inner) => inner, - Err(TryLockError::WouldBlock) => return Ok(()), - Err(TryLockError::Poisoned(_)) => panic!("DeltaLayer lock was poisoned"), - }; - inner.index = HashMap::default(); - inner.loaded = false; - - // Note: we keep the Book open. Is that a good idea? The virtual file - // machinery has its own rules for closing the file descriptor if it's not - // needed, but the Book struct uses up some memory, too. - - Ok(()) - } - fn delete(&self) -> Result<()> { // delete underlying file fs::remove_file(self.path())?; @@ -303,21 +327,36 @@ impl Layer for DeltaLayer { let inner = self.load()?; - let mut values: Vec<(&Key, &VecMap)> = inner.index.iter().collect(); - values.sort_by_key(|k| k.0); + println!( + "index_start_blk: {}, root {}", + inner.index_start_blk, inner.index_root_blk + ); - let mut reader = inner.file.as_ref().unwrap().block_cursor(); + let file = inner.file.as_ref().unwrap(); + let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( + inner.index_start_blk, + inner.index_root_blk, + file, + ); + + tree_reader.dump()?; + + let mut cursor = file.block_cursor(); + tree_reader.visit( + &[0u8; DELTA_KEY_SIZE], + VisitDirection::Forwards, + |delta_key, val| { + let blob_ref = BlobRef(val); + let key = DeltaKey::extract_key_from_buf(delta_key); + let lsn = DeltaKey::extract_lsn_from_buf(delta_key); - for (key, versions) in values { - for (lsn, blob_ref) in versions.as_slice() { let mut desc = String::new(); - match reader.read_blob(blob_ref.pos()) { + match cursor.read_blob(blob_ref.pos()) { Ok(buf) => { let val = Value::des(&buf); - match val { Ok(Value::Image(img)) => { - write!(&mut desc, " img {} bytes", img.len())?; + write!(&mut desc, " img {} bytes", img.len()).unwrap(); } Ok(Value::WalRecord(rec)) => { let wal_desc = walrecord::describe_wal_record(&rec); @@ -327,20 +366,22 @@ impl Layer for DeltaLayer { buf.len(), rec.will_init(), wal_desc - )?; + ) + .unwrap(); } Err(err) => { - write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?; + write!(&mut desc, " DESERIALIZATION ERROR: {}", err).unwrap(); } } } Err(err) => { - write!(&mut desc, " READ ERROR: {}", err)?; + write!(&mut desc, " READ ERROR: {}", err).unwrap(); } } println!(" key {} at {}: {}", key, lsn, desc); - } - } + true + }, + )?; Ok(()) } @@ -409,6 +450,7 @@ impl DeltaLayer { PathOrConf::Conf(_) => { let mut expected_summary = Summary::from(self); expected_summary.index_start_blk = actual_summary.index_start_blk; + expected_summary.index_root_blk = actual_summary.index_root_blk; if actual_summary != expected_summary { bail!("in-file summary does not match expected summary. actual = {:?} expected = {:?}", actual_summary, expected_summary); } @@ -427,17 +469,11 @@ impl DeltaLayer { } } - file.file.seek(SeekFrom::Start( - actual_summary.index_start_blk as u64 * PAGE_SZ as u64, - ))?; - let mut buf_reader = std::io::BufReader::new(&mut file.file); - let index = HashMap::des_from(&mut buf_reader)?; - inner.index_start_blk = actual_summary.index_start_blk; + inner.index_root_blk = actual_summary.index_root_blk; debug!("loaded from {}", &path.display()); - inner.index = index; inner.loaded = true; Ok(()) } @@ -457,9 +493,9 @@ impl DeltaLayer { lsn_range: filename.lsn_range.clone(), inner: RwLock::new(DeltaLayerInner { loaded: false, - index: HashMap::default(), file: None, index_start_blk: 0, + index_root_blk: 0, }), } } @@ -485,8 +521,8 @@ impl DeltaLayer { inner: RwLock::new(DeltaLayerInner { loaded: false, file: None, - index: HashMap::default(), index_start_blk: 0, + index_root_blk: 0, }), }) } @@ -529,7 +565,7 @@ pub struct DeltaLayerWriter { key_start: Key, lsn_range: Range, - index: HashMap>, + tree: DiskBtreeBuilder, blob_writer: WriteBlobWriter>, } @@ -558,10 +594,15 @@ impl DeltaLayerWriter { u64::from(lsn_range.end) )); let mut file = VirtualFile::create(&path)?; + // make room for the header block file.seek(SeekFrom::Start(PAGE_SZ as u64))?; let buf_writer = BufWriter::new(file); let blob_writer = WriteBlobWriter::new(buf_writer, PAGE_SZ as u64); + // Initialize the b-tree index builder + let block_buf = BlockBuf::new(); + let tree_builder = DiskBtreeBuilder::new(block_buf); + Ok(DeltaLayerWriter { conf, path, @@ -569,7 +610,7 @@ impl DeltaLayerWriter { tenantid, key_start, lsn_range, - index: HashMap::new(), + tree: tree_builder, blob_writer, }) } @@ -584,23 +625,16 @@ impl DeltaLayerWriter { let off = self.blob_writer.write_blob(&Value::ser(&val)?)?; - let vec_map = self.index.entry(key).or_default(); let blob_ref = BlobRef::new(off, val.will_init()); - let old = vec_map.append_or_update_last(lsn, blob_ref).unwrap().0; - if old.is_some() { - // We already had an entry for this LSN. That's odd.. - bail!( - "Value for {} at {} already exists in delta layer being built", - key, - lsn - ); - } + + let delta_key = DeltaKey::from_key_lsn(&key, lsn); + self.tree.append(&delta_key.0, blob_ref.0)?; Ok(()) } pub fn size(&self) -> u64 { - self.blob_writer.size() + self.blob_writer.size() + self.tree.borrow_writer().size() } /// @@ -614,9 +648,11 @@ impl DeltaLayerWriter { let mut file = buf_writer.into_inner()?; // Write out the index - let buf = HashMap::ser(&self.index)?; + let (index_root_blk, block_buf) = self.tree.finish()?; file.seek(SeekFrom::Start(index_start_blk as u64 * PAGE_SZ as u64))?; - file.write_all(&buf)?; + for buf in block_buf.blocks { + file.write_all(buf.as_ref())?; + } // Fill in the summary on blk 0 let summary = Summary { @@ -627,6 +663,7 @@ impl DeltaLayerWriter { key_range: self.key_start..key_end, lsn_range: self.lsn_range.clone(), index_start_blk, + index_root_blk, }; file.seek(SeekFrom::Start(0))?; Summary::ser_into(&summary, &mut file)?; @@ -642,9 +679,9 @@ impl DeltaLayerWriter { lsn_range: self.lsn_range.clone(), inner: RwLock::new(DeltaLayerInner { loaded: false, - index: HashMap::new(), file: None, index_start_blk, + index_root_blk, }), }; @@ -677,7 +714,7 @@ impl DeltaLayerWriter { /// fashion. /// struct DeltaValueIter<'a> { - all_offsets: Vec<(Key, Lsn, BlobRef)>, + all_offsets: Vec<(DeltaKey, BlobRef)>, next_idx: usize, reader: BlockCursor>, } @@ -702,15 +739,22 @@ impl<'a> Iterator for DeltaValueIter<'a> { impl<'a> DeltaValueIter<'a> { fn new(inner: RwLockReadGuard<'a, DeltaLayerInner>) -> Result { - let mut index: Vec<(&Key, &VecMap)> = inner.index.iter().collect(); - index.sort_by_key(|x| x.0); + let file = inner.file.as_ref().unwrap(); + let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( + inner.index_start_blk, + inner.index_root_blk, + file, + ); - let mut all_offsets: Vec<(Key, Lsn, BlobRef)> = Vec::new(); - for (key, vec_map) in index.iter() { - for (lsn, blob_ref) in vec_map.as_slice().iter() { - all_offsets.push((**key, *lsn, *blob_ref)); - } - } + let mut all_offsets: Vec<(DeltaKey, BlobRef)> = Vec::new(); + tree_reader.visit( + &[0u8; DELTA_KEY_SIZE], + VisitDirection::Forwards, + |key, value| { + all_offsets.push((DeltaKey::from_slice(key), BlobRef(value))); + true + }, + )?; let iter = DeltaValueIter { all_offsets, @@ -723,13 +767,15 @@ impl<'a> DeltaValueIter<'a> { fn next_res(&mut self) -> Result> { if self.next_idx < self.all_offsets.len() { - let (key, lsn, off) = &self.all_offsets[self.next_idx]; + let (delta_key, blob_ref) = &self.all_offsets[self.next_idx]; - //let mut reader = BlobReader::new(self.inner.file.as_ref().unwrap()); - let buf = self.reader.read_blob(off.pos())?; + let key = delta_key.key(); + let lsn = delta_key.lsn(); + + let buf = self.reader.read_blob(blob_ref.pos())?; let val = Value::des(&buf)?; self.next_idx += 1; - Ok(Some((*key, *lsn, val))) + Ok(Some((key, lsn, val))) } else { Ok(None) } diff --git a/pageserver/src/layered_repository/disk_btree.rs b/pageserver/src/layered_repository/disk_btree.rs new file mode 100644 index 0000000000..7a9fe6f2b7 --- /dev/null +++ b/pageserver/src/layered_repository/disk_btree.rs @@ -0,0 +1,979 @@ +//! +//! Simple on-disk B-tree implementation +//! +//! This is used as the index structure within image and delta layers +//! +//! Features: +//! - Fixed-width keys +//! - Fixed-width values (VALUE_SZ) +//! - The tree is created in a bulk operation. Insert/deletion after creation +//! is not suppported +//! - page-oriented +//! +//! TODO: +//! - better errors (e.g. with thiserror?) +//! - maybe something like an Adaptive Radix Tree would be more efficient? +//! - the values stored by image and delta layers are offsets into the file, +//! and they are in monotonically increasing order. Prefix compression would +//! be very useful for them, too. +//! - An Iterator interface would be more convenient for the callers than the +//! 'visit' function +//! +use anyhow; +use byteorder::{ReadBytesExt, BE}; +use bytes::{BufMut, Bytes, BytesMut}; +use hex; +use std::cmp::Ordering; + +use crate::layered_repository::block_io::{BlockReader, BlockWriter}; + +// The maximum size of a value stored in the B-tree. 5 bytes is enough currently. +pub const VALUE_SZ: usize = 5; +pub const MAX_VALUE: u64 = 0x007f_ffff_ffff; + +#[allow(dead_code)] +pub const PAGE_SZ: usize = 8192; + +#[derive(Clone, Copy, Debug)] +struct Value([u8; VALUE_SZ]); + +impl Value { + fn from_slice(slice: &[u8]) -> Value { + let mut b = [0u8; VALUE_SZ]; + b.copy_from_slice(slice); + Value(b) + } + + fn from_u64(x: u64) -> Value { + assert!(x <= 0x007f_ffff_ffff); + Value([ + (x >> 32) as u8, + (x >> 24) as u8, + (x >> 16) as u8, + (x >> 8) as u8, + x as u8, + ]) + } + + fn from_blknum(x: u32) -> Value { + Value([ + 0x80, + (x >> 24) as u8, + (x >> 16) as u8, + (x >> 8) as u8, + x as u8, + ]) + } + + #[allow(dead_code)] + fn is_offset(self) -> bool { + self.0[0] & 0x80 != 0 + } + + fn to_u64(self) -> u64 { + let b = &self.0; + (b[0] as u64) << 32 + | (b[1] as u64) << 24 + | (b[2] as u64) << 16 + | (b[3] as u64) << 8 + | b[4] as u64 + } + + fn to_blknum(self) -> u32 { + let b = &self.0; + assert!(b[0] == 0x80); + (b[1] as u32) << 24 | (b[2] as u32) << 16 | (b[3] as u32) << 8 | b[4] as u32 + } +} + +/// This is the on-disk representation. +struct OnDiskNode<'a, const L: usize> { + // Fixed-width fields + num_children: u16, + level: u8, + prefix_len: u8, + suffix_len: u8, + + // Variable-length fields. These are stored on-disk after the fixed-width + // fields, in this order. In the in-memory representation, these point to + // the right parts in the page buffer. + prefix: &'a [u8], + keys: &'a [u8], + values: &'a [u8], +} + +impl<'a, const L: usize> OnDiskNode<'a, L> { + /// + /// Interpret a PAGE_SZ page as a node. + /// + fn deparse(buf: &[u8]) -> OnDiskNode { + let mut cursor = std::io::Cursor::new(buf); + let num_children = cursor.read_u16::().unwrap(); + let level = cursor.read_u8().unwrap(); + let prefix_len = cursor.read_u8().unwrap(); + let suffix_len = cursor.read_u8().unwrap(); + + let mut off = cursor.position(); + let prefix_off = off as usize; + off += prefix_len as u64; + + let keys_off = off as usize; + let keys_len = num_children as usize * suffix_len as usize; + off += keys_len as u64; + + let values_off = off as usize; + let values_len = num_children as usize * VALUE_SZ as usize; + //off += values_len as u64; + + let prefix = &buf[prefix_off..prefix_off + prefix_len as usize]; + let keys = &buf[keys_off..keys_off + keys_len]; + let values = &buf[values_off..values_off + values_len]; + + OnDiskNode { + num_children, + level, + prefix_len, + suffix_len, + prefix, + keys, + values, + } + } + + /// + /// Read a value at 'idx' + /// + fn value(&self, idx: usize) -> Value { + let value_off = idx * VALUE_SZ; + let value_slice = &self.values[value_off..value_off + VALUE_SZ]; + Value::from_slice(value_slice) + } + + fn binary_search(&self, search_key: &[u8; L], keybuf: &mut [u8]) -> Result { + let mut size = self.num_children as usize; + let mut low = 0; + let mut high = size; + while low < high { + let mid = low + size / 2; + + let key_off = mid as usize * self.suffix_len as usize; + let suffix = &self.keys[key_off..key_off + self.suffix_len as usize]; + // Does this match? + keybuf[self.prefix_len as usize..].copy_from_slice(suffix); + + let cmp = keybuf[..].cmp(search_key); + + if cmp == Ordering::Less { + low = mid + 1; + } else if cmp == Ordering::Greater { + high = mid; + } else { + return Ok(mid); + } + size = high - low; + } + Err(low) + } +} + +/// +/// Public reader object, to search the tree. +/// +pub struct DiskBtreeReader +where + R: BlockReader, +{ + start_blk: u32, + root_blk: u32, + reader: R, +} + +#[derive(Clone, Copy, Debug, PartialEq)] +pub enum VisitDirection { + Forwards, + Backwards, +} + +impl DiskBtreeReader +where + R: BlockReader, +{ + pub fn new(start_blk: u32, root_blk: u32, reader: R) -> Self { + DiskBtreeReader { + start_blk, + root_blk, + reader, + } + } + + /// + /// Read the value for given key. Returns the value, or None if it doesn't exist. + /// + pub fn get(&self, search_key: &[u8; L]) -> anyhow::Result> { + let mut result: Option = None; + self.visit(search_key, VisitDirection::Forwards, |key, value| { + if key == search_key { + result = Some(value); + } + false + })?; + Ok(result) + } + + /// + /// Scan the tree, starting from 'search_key', in the given direction. 'visitor' + /// will be called for every key >= 'search_key' (or <= 'search_key', if scanning + /// backwards) + /// + pub fn visit( + &self, + search_key: &[u8; L], + dir: VisitDirection, + mut visitor: V, + ) -> anyhow::Result + where + V: FnMut(&[u8], u64) -> bool, + { + self.search_recurse(self.root_blk, search_key, dir, &mut visitor) + } + + fn search_recurse( + &self, + node_blknum: u32, + search_key: &[u8; L], + dir: VisitDirection, + visitor: &mut V, + ) -> anyhow::Result + where + V: FnMut(&[u8], u64) -> bool, + { + // Locate the node. + let blk = self.reader.read_blk(self.start_blk + node_blknum)?; + + // Search all entries on this node + self.search_node(blk.as_ref(), search_key, dir, visitor) + } + + fn search_node( + &self, + node_buf: &[u8], + search_key: &[u8; L], + dir: VisitDirection, + visitor: &mut V, + ) -> anyhow::Result + where + V: FnMut(&[u8], u64) -> bool, + { + let node = OnDiskNode::deparse(node_buf); + let prefix_len = node.prefix_len as usize; + let suffix_len = node.suffix_len as usize; + + assert!(node.num_children > 0); + + let mut keybuf = Vec::new(); + keybuf.extend(node.prefix); + keybuf.resize(prefix_len + suffix_len, 0); + + if dir == VisitDirection::Forwards { + // Locate the first match + let mut idx = match node.binary_search(search_key, keybuf.as_mut_slice()) { + Ok(idx) => idx, + Err(idx) => { + if node.level == 0 { + // Imagine that the node contains the following keys: + // + // 1 + // 3 <-- idx + // 5 + // + // If the search key is '2' and there is exact match, + // the binary search would return the index of key + // '3'. That's cool, '3' is the first key to return. + idx + } else { + // This is an internal page, so each key represents a lower + // bound for what's in the child page. If there is no exact + // match, we have to return the *previous* entry. + // + // 1 <-- return this + // 3 <-- idx + // 5 + idx.saturating_sub(1) + } + } + }; + // idx points to the first match now. Keep going from there + let mut key_off = idx * suffix_len; + while idx < node.num_children as usize { + let suffix = &node.keys[key_off..key_off + suffix_len]; + keybuf[prefix_len..].copy_from_slice(suffix); + let value = node.value(idx as usize); + #[allow(clippy::collapsible_if)] + if node.level == 0 { + // leaf + if !visitor(&keybuf, value.to_u64()) { + return Ok(false); + } + } else { + #[allow(clippy::collapsible_if)] + if !self.search_recurse(value.to_blknum(), search_key, dir, visitor)? { + return Ok(false); + } + } + idx += 1; + key_off += suffix_len; + } + } else { + let mut idx = match node.binary_search(search_key, keybuf.as_mut_slice()) { + Ok(idx) => { + // Exact match. That's the first entry to return, and walk + // backwards from there. (The loop below starts from 'idx - + // 1', so add one here to compensate.) + idx + 1 + } + Err(idx) => { + // No exact match. The binary search returned the index of the + // first key that's > search_key. Back off by one, and walk + // backwards from there. (The loop below starts from idx - 1, + // so we don't need to subtract one here) + idx + } + }; + + // idx points to the first match + 1 now. Keep going from there. + let mut key_off = idx * suffix_len; + while idx > 0 { + idx -= 1; + key_off -= suffix_len; + let suffix = &node.keys[key_off..key_off + suffix_len]; + keybuf[prefix_len..].copy_from_slice(suffix); + let value = node.value(idx as usize); + #[allow(clippy::collapsible_if)] + if node.level == 0 { + // leaf + if !visitor(&keybuf, value.to_u64()) { + return Ok(false); + } + } else { + #[allow(clippy::collapsible_if)] + if !self.search_recurse(value.to_blknum(), search_key, dir, visitor)? { + return Ok(false); + } + } + if idx == 0 { + break; + } + } + } + Ok(true) + } + + #[allow(dead_code)] + pub fn dump(&self) -> anyhow::Result<()> { + self.dump_recurse(self.root_blk, &[], 0) + } + + fn dump_recurse(&self, blknum: u32, path: &[u8], depth: usize) -> anyhow::Result<()> { + let blk = self.reader.read_blk(self.start_blk + blknum)?; + let buf: &[u8] = blk.as_ref(); + + let node = OnDiskNode::::deparse(buf); + + print!("{:indent$}", "", indent = depth * 2); + println!( + "blk #{}: path {}: prefix {}, suffix_len {}", + blknum, + hex::encode(path), + hex::encode(node.prefix), + node.suffix_len + ); + + let mut idx = 0; + let mut key_off = 0; + while idx < node.num_children { + let key = &node.keys[key_off..key_off + node.suffix_len as usize]; + let val = node.value(idx as usize); + print!("{:indent$}", "", indent = depth * 2 + 2); + println!("{}: {}", hex::encode(key), hex::encode(val.0)); + + if node.level > 0 { + let child_path = [path, node.prefix].concat(); + self.dump_recurse(val.to_blknum(), &child_path, depth + 1)?; + } + idx += 1; + key_off += node.suffix_len as usize; + } + Ok(()) + } +} + +/// +/// Public builder object, for creating a new tree. +/// +/// Usage: Create a builder object by calling 'new', load all the data into the +/// tree by calling 'append' for each key-value pair, and then call 'finish' +/// +/// 'L' is the key length in bytes +pub struct DiskBtreeBuilder +where + W: BlockWriter, +{ + writer: W, + + /// + /// stack[0] is the current root page, stack.last() is the leaf. + /// + stack: Vec>, + + /// Last key that was appended to the tree. Used to sanity check that append + /// is called in increasing key order. + last_key: Option<[u8; L]>, +} + +impl DiskBtreeBuilder +where + W: BlockWriter, +{ + pub fn new(writer: W) -> Self { + DiskBtreeBuilder { + writer, + last_key: None, + stack: vec![BuildNode::new(0)], + } + } + + pub fn append(&mut self, key: &[u8; L], value: u64) -> Result<(), anyhow::Error> { + assert!(value <= MAX_VALUE); + if let Some(last_key) = &self.last_key { + assert!(key > last_key, "unsorted input"); + } + self.last_key = Some(*key); + + Ok(self.append_internal(key, Value::from_u64(value))?) + } + + fn append_internal(&mut self, key: &[u8; L], value: Value) -> Result<(), std::io::Error> { + // Try to append to the current leaf buffer + let last = self.stack.last_mut().unwrap(); + let level = last.level; + if last.push(key, value) { + return Ok(()); + } + + // It did not fit. Try to compress, and it it succeeds to make some room + // on the node, try appending to it again. + #[allow(clippy::collapsible_if)] + if last.compress() { + if last.push(key, value) { + return Ok(()); + } + } + + // Could not append to the current leaf. Flush it and create a new one. + self.flush_node()?; + + // Replace the node we flushed with an empty one and append the new + // key to it. + let mut last = BuildNode::new(level); + if !last.push(key, value) { + panic!("could not push to new leaf node"); + } + self.stack.push(last); + + Ok(()) + } + + fn flush_node(&mut self) -> Result<(), std::io::Error> { + let last = self.stack.pop().unwrap(); + let buf = last.pack(); + let downlink_key = last.first_key(); + let downlink_ptr = self.writer.write_blk(buf)?; + + // Append the downlink to the parent + if self.stack.is_empty() { + self.stack.push(BuildNode::new(last.level + 1)); + } + self.append_internal(&downlink_key, Value::from_blknum(downlink_ptr))?; + + Ok(()) + } + + /// + /// Flushes everything to disk, and returns the block number of the root page. + /// The caller must store the root block number "out-of-band", and pass it + /// to the DiskBtreeReader::new() when you want to read the tree again. + /// (In the image and delta layers, it is stored in the beginning of the file, + /// in the summary header) + /// + pub fn finish(mut self) -> Result<(u32, W), std::io::Error> { + // flush all levels, except the root. + while self.stack.len() > 1 { + self.flush_node()?; + } + + let root = self.stack.first().unwrap(); + let buf = root.pack(); + let root_blknum = self.writer.write_blk(buf)?; + + Ok((root_blknum, self.writer)) + } + + pub fn borrow_writer(&self) -> &W { + &self.writer + } +} + +/// +/// BuildNode represesnts an incomplete page that we are appending to. +/// +#[derive(Clone, Debug)] +struct BuildNode { + num_children: u16, + level: u8, + prefix: Vec, + suffix_len: usize, + + keys: Vec, + values: Vec, + + size: usize, // physical size of this node, if it was written to disk like this +} + +const NODE_SIZE: usize = PAGE_SZ; + +const NODE_HDR_SIZE: usize = 2 + 1 + 1 + 1; + +impl BuildNode { + fn new(level: u8) -> Self { + BuildNode { + num_children: 0, + level, + prefix: Vec::new(), + suffix_len: 0, + keys: Vec::new(), + values: Vec::new(), + size: NODE_HDR_SIZE, + } + } + + /// Try to append a key-value pair to this node. Returns 'true' on + /// success, 'false' if the page was full or the key was + /// incompatible with the prefix of the existing keys. + fn push(&mut self, key: &[u8; L], value: Value) -> bool { + // If we have already performed prefix-compression on the page, + // check that the incoming key has the same prefix. + if self.num_children > 0 { + // does the prefix allow it? + if !key.starts_with(&self.prefix) { + return false; + } + } else { + self.suffix_len = key.len(); + } + + // Is the node too full? + if self.size + self.suffix_len + VALUE_SZ >= NODE_SIZE { + return false; + } + + // All clear + self.num_children += 1; + self.keys.extend(&key[self.prefix.len()..]); + self.values.extend(value.0); + + assert!(self.keys.len() == self.num_children as usize * self.suffix_len as usize); + assert!(self.values.len() == self.num_children as usize * VALUE_SZ); + + self.size += self.suffix_len + VALUE_SZ; + + true + } + + /// + /// Perform prefix-compression. + /// + /// Returns 'true' on success, 'false' if no compression was possible. + /// + fn compress(&mut self) -> bool { + let first_suffix = self.first_suffix(); + let last_suffix = self.last_suffix(); + + // Find the common prefix among all keys + let mut prefix_len = 0; + while prefix_len < self.suffix_len { + if first_suffix[prefix_len] != last_suffix[prefix_len] { + break; + } + prefix_len += 1; + } + if prefix_len == 0 { + return false; + } + + // Can compress. Rewrite the keys without the common prefix. + self.prefix.extend(&self.keys[..prefix_len]); + + let mut new_keys = Vec::new(); + let mut key_off = 0; + while key_off < self.keys.len() { + let next_key_off = key_off + self.suffix_len; + new_keys.extend(&self.keys[key_off + prefix_len..next_key_off]); + key_off = next_key_off; + } + self.keys = new_keys; + self.suffix_len -= prefix_len; + + self.size -= prefix_len * self.num_children as usize; + self.size += prefix_len; + + assert!(self.keys.len() == self.num_children as usize * self.suffix_len as usize); + assert!(self.values.len() == self.num_children as usize * VALUE_SZ); + + true + } + + /// + /// Serialize the node to on-disk format. + /// + fn pack(&self) -> Bytes { + assert!(self.keys.len() == self.num_children as usize * self.suffix_len as usize); + assert!(self.values.len() == self.num_children as usize * VALUE_SZ); + assert!(self.num_children > 0); + + let mut buf = BytesMut::new(); + + buf.put_u16(self.num_children); + buf.put_u8(self.level); + buf.put_u8(self.prefix.len() as u8); + buf.put_u8(self.suffix_len as u8); + buf.put(&self.prefix[..]); + buf.put(&self.keys[..]); + buf.put(&self.values[..]); + + assert!(buf.len() == self.size); + + assert!(buf.len() <= PAGE_SZ); + buf.resize(PAGE_SZ, 0); + buf.freeze() + } + + fn first_suffix(&self) -> &[u8] { + &self.keys[..self.suffix_len] + } + fn last_suffix(&self) -> &[u8] { + &self.keys[self.keys.len() - self.suffix_len..] + } + + /// Return the full first key of the page, including the prefix + fn first_key(&self) -> [u8; L] { + let mut key = [0u8; L]; + key[..self.prefix.len()].copy_from_slice(&self.prefix); + key[self.prefix.len()..].copy_from_slice(self.first_suffix()); + key + } +} + +#[cfg(test)] +mod tests { + use super::*; + use rand::Rng; + use std::collections::BTreeMap; + use std::sync::atomic::{AtomicUsize, Ordering}; + + #[derive(Clone, Default)] + struct TestDisk { + blocks: Vec, + } + impl TestDisk { + fn new() -> Self { + Self::default() + } + } + impl BlockReader for TestDisk { + type BlockLease = std::rc::Rc<[u8; PAGE_SZ]>; + + fn read_blk(&self, blknum: u32) -> Result { + let mut buf = [0u8; PAGE_SZ]; + buf.copy_from_slice(&self.blocks[blknum as usize]); + Ok(std::rc::Rc::new(buf)) + } + } + impl BlockWriter for &mut TestDisk { + fn write_blk(&mut self, buf: Bytes) -> Result { + let blknum = self.blocks.len(); + self.blocks.push(buf); + Ok(blknum as u32) + } + } + + #[test] + fn basic() -> anyhow::Result<()> { + let mut disk = TestDisk::new(); + let mut writer = DiskBtreeBuilder::<_, 6>::new(&mut disk); + + let all_keys: Vec<&[u8; 6]> = vec![ + b"xaaaaa", b"xaaaba", b"xaaaca", b"xabaaa", b"xababa", b"xabaca", b"xabada", b"xabadb", + ]; + let all_data: Vec<(&[u8; 6], u64)> = all_keys + .iter() + .enumerate() + .map(|(idx, key)| (*key, idx as u64)) + .collect(); + for (key, val) in all_data.iter() { + writer.append(key, *val)?; + } + + let (root_offset, _writer) = writer.finish()?; + + let reader = DiskBtreeReader::new(0, root_offset, disk); + + reader.dump()?; + + // Test the `get` function on all the keys. + for (key, val) in all_data.iter() { + assert_eq!(reader.get(key)?, Some(*val)); + } + // And on some keys that don't exist + assert_eq!(reader.get(b"aaaaaa")?, None); + assert_eq!(reader.get(b"zzzzzz")?, None); + assert_eq!(reader.get(b"xaaabx")?, None); + + // Test search with `visit` function + let search_key = b"xabaaa"; + let expected: Vec<(Vec, u64)> = all_data + .iter() + .filter(|(key, _value)| key[..] >= search_key[..]) + .map(|(key, value)| (key.to_vec(), *value)) + .collect(); + + let mut data = Vec::new(); + reader.visit(search_key, VisitDirection::Forwards, |key, value| { + data.push((key.to_vec(), value)); + true + })?; + assert_eq!(data, expected); + + // Test a backwards scan + let mut expected: Vec<(Vec, u64)> = all_data + .iter() + .filter(|(key, _value)| key[..] <= search_key[..]) + .map(|(key, value)| (key.to_vec(), *value)) + .collect(); + expected.reverse(); + let mut data = Vec::new(); + reader.visit(search_key, VisitDirection::Backwards, |key, value| { + data.push((key.to_vec(), value)); + true + })?; + assert_eq!(data, expected); + + // Backward scan where nothing matches + reader.visit(b"aaaaaa", VisitDirection::Backwards, |key, value| { + panic!("found unexpected key {}: {}", hex::encode(key), value); + })?; + + // Full scan + let expected: Vec<(Vec, u64)> = all_data + .iter() + .map(|(key, value)| (key.to_vec(), *value)) + .collect(); + let mut data = Vec::new(); + reader.visit(&[0u8; 6], VisitDirection::Forwards, |key, value| { + data.push((key.to_vec(), value)); + true + })?; + assert_eq!(data, expected); + + Ok(()) + } + + #[test] + fn lots_of_keys() -> anyhow::Result<()> { + let mut disk = TestDisk::new(); + let mut writer = DiskBtreeBuilder::<_, 8>::new(&mut disk); + + const NUM_KEYS: u64 = 1000; + + let mut all_data: BTreeMap = BTreeMap::new(); + + for idx in 0..NUM_KEYS { + let key_int: u64 = 1 + idx * 2; + let key = u64::to_be_bytes(key_int); + writer.append(&key, idx)?; + + all_data.insert(key_int, idx); + } + + let (root_offset, _writer) = writer.finish()?; + + let reader = DiskBtreeReader::new(0, root_offset, disk); + + reader.dump()?; + + use std::sync::Mutex; + + let result = Mutex::new(Vec::new()); + let limit: AtomicUsize = AtomicUsize::new(10); + let take_ten = |key: &[u8], value: u64| { + let mut keybuf = [0u8; 8]; + keybuf.copy_from_slice(key); + let key_int = u64::from_be_bytes(keybuf); + + let mut result = result.lock().unwrap(); + result.push((key_int, value)); + + // keep going until we have 10 matches + result.len() < limit.load(Ordering::Relaxed) + }; + + for search_key_int in 0..(NUM_KEYS * 2 + 10) { + let search_key = u64::to_be_bytes(search_key_int); + assert_eq!( + reader.get(&search_key)?, + all_data.get(&search_key_int).cloned() + ); + + // Test a forward scan starting with this key + result.lock().unwrap().clear(); + reader.visit(&search_key, VisitDirection::Forwards, take_ten)?; + let expected = all_data + .range(search_key_int..) + .take(10) + .map(|(&key, &val)| (key, val)) + .collect::>(); + assert_eq!(*result.lock().unwrap(), expected); + + // And a backwards scan + result.lock().unwrap().clear(); + reader.visit(&search_key, VisitDirection::Backwards, take_ten)?; + let expected = all_data + .range(..=search_key_int) + .rev() + .take(10) + .map(|(&key, &val)| (key, val)) + .collect::>(); + assert_eq!(*result.lock().unwrap(), expected); + } + + // full scan + let search_key = u64::to_be_bytes(0); + limit.store(usize::MAX, Ordering::Relaxed); + result.lock().unwrap().clear(); + reader.visit(&search_key, VisitDirection::Forwards, take_ten)?; + let expected = all_data + .iter() + .map(|(&key, &val)| (key, val)) + .collect::>(); + assert_eq!(*result.lock().unwrap(), expected); + + // full scan + let search_key = u64::to_be_bytes(u64::MAX); + limit.store(usize::MAX, Ordering::Relaxed); + result.lock().unwrap().clear(); + reader.visit(&search_key, VisitDirection::Backwards, take_ten)?; + let expected = all_data + .iter() + .rev() + .map(|(&key, &val)| (key, val)) + .collect::>(); + assert_eq!(*result.lock().unwrap(), expected); + + Ok(()) + } + + #[test] + fn random_data() -> anyhow::Result<()> { + // Generate random keys with exponential distribution, to + // exercise the prefix compression + const NUM_KEYS: usize = 100000; + let mut all_data: BTreeMap = BTreeMap::new(); + for idx in 0..NUM_KEYS { + let u: f64 = rand::thread_rng().gen_range(0.0..1.0); + let t = -(f64::ln(u)); + let key_int = (t * 1000000.0) as u128; + + all_data.insert(key_int as u128, idx as u64); + } + + // Build a tree from it + let mut disk = TestDisk::new(); + let mut writer = DiskBtreeBuilder::<_, 16>::new(&mut disk); + + for (&key, &val) in all_data.iter() { + writer.append(&u128::to_be_bytes(key), val)?; + } + let (root_offset, _writer) = writer.finish()?; + + let reader = DiskBtreeReader::new(0, root_offset, disk); + + // Test get() operation on all the keys + for (&key, &val) in all_data.iter() { + let search_key = u128::to_be_bytes(key); + assert_eq!(reader.get(&search_key)?, Some(val)); + } + + // Test get() operations on random keys, most of which will not exist + for _ in 0..100000 { + let key_int = rand::thread_rng().gen::(); + let search_key = u128::to_be_bytes(key_int); + assert!(reader.get(&search_key)? == all_data.get(&key_int).cloned()); + } + + // Test boundary cases + assert!(reader.get(&u128::to_be_bytes(u128::MIN))? == all_data.get(&u128::MIN).cloned()); + assert!(reader.get(&u128::to_be_bytes(u128::MAX))? == all_data.get(&u128::MAX).cloned()); + + Ok(()) + } + + #[test] + #[should_panic(expected = "unsorted input")] + fn unsorted_input() { + let mut disk = TestDisk::new(); + let mut writer = DiskBtreeBuilder::<_, 2>::new(&mut disk); + + let _ = writer.append(b"ba", 1); + let _ = writer.append(b"bb", 2); + let _ = writer.append(b"aa", 3); + } + + /// + /// This test contains a particular data set, see disk_btree_test_data.rs + /// + #[test] + fn particular_data() -> anyhow::Result<()> { + // Build a tree from it + let mut disk = TestDisk::new(); + let mut writer = DiskBtreeBuilder::<_, 26>::new(&mut disk); + + for (key, val) in disk_btree_test_data::TEST_DATA { + writer.append(&key, val)?; + } + let (root_offset, writer) = writer.finish()?; + + println!("SIZE: {} blocks", writer.blocks.len()); + + let reader = DiskBtreeReader::new(0, root_offset, disk); + + // Test get() operation on all the keys + for (key, val) in disk_btree_test_data::TEST_DATA { + assert_eq!(reader.get(&key)?, Some(val)); + } + + // Test full scan + let mut count = 0; + reader.visit(&[0u8; 26], VisitDirection::Forwards, |_key, _value| { + count += 1; + true + })?; + assert_eq!(count, disk_btree_test_data::TEST_DATA.len()); + + reader.dump()?; + + Ok(()) + } +} + +#[cfg(test)] +#[path = "disk_btree_test_data.rs"] +mod disk_btree_test_data; diff --git a/pageserver/src/layered_repository/disk_btree_test_data.rs b/pageserver/src/layered_repository/disk_btree_test_data.rs new file mode 100644 index 0000000000..9462573f03 --- /dev/null +++ b/pageserver/src/layered_repository/disk_btree_test_data.rs @@ -0,0 +1,2013 @@ +use hex_literal::hex; + +/// Test data set for the 'particular_data' test in disk_btree.rs +/// +/// This test contains a particular data set, representing all the keys +/// generated by the 'test_random_updates' unit test. I extracted this while +/// trying to debug a failure in that test. The bug turned out to be +/// elsewhere, and I'm not sure if this is still useful, but keeping it for +/// now... Maybe it's a useful data set to show the typical key-values used +/// by a delta layer, for evaluating how well the prefix compression works. +#[rustfmt::skip] +pub static TEST_DATA: [([u8; 26], u64); 2000] = [ + (hex!("0122222222333333334444444455000000000000000000000010"), 0x004001), + (hex!("0122222222333333334444444455000000000000000000007cb0"), 0x0040a1), + (hex!("0122222222333333334444444455000000010000000000000020"), 0x004141), + (hex!("0122222222333333334444444455000000020000000000000030"), 0x0041e1), + (hex!("01222222223333333344444444550000000200000000000051a0"), 0x004281), + (hex!("0122222222333333334444444455000000030000000000000040"), 0x004321), + (hex!("0122222222333333334444444455000000030000000000006cf0"), 0x0043c1), + (hex!("0122222222333333334444444455000000030000000000007140"), 0x004461), + (hex!("0122222222333333334444444455000000040000000000000050"), 0x004501), + (hex!("01222222223333333344444444550000000400000000000047f0"), 0x0045a1), + (hex!("01222222223333333344444444550000000400000000000072b0"), 0x004641), + (hex!("0122222222333333334444444455000000050000000000000060"), 0x0046e1), + (hex!("0122222222333333334444444455000000050000000000005550"), 0x004781), + (hex!("0122222222333333334444444455000000060000000000000070"), 0x004821), + (hex!("01222222223333333344444444550000000600000000000044a0"), 0x0048c1), + (hex!("0122222222333333334444444455000000060000000000006870"), 0x004961), + (hex!("0122222222333333334444444455000000070000000000000080"), 0x004a01), + (hex!("0122222222333333334444444455000000080000000000000090"), 0x004aa1), + (hex!("0122222222333333334444444455000000080000000000004150"), 0x004b41), + (hex!("01222222223333333344444444550000000900000000000000a0"), 0x004be1), + (hex!("01222222223333333344444444550000000a00000000000000b0"), 0x004c81), + (hex!("01222222223333333344444444550000000a0000000000006680"), 0x004d21), + (hex!("01222222223333333344444444550000000b00000000000000c0"), 0x004dc1), + (hex!("01222222223333333344444444550000000b0000000000006230"), 0x004e61), + (hex!("01222222223333333344444444550000000c00000000000000d0"), 0x004f01), + (hex!("01222222223333333344444444550000000d00000000000000e0"), 0x004fa1), + (hex!("01222222223333333344444444550000000e00000000000000f0"), 0x005041), + (hex!("01222222223333333344444444550000000e0000000000006000"), 0x0050e1), + (hex!("01222222223333333344444444550000000f0000000000000100"), 0x005181), + (hex!("01222222223333333344444444550000000f00000000000053c0"), 0x005221), + (hex!("01222222223333333344444444550000000f0000000000006580"), 0x0052c1), + (hex!("0122222222333333334444444455000000100000000000000110"), 0x005361), + (hex!("01222222223333333344444444550000001000000000000046c0"), 0x005401), + (hex!("0122222222333333334444444455000000100000000000004e40"), 0x0054a1), + (hex!("0122222222333333334444444455000000110000000000000120"), 0x005541), + (hex!("0122222222333333334444444455000000120000000000000130"), 0x0055e1), + (hex!("01222222223333333344444444550000001200000000000066d0"), 0x005681), + (hex!("0122222222333333334444444455000000130000000000000140"), 0x005721), + (hex!("0122222222333333334444444455000000130000000000007710"), 0x0057c1), + (hex!("0122222222333333334444444455000000140000000000000150"), 0x005861), + (hex!("0122222222333333334444444455000000140000000000006c40"), 0x005901), + (hex!("0122222222333333334444444455000000150000000000000160"), 0x0059a1), + (hex!("0122222222333333334444444455000000150000000000005990"), 0x005a41), + (hex!("0122222222333333334444444455000000160000000000000170"), 0x005ae1), + (hex!("0122222222333333334444444455000000160000000000005530"), 0x005b81), + (hex!("0122222222333333334444444455000000170000000000000180"), 0x005c21), + (hex!("0122222222333333334444444455000000170000000000004290"), 0x005cc1), + (hex!("0122222222333333334444444455000000180000000000000190"), 0x005d61), + (hex!("01222222223333333344444444550000001800000000000051c0"), 0x005e01), + (hex!("01222222223333333344444444550000001900000000000001a0"), 0x005ea1), + (hex!("0122222222333333334444444455000000190000000000005420"), 0x005f41), + (hex!("0122222222333333334444444455000000190000000000005770"), 0x005fe1), + (hex!("01222222223333333344444444550000001900000000000079d0"), 0x006081), + (hex!("01222222223333333344444444550000001a00000000000001b0"), 0x006121), + (hex!("01222222223333333344444444550000001a0000000000006f70"), 0x0061c1), + (hex!("01222222223333333344444444550000001a0000000000007150"), 0x006261), + (hex!("01222222223333333344444444550000001b00000000000001c0"), 0x006301), + (hex!("01222222223333333344444444550000001b0000000000005070"), 0x0063a1), + (hex!("01222222223333333344444444550000001c00000000000001d0"), 0x006441), + (hex!("01222222223333333344444444550000001d00000000000001e0"), 0x0064e1), + (hex!("01222222223333333344444444550000001e00000000000001f0"), 0x006581), + (hex!("01222222223333333344444444550000001e0000000000005650"), 0x006621), + (hex!("01222222223333333344444444550000001f0000000000000200"), 0x0066c1), + (hex!("01222222223333333344444444550000001f0000000000006ca0"), 0x006761), + (hex!("0122222222333333334444444455000000200000000000000210"), 0x006801), + (hex!("0122222222333333334444444455000000200000000000005fc0"), 0x0068a1), + (hex!("0122222222333333334444444455000000210000000000000220"), 0x006941), + (hex!("0122222222333333334444444455000000210000000000006430"), 0x0069e1), + (hex!("0122222222333333334444444455000000220000000000000230"), 0x006a81), + (hex!("01222222223333333344444444550000002200000000000040e0"), 0x006b21), + (hex!("0122222222333333334444444455000000230000000000000240"), 0x006bc1), + (hex!("01222222223333333344444444550000002300000000000042d0"), 0x006c61), + (hex!("0122222222333333334444444455000000240000000000000250"), 0x006d01), + (hex!("0122222222333333334444444455000000250000000000000260"), 0x006da1), + (hex!("01222222223333333344444444550000002500000000000058c0"), 0x006e41), + (hex!("0122222222333333334444444455000000260000000000000270"), 0x006ee1), + (hex!("0122222222333333334444444455000000260000000000004020"), 0x006f81), + (hex!("0122222222333333334444444455000000270000000000000280"), 0x007021), + (hex!("0122222222333333334444444455000000280000000000000290"), 0x0070c1), + (hex!("0122222222333333334444444455000000280000000000007c00"), 0x007161), + (hex!("01222222223333333344444444550000002900000000000002a0"), 0x007201), + (hex!("01222222223333333344444444550000002a00000000000002b0"), 0x0072a1), + (hex!("01222222223333333344444444550000002b00000000000002c0"), 0x007341), + (hex!("01222222223333333344444444550000002c00000000000002d0"), 0x0073e1), + (hex!("01222222223333333344444444550000002c00000000000041b0"), 0x007481), + (hex!("01222222223333333344444444550000002c0000000000004c30"), 0x007521), + (hex!("01222222223333333344444444550000002d00000000000002e0"), 0x0075c1), + (hex!("01222222223333333344444444550000002d0000000000005e40"), 0x007661), + (hex!("01222222223333333344444444550000002d0000000000006990"), 0x007701), + (hex!("01222222223333333344444444550000002e00000000000002f0"), 0x0077a1), + (hex!("01222222223333333344444444550000002f0000000000000300"), 0x007841), + (hex!("01222222223333333344444444550000002f0000000000004a70"), 0x0078e1), + (hex!("01222222223333333344444444550000002f0000000000006b40"), 0x007981), + (hex!("0122222222333333334444444455000000300000000000000310"), 0x007a21), + (hex!("0122222222333333334444444455000000310000000000000320"), 0x007ac1), + (hex!("0122222222333333334444444455000000320000000000000330"), 0x007b61), + (hex!("01222222223333333344444444550000003200000000000041a0"), 0x007c01), + (hex!("0122222222333333334444444455000000320000000000007340"), 0x007ca1), + (hex!("0122222222333333334444444455000000320000000000007730"), 0x007d41), + (hex!("0122222222333333334444444455000000330000000000000340"), 0x007de1), + (hex!("01222222223333333344444444550000003300000000000055a0"), 0x007e81), + (hex!("0122222222333333334444444455000000340000000000000350"), 0x007f21), + (hex!("0122222222333333334444444455000000350000000000000360"), 0x007fc1), + (hex!("01222222223333333344444444550000003500000000000077a0"), 0x008061), + (hex!("0122222222333333334444444455000000360000000000000370"), 0x008101), + (hex!("0122222222333333334444444455000000370000000000000380"), 0x0081a1), + (hex!("0122222222333333334444444455000000380000000000000390"), 0x008241), + (hex!("01222222223333333344444444550000003900000000000003a0"), 0x0082e1), + (hex!("01222222223333333344444444550000003a00000000000003b0"), 0x008381), + (hex!("01222222223333333344444444550000003a00000000000071c0"), 0x008421), + (hex!("01222222223333333344444444550000003b00000000000003c0"), 0x0084c1), + (hex!("01222222223333333344444444550000003c00000000000003d0"), 0x008561), + (hex!("01222222223333333344444444550000003d00000000000003e0"), 0x008601), + (hex!("01222222223333333344444444550000003e00000000000003f0"), 0x0086a1), + (hex!("01222222223333333344444444550000003e00000000000062e0"), 0x008741), + (hex!("01222222223333333344444444550000003f0000000000000400"), 0x0087e1), + (hex!("0122222222333333334444444455000000400000000000000410"), 0x008881), + (hex!("0122222222333333334444444455000000400000000000004460"), 0x008921), + (hex!("0122222222333333334444444455000000400000000000005b90"), 0x0089c1), + (hex!("01222222223333333344444444550000004000000000000079b0"), 0x008a61), + (hex!("0122222222333333334444444455000000410000000000000420"), 0x008b01), + (hex!("0122222222333333334444444455000000420000000000000430"), 0x008ba1), + (hex!("0122222222333333334444444455000000420000000000005640"), 0x008c41), + (hex!("0122222222333333334444444455000000430000000000000440"), 0x008ce1), + (hex!("01222222223333333344444444550000004300000000000072a0"), 0x008d81), + (hex!("0122222222333333334444444455000000440000000000000450"), 0x008e21), + (hex!("0122222222333333334444444455000000450000000000000460"), 0x008ec1), + (hex!("0122222222333333334444444455000000450000000000005750"), 0x008f61), + (hex!("01222222223333333344444444550000004500000000000077b0"), 0x009001), + (hex!("0122222222333333334444444455000000460000000000000470"), 0x0090a1), + (hex!("0122222222333333334444444455000000470000000000000480"), 0x009141), + (hex!("0122222222333333334444444455000000480000000000000490"), 0x0091e1), + (hex!("01222222223333333344444444550000004800000000000069e0"), 0x009281), + (hex!("01222222223333333344444444550000004900000000000004a0"), 0x009321), + (hex!("0122222222333333334444444455000000490000000000007370"), 0x0093c1), + (hex!("01222222223333333344444444550000004a00000000000004b0"), 0x009461), + (hex!("01222222223333333344444444550000004a0000000000005cb0"), 0x009501), + (hex!("01222222223333333344444444550000004b00000000000004c0"), 0x0095a1), + (hex!("01222222223333333344444444550000004c00000000000004d0"), 0x009641), + (hex!("01222222223333333344444444550000004c0000000000004880"), 0x0096e1), + (hex!("01222222223333333344444444550000004c0000000000007a40"), 0x009781), + (hex!("01222222223333333344444444550000004d00000000000004e0"), 0x009821), + (hex!("01222222223333333344444444550000004d0000000000006390"), 0x0098c1), + (hex!("01222222223333333344444444550000004e00000000000004f0"), 0x009961), + (hex!("01222222223333333344444444550000004e0000000000004db0"), 0x009a01), + (hex!("01222222223333333344444444550000004f0000000000000500"), 0x009aa1), + (hex!("0122222222333333334444444455000000500000000000000510"), 0x009b41), + (hex!("0122222222333333334444444455000000510000000000000520"), 0x009be1), + (hex!("01222222223333333344444444550000005100000000000069c0"), 0x009c81), + (hex!("0122222222333333334444444455000000520000000000000530"), 0x009d21), + (hex!("0122222222333333334444444455000000520000000000006e60"), 0x009dc1), + (hex!("01222222223333333344444444550000005200000000000070c0"), 0x009e61), + (hex!("0122222222333333334444444455000000530000000000000540"), 0x009f01), + (hex!("0122222222333333334444444455000000530000000000005840"), 0x009fa1), + (hex!("0122222222333333334444444455000000540000000000000550"), 0x00a041), + (hex!("01222222223333333344444444550000005400000000000043e0"), 0x00a0e1), + (hex!("01222222223333333344444444550000005400000000000074e0"), 0x00a181), + (hex!("0122222222333333334444444455000000550000000000000560"), 0x00a221), + (hex!("0122222222333333334444444455000000550000000000003ee0"), 0x00a2c1), + (hex!("0122222222333333334444444455000000560000000000000570"), 0x00a361), + (hex!("0122222222333333334444444455000000570000000000000580"), 0x00a401), + (hex!("0122222222333333334444444455000000570000000000007030"), 0x00a4a1), + (hex!("0122222222333333334444444455000000580000000000000590"), 0x00a541), + (hex!("0122222222333333334444444455000000580000000000005340"), 0x00a5e1), + (hex!("01222222223333333344444444550000005800000000000059f0"), 0x00a681), + (hex!("0122222222333333334444444455000000580000000000006930"), 0x00a721), + (hex!("01222222223333333344444444550000005900000000000005a0"), 0x00a7c1), + (hex!("0122222222333333334444444455000000590000000000003f90"), 0x00a861), + (hex!("01222222223333333344444444550000005a00000000000005b0"), 0x00a901), + (hex!("01222222223333333344444444550000005b00000000000005c0"), 0x00a9a1), + (hex!("01222222223333333344444444550000005b00000000000062c0"), 0x00aa41), + (hex!("01222222223333333344444444550000005c00000000000005d0"), 0x00aae1), + (hex!("01222222223333333344444444550000005c0000000000005a70"), 0x00ab81), + (hex!("01222222223333333344444444550000005c0000000000005dd0"), 0x00ac21), + (hex!("01222222223333333344444444550000005d00000000000005e0"), 0x00acc1), + (hex!("01222222223333333344444444550000005d0000000000005730"), 0x00ad61), + (hex!("01222222223333333344444444550000005e00000000000005f0"), 0x00ae01), + (hex!("01222222223333333344444444550000005e0000000000004f40"), 0x00aea1), + (hex!("01222222223333333344444444550000005f0000000000000600"), 0x00af41), + (hex!("0122222222333333334444444455000000600000000000000610"), 0x00afe1), + (hex!("0122222222333333334444444455000000600000000000007c40"), 0x00b081), + (hex!("0122222222333333334444444455000000610000000000000620"), 0x00b121), + (hex!("0122222222333333334444444455000000610000000000007860"), 0x00b1c1), + (hex!("0122222222333333334444444455000000620000000000000630"), 0x00b261), + (hex!("0122222222333333334444444455000000620000000000005050"), 0x00b301), + (hex!("0122222222333333334444444455000000630000000000000640"), 0x00b3a1), + (hex!("0122222222333333334444444455000000640000000000000650"), 0x00b441), + (hex!("0122222222333333334444444455000000650000000000000660"), 0x00b4e1), + (hex!("0122222222333333334444444455000000650000000000005330"), 0x00b581), + (hex!("0122222222333333334444444455000000660000000000000670"), 0x00b621), + (hex!("0122222222333333334444444455000000660000000000004e20"), 0x00b6c1), + (hex!("0122222222333333334444444455000000660000000000005ee0"), 0x00b761), + (hex!("0122222222333333334444444455000000660000000000006360"), 0x00b801), + (hex!("0122222222333333334444444455000000670000000000000680"), 0x00b8a1), + (hex!("0122222222333333334444444455000000670000000000004040"), 0x00b941), + (hex!("0122222222333333334444444455000000680000000000000690"), 0x00b9e1), + (hex!("0122222222333333334444444455000000680000000000003f80"), 0x00ba81), + (hex!("01222222223333333344444444550000006800000000000041e0"), 0x00bb21), + (hex!("01222222223333333344444444550000006900000000000006a0"), 0x00bbc1), + (hex!("0122222222333333334444444455000000690000000000006080"), 0x00bc61), + (hex!("01222222223333333344444444550000006a00000000000006b0"), 0x00bd01), + (hex!("01222222223333333344444444550000006a00000000000042f0"), 0x00bda1), + (hex!("01222222223333333344444444550000006b00000000000006c0"), 0x00be41), + (hex!("01222222223333333344444444550000006b00000000000052f0"), 0x00bee1), + (hex!("01222222223333333344444444550000006b0000000000005980"), 0x00bf81), + (hex!("01222222223333333344444444550000006b0000000000006170"), 0x00c021), + (hex!("01222222223333333344444444550000006c00000000000006d0"), 0x00c0c1), + (hex!("01222222223333333344444444550000006d00000000000006e0"), 0x00c161), + (hex!("01222222223333333344444444550000006d0000000000006fb0"), 0x00c201), + (hex!("01222222223333333344444444550000006e00000000000006f0"), 0x00c2a1), + (hex!("01222222223333333344444444550000006e00000000000065b0"), 0x00c341), + (hex!("01222222223333333344444444550000006e0000000000007970"), 0x00c3e1), + (hex!("01222222223333333344444444550000006f0000000000000700"), 0x00c481), + (hex!("01222222223333333344444444550000006f0000000000005900"), 0x00c521), + (hex!("01222222223333333344444444550000006f0000000000006d90"), 0x00c5c1), + (hex!("0122222222333333334444444455000000700000000000000710"), 0x00c661), + (hex!("01222222223333333344444444550000007000000000000045c0"), 0x00c701), + (hex!("0122222222333333334444444455000000700000000000004d40"), 0x00c7a1), + (hex!("0122222222333333334444444455000000710000000000000720"), 0x00c841), + (hex!("0122222222333333334444444455000000710000000000004dc0"), 0x00c8e1), + (hex!("0122222222333333334444444455000000710000000000007550"), 0x00c981), + (hex!("0122222222333333334444444455000000720000000000000730"), 0x00ca21), + (hex!("0122222222333333334444444455000000720000000000003ec0"), 0x00cac1), + (hex!("01222222223333333344444444550000007200000000000045a0"), 0x00cb61), + (hex!("0122222222333333334444444455000000720000000000006770"), 0x00cc01), + (hex!("0122222222333333334444444455000000720000000000006bc0"), 0x00cca1), + (hex!("0122222222333333334444444455000000730000000000000740"), 0x00cd41), + (hex!("0122222222333333334444444455000000730000000000005250"), 0x00cde1), + (hex!("01222222223333333344444444550000007300000000000075f0"), 0x00ce81), + (hex!("0122222222333333334444444455000000740000000000000750"), 0x00cf21), + (hex!("0122222222333333334444444455000000740000000000003ff0"), 0x00cfc1), + (hex!("01222222223333333344444444550000007400000000000079e0"), 0x00d061), + (hex!("0122222222333333334444444455000000750000000000000760"), 0x00d101), + (hex!("0122222222333333334444444455000000750000000000004310"), 0x00d1a1), + (hex!("0122222222333333334444444455000000760000000000000770"), 0x00d241), + (hex!("0122222222333333334444444455000000770000000000000780"), 0x00d2e1), + (hex!("01222222223333333344444444550000007700000000000062f0"), 0x00d381), + (hex!("0122222222333333334444444455000000770000000000006940"), 0x00d421), + (hex!("0122222222333333334444444455000000780000000000000790"), 0x00d4c1), + (hex!("01222222223333333344444444550000007900000000000007a0"), 0x00d561), + (hex!("0122222222333333334444444455000000790000000000007af0"), 0x00d601), + (hex!("01222222223333333344444444550000007a00000000000007b0"), 0x00d6a1), + (hex!("01222222223333333344444444550000007b00000000000007c0"), 0x00d741), + (hex!("01222222223333333344444444550000007b00000000000067e0"), 0x00d7e1), + (hex!("01222222223333333344444444550000007b0000000000007890"), 0x00d881), + (hex!("01222222223333333344444444550000007c00000000000007d0"), 0x00d921), + (hex!("01222222223333333344444444550000007d00000000000007e0"), 0x00d9c1), + (hex!("01222222223333333344444444550000007e00000000000007f0"), 0x00da61), + (hex!("01222222223333333344444444550000007f0000000000000800"), 0x00db01), + (hex!("01222222223333333344444444550000007f0000000000005be0"), 0x00dba1), + (hex!("0122222222333333334444444455000000800000000000000810"), 0x00dc41), + (hex!("0122222222333333334444444455000000810000000000000820"), 0x00dce1), + (hex!("0122222222333333334444444455000000810000000000007190"), 0x00dd81), + (hex!("0122222222333333334444444455000000820000000000000830"), 0x00de21), + (hex!("0122222222333333334444444455000000820000000000004ab0"), 0x00dec1), + (hex!("0122222222333333334444444455000000830000000000000840"), 0x00df61), + (hex!("0122222222333333334444444455000000830000000000006720"), 0x00e001), + (hex!("0122222222333333334444444455000000840000000000000850"), 0x00e0a1), + (hex!("0122222222333333334444444455000000850000000000000860"), 0x00e141), + (hex!("01222222223333333344444444550000008500000000000054f0"), 0x00e1e1), + (hex!("0122222222333333334444444455000000850000000000007920"), 0x00e281), + (hex!("0122222222333333334444444455000000860000000000000870"), 0x00e321), + (hex!("01222222223333333344444444550000008600000000000060e0"), 0x00e3c1), + (hex!("0122222222333333334444444455000000860000000000006be0"), 0x00e461), + (hex!("0122222222333333334444444455000000870000000000000880"), 0x00e501), + (hex!("0122222222333333334444444455000000870000000000006820"), 0x00e5a1), + (hex!("0122222222333333334444444455000000880000000000000890"), 0x00e641), + (hex!("01222222223333333344444444550000008900000000000008a0"), 0x00e6e1), + (hex!("0122222222333333334444444455000000890000000000007c30"), 0x00e781), + (hex!("01222222223333333344444444550000008a00000000000008b0"), 0x00e821), + (hex!("01222222223333333344444444550000008b00000000000008c0"), 0x00e8c1), + (hex!("01222222223333333344444444550000008b0000000000005910"), 0x00e961), + (hex!("01222222223333333344444444550000008b0000000000006fe0"), 0x00ea01), + (hex!("01222222223333333344444444550000008c00000000000008d0"), 0x00eaa1), + (hex!("01222222223333333344444444550000008c0000000000006800"), 0x00eb41), + (hex!("01222222223333333344444444550000008d00000000000008e0"), 0x00ebe1), + (hex!("01222222223333333344444444550000008d0000000000005810"), 0x00ec81), + (hex!("01222222223333333344444444550000008d0000000000007c90"), 0x00ed21), + (hex!("01222222223333333344444444550000008e00000000000008f0"), 0x00edc1), + (hex!("01222222223333333344444444550000008e00000000000058f0"), 0x00ee61), + (hex!("01222222223333333344444444550000008f0000000000000900"), 0x00ef01), + (hex!("01222222223333333344444444550000008f0000000000005a30"), 0x00efa1), + (hex!("0122222222333333334444444455000000900000000000000910"), 0x00f041), + (hex!("0122222222333333334444444455000000900000000000006130"), 0x00f0e1), + (hex!("0122222222333333334444444455000000900000000000006550"), 0x00f181), + (hex!("0122222222333333334444444455000000910000000000000920"), 0x00f221), + (hex!("01222222223333333344444444550000009100000000000079f0"), 0x00f2c1), + (hex!("0122222222333333334444444455000000920000000000000930"), 0x00f361), + (hex!("0122222222333333334444444455000000920000000000005620"), 0x00f401), + (hex!("0122222222333333334444444455000000920000000000005e90"), 0x00f4a1), + (hex!("01222222223333333344444444550000009200000000000063d0"), 0x00f541), + (hex!("01222222223333333344444444550000009200000000000076c0"), 0x00f5e1), + (hex!("0122222222333333334444444455000000930000000000000940"), 0x00f681), + (hex!("01222222223333333344444444550000009300000000000044e0"), 0x00f721), + (hex!("0122222222333333334444444455000000940000000000000950"), 0x00f7c1), + (hex!("0122222222333333334444444455000000940000000000007a30"), 0x00f861), + (hex!("0122222222333333334444444455000000950000000000000960"), 0x00f901), + (hex!("0122222222333333334444444455000000950000000000007a70"), 0x00f9a1), + (hex!("0122222222333333334444444455000000960000000000000970"), 0x00fa41), + (hex!("0122222222333333334444444455000000970000000000000980"), 0x00fae1), + (hex!("0122222222333333334444444455000000970000000000007330"), 0x00fb81), + (hex!("0122222222333333334444444455000000980000000000000990"), 0x00fc21), + (hex!("0122222222333333334444444455000000980000000000005af0"), 0x00fcc1), + (hex!("0122222222333333334444444455000000980000000000007ae0"), 0x00fd61), + (hex!("01222222223333333344444444550000009900000000000009a0"), 0x00fe01), + (hex!("0122222222333333334444444455000000990000000000005160"), 0x00fea1), + (hex!("0122222222333333334444444455000000990000000000006850"), 0x00ff41), + (hex!("01222222223333333344444444550000009a00000000000009b0"), 0x00ffe1), + (hex!("01222222223333333344444444550000009b00000000000009c0"), 0x010081), + (hex!("01222222223333333344444444550000009b0000000000005010"), 0x010121), + (hex!("01222222223333333344444444550000009c00000000000009d0"), 0x0101c1), + (hex!("01222222223333333344444444550000009c00000000000042e0"), 0x010261), + (hex!("01222222223333333344444444550000009d00000000000009e0"), 0x010301), + (hex!("01222222223333333344444444550000009d00000000000057f0"), 0x0103a1), + (hex!("01222222223333333344444444550000009e00000000000009f0"), 0x010441), + (hex!("01222222223333333344444444550000009e0000000000004ef0"), 0x0104e1), + (hex!("01222222223333333344444444550000009f0000000000000a00"), 0x010581), + (hex!("01222222223333333344444444550000009f0000000000006110"), 0x010621), + (hex!("0122222222333333334444444455000000a00000000000000a10"), 0x0106c1), + (hex!("0122222222333333334444444455000000a10000000000000a20"), 0x010761), + (hex!("0122222222333333334444444455000000a100000000000040d0"), 0x010801), + (hex!("0122222222333333334444444455000000a10000000000007670"), 0x0108a1), + (hex!("0122222222333333334444444455000000a20000000000000a30"), 0x010941), + (hex!("0122222222333333334444444455000000a200000000000074d0"), 0x0109e1), + (hex!("0122222222333333334444444455000000a30000000000000a40"), 0x010a81), + (hex!("0122222222333333334444444455000000a30000000000004c90"), 0x010b21), + (hex!("0122222222333333334444444455000000a40000000000000a50"), 0x010bc1), + (hex!("0122222222333333334444444455000000a50000000000000a60"), 0x010c61), + (hex!("0122222222333333334444444455000000a60000000000000a70"), 0x010d01), + (hex!("0122222222333333334444444455000000a60000000000006d80"), 0x010da1), + (hex!("0122222222333333334444444455000000a60000000000007830"), 0x010e41), + (hex!("0122222222333333334444444455000000a70000000000000a80"), 0x010ee1), + (hex!("0122222222333333334444444455000000a700000000000064f0"), 0x010f81), + (hex!("0122222222333333334444444455000000a80000000000000a90"), 0x011021), + (hex!("0122222222333333334444444455000000a90000000000000aa0"), 0x0110c1), + (hex!("0122222222333333334444444455000000a90000000000005e30"), 0x011161), + (hex!("0122222222333333334444444455000000aa0000000000000ab0"), 0x011201), + (hex!("0122222222333333334444444455000000ab0000000000000ac0"), 0x0112a1), + (hex!("0122222222333333334444444455000000ac0000000000000ad0"), 0x011341), + (hex!("0122222222333333334444444455000000ac0000000000006d20"), 0x0113e1), + (hex!("0122222222333333334444444455000000ac0000000000007000"), 0x011481), + (hex!("0122222222333333334444444455000000ad0000000000000ae0"), 0x011521), + (hex!("0122222222333333334444444455000000ae0000000000000af0"), 0x0115c1), + (hex!("0122222222333333334444444455000000ae0000000000004a10"), 0x011661), + (hex!("0122222222333333334444444455000000af0000000000000b00"), 0x011701), + (hex!("0122222222333333334444444455000000af0000000000004e10"), 0x0117a1), + (hex!("0122222222333333334444444455000000b00000000000000b10"), 0x011841), + (hex!("0122222222333333334444444455000000b00000000000004280"), 0x0118e1), + (hex!("0122222222333333334444444455000000b000000000000077e0"), 0x011981), + (hex!("0122222222333333334444444455000000b10000000000000b20"), 0x011a21), + (hex!("0122222222333333334444444455000000b20000000000000b30"), 0x011ac1), + (hex!("0122222222333333334444444455000000b30000000000000b40"), 0x011b61), + (hex!("0122222222333333334444444455000000b30000000000004bc0"), 0x011c01), + (hex!("0122222222333333334444444455000000b40000000000000b50"), 0x011ca1), + (hex!("0122222222333333334444444455000000b50000000000000b60"), 0x011d41), + (hex!("0122222222333333334444444455000000b50000000000004fa0"), 0x011de1), + (hex!("0122222222333333334444444455000000b50000000000006a60"), 0x011e81), + (hex!("0122222222333333334444444455000000b60000000000000b70"), 0x011f21), + (hex!("0122222222333333334444444455000000b60000000000005630"), 0x011fc1), + (hex!("0122222222333333334444444455000000b70000000000000b80"), 0x012061), + (hex!("0122222222333333334444444455000000b80000000000000b90"), 0x012101), + (hex!("0122222222333333334444444455000000b80000000000006f80"), 0x0121a1), + (hex!("0122222222333333334444444455000000b90000000000000ba0"), 0x012241), + (hex!("0122222222333333334444444455000000ba0000000000000bb0"), 0x0122e1), + (hex!("0122222222333333334444444455000000bb0000000000000bc0"), 0x012381), + (hex!("0122222222333333334444444455000000bb00000000000047c0"), 0x012421), + (hex!("0122222222333333334444444455000000bb0000000000006060"), 0x0124c1), + (hex!("0122222222333333334444444455000000bc0000000000000bd0"), 0x012561), + (hex!("0122222222333333334444444455000000bd0000000000000be0"), 0x012601), + (hex!("0122222222333333334444444455000000bd0000000000004e80"), 0x0126a1), + (hex!("0122222222333333334444444455000000be0000000000000bf0"), 0x012741), + (hex!("0122222222333333334444444455000000bf0000000000000c00"), 0x0127e1), + (hex!("0122222222333333334444444455000000bf00000000000047a0"), 0x012881), + (hex!("0122222222333333334444444455000000bf0000000000006da0"), 0x012921), + (hex!("0122222222333333334444444455000000c00000000000000c10"), 0x0129c1), + (hex!("0122222222333333334444444455000000c10000000000000c20"), 0x012a61), + (hex!("0122222222333333334444444455000000c20000000000000c30"), 0x012b01), + (hex!("0122222222333333334444444455000000c20000000000004bd0"), 0x012ba1), + (hex!("0122222222333333334444444455000000c20000000000006ac0"), 0x012c41), + (hex!("0122222222333333334444444455000000c30000000000000c40"), 0x012ce1), + (hex!("0122222222333333334444444455000000c30000000000004660"), 0x012d81), + (hex!("0122222222333333334444444455000000c40000000000000c50"), 0x012e21), + (hex!("0122222222333333334444444455000000c50000000000000c60"), 0x012ec1), + (hex!("0122222222333333334444444455000000c60000000000000c70"), 0x012f61), + (hex!("0122222222333333334444444455000000c60000000000005880"), 0x013001), + (hex!("0122222222333333334444444455000000c60000000000006b70"), 0x0130a1), + (hex!("0122222222333333334444444455000000c70000000000000c80"), 0x013141), + (hex!("0122222222333333334444444455000000c80000000000000c90"), 0x0131e1), + (hex!("0122222222333333334444444455000000c80000000000005310"), 0x013281), + (hex!("0122222222333333334444444455000000c80000000000005db0"), 0x013321), + (hex!("0122222222333333334444444455000000c80000000000007040"), 0x0133c1), + (hex!("0122222222333333334444444455000000c80000000000007290"), 0x013461), + (hex!("0122222222333333334444444455000000c90000000000000ca0"), 0x013501), + (hex!("0122222222333333334444444455000000c90000000000004fe0"), 0x0135a1), + (hex!("0122222222333333334444444455000000ca0000000000000cb0"), 0x013641), + (hex!("0122222222333333334444444455000000ca0000000000006140"), 0x0136e1), + (hex!("0122222222333333334444444455000000ca0000000000007700"), 0x013781), + (hex!("0122222222333333334444444455000000cb0000000000000cc0"), 0x013821), + (hex!("0122222222333333334444444455000000cc0000000000000cd0"), 0x0138c1), + (hex!("0122222222333333334444444455000000cd0000000000000ce0"), 0x013961), + (hex!("0122222222333333334444444455000000cd0000000000003f20"), 0x013a01), + (hex!("0122222222333333334444444455000000cd00000000000040f0"), 0x013aa1), + (hex!("0122222222333333334444444455000000cd0000000000004ec0"), 0x013b41), + (hex!("0122222222333333334444444455000000ce0000000000000cf0"), 0x013be1), + (hex!("0122222222333333334444444455000000ce0000000000007200"), 0x013c81), + (hex!("0122222222333333334444444455000000cf0000000000000d00"), 0x013d21), + (hex!("0122222222333333334444444455000000cf00000000000046a0"), 0x013dc1), + (hex!("0122222222333333334444444455000000cf0000000000005960"), 0x013e61), + (hex!("0122222222333333334444444455000000d00000000000000d10"), 0x013f01), + (hex!("0122222222333333334444444455000000d00000000000005f30"), 0x013fa1), + (hex!("0122222222333333334444444455000000d10000000000000d20"), 0x014041), + (hex!("0122222222333333334444444455000000d10000000000007a00"), 0x0140e1), + (hex!("0122222222333333334444444455000000d20000000000000d30"), 0x014181), + (hex!("0122222222333333334444444455000000d30000000000000d40"), 0x014221), + (hex!("0122222222333333334444444455000000d40000000000000d50"), 0x0142c1), + (hex!("0122222222333333334444444455000000d50000000000000d60"), 0x014361), + (hex!("0122222222333333334444444455000000d50000000000004960"), 0x014401), + (hex!("0122222222333333334444444455000000d500000000000055d0"), 0x0144a1), + (hex!("0122222222333333334444444455000000d500000000000067d0"), 0x014541), + (hex!("0122222222333333334444444455000000d60000000000000d70"), 0x0145e1), + (hex!("0122222222333333334444444455000000d70000000000000d80"), 0x014681), + (hex!("0122222222333333334444444455000000d80000000000000d90"), 0x014721), + (hex!("0122222222333333334444444455000000d800000000000065f0"), 0x0147c1), + (hex!("0122222222333333334444444455000000d90000000000000da0"), 0x014861), + (hex!("0122222222333333334444444455000000d90000000000004980"), 0x014901), + (hex!("0122222222333333334444444455000000da0000000000000db0"), 0x0149a1), + (hex!("0122222222333333334444444455000000da00000000000048c0"), 0x014a41), + (hex!("0122222222333333334444444455000000da00000000000072c0"), 0x014ae1), + (hex!("0122222222333333334444444455000000da00000000000076b0"), 0x014b81), + (hex!("0122222222333333334444444455000000db0000000000000dc0"), 0x014c21), + (hex!("0122222222333333334444444455000000dc0000000000000dd0"), 0x014cc1), + (hex!("0122222222333333334444444455000000dc00000000000040a0"), 0x014d61), + (hex!("0122222222333333334444444455000000dc00000000000074c0"), 0x014e01), + (hex!("0122222222333333334444444455000000dd0000000000000de0"), 0x014ea1), + (hex!("0122222222333333334444444455000000dd0000000000004e50"), 0x014f41), + (hex!("0122222222333333334444444455000000dd0000000000007270"), 0x014fe1), + (hex!("0122222222333333334444444455000000de0000000000000df0"), 0x015081), + (hex!("0122222222333333334444444455000000de00000000000078d0"), 0x015121), + (hex!("0122222222333333334444444455000000df0000000000000e00"), 0x0151c1), + (hex!("0122222222333333334444444455000000df0000000000004d30"), 0x015261), + (hex!("0122222222333333334444444455000000df0000000000006c30"), 0x015301), + (hex!("0122222222333333334444444455000000e00000000000000e10"), 0x0153a1), + (hex!("0122222222333333334444444455000000e00000000000005d30"), 0x015441), + (hex!("0122222222333333334444444455000000e10000000000000e20"), 0x0154e1), + (hex!("0122222222333333334444444455000000e10000000000004610"), 0x015581), + (hex!("0122222222333333334444444455000000e100000000000051d0"), 0x015621), + (hex!("0122222222333333334444444455000000e10000000000005f10"), 0x0156c1), + (hex!("0122222222333333334444444455000000e20000000000000e30"), 0x015761), + (hex!("0122222222333333334444444455000000e20000000000007a90"), 0x015801), + (hex!("0122222222333333334444444455000000e30000000000000e40"), 0x0158a1), + (hex!("0122222222333333334444444455000000e30000000000005ae0"), 0x015941), + (hex!("0122222222333333334444444455000000e40000000000000e50"), 0x0159e1), + (hex!("0122222222333333334444444455000000e50000000000000e60"), 0x015a81), + (hex!("0122222222333333334444444455000000e50000000000004700"), 0x015b21), + (hex!("0122222222333333334444444455000000e500000000000065d0"), 0x015bc1), + (hex!("0122222222333333334444444455000000e60000000000000e70"), 0x015c61), + (hex!("0122222222333333334444444455000000e60000000000004fd0"), 0x015d01), + (hex!("0122222222333333334444444455000000e70000000000000e80"), 0x015da1), + (hex!("0122222222333333334444444455000000e70000000000005150"), 0x015e41), + (hex!("0122222222333333334444444455000000e70000000000005920"), 0x015ee1), + (hex!("0122222222333333334444444455000000e80000000000000e90"), 0x015f81), + (hex!("0122222222333333334444444455000000e80000000000004320"), 0x016021), + (hex!("0122222222333333334444444455000000e80000000000005ec0"), 0x0160c1), + (hex!("0122222222333333334444444455000000e90000000000000ea0"), 0x016161), + (hex!("0122222222333333334444444455000000e900000000000043b0"), 0x016201), + (hex!("0122222222333333334444444455000000ea0000000000000eb0"), 0x0162a1), + (hex!("0122222222333333334444444455000000ea0000000000003ea0"), 0x016341), + (hex!("0122222222333333334444444455000000ea0000000000004f50"), 0x0163e1), + (hex!("0122222222333333334444444455000000ea0000000000007520"), 0x016481), + (hex!("0122222222333333334444444455000000eb0000000000000ec0"), 0x016521), + (hex!("0122222222333333334444444455000000ec0000000000000ed0"), 0x0165c1), + (hex!("0122222222333333334444444455000000ec0000000000006670"), 0x016661), + (hex!("0122222222333333334444444455000000ed0000000000000ee0"), 0x016701), + (hex!("0122222222333333334444444455000000ee0000000000000ef0"), 0x0167a1), + (hex!("0122222222333333334444444455000000ee0000000000004d10"), 0x016841), + (hex!("0122222222333333334444444455000000ef0000000000000f00"), 0x0168e1), + (hex!("0122222222333333334444444455000000f00000000000000f10"), 0x016981), + (hex!("0122222222333333334444444455000000f00000000000007220"), 0x016a21), + (hex!("0122222222333333334444444455000000f00000000000007540"), 0x016ac1), + (hex!("0122222222333333334444444455000000f10000000000000f20"), 0x016b61), + (hex!("0122222222333333334444444455000000f100000000000066f0"), 0x016c01), + (hex!("0122222222333333334444444455000000f20000000000000f30"), 0x016ca1), + (hex!("0122222222333333334444444455000000f20000000000007810"), 0x016d41), + (hex!("0122222222333333334444444455000000f30000000000000f40"), 0x016de1), + (hex!("0122222222333333334444444455000000f30000000000007b70"), 0x016e81), + (hex!("0122222222333333334444444455000000f40000000000000f50"), 0x016f21), + (hex!("0122222222333333334444444455000000f400000000000059c0"), 0x016fc1), + (hex!("0122222222333333334444444455000000f50000000000000f60"), 0x017061), + (hex!("0122222222333333334444444455000000f50000000000003fb0"), 0x017101), + (hex!("0122222222333333334444444455000000f50000000000005740"), 0x0171a1), + (hex!("0122222222333333334444444455000000f500000000000064d0"), 0x017241), + (hex!("0122222222333333334444444455000000f50000000000006960"), 0x0172e1), + (hex!("0122222222333333334444444455000000f60000000000000f70"), 0x017381), + (hex!("0122222222333333334444444455000000f60000000000006d00"), 0x017421), + (hex!("0122222222333333334444444455000000f70000000000000f80"), 0x0174c1), + (hex!("0122222222333333334444444455000000f80000000000000f90"), 0x017561), + (hex!("0122222222333333334444444455000000f90000000000000fa0"), 0x017601), + (hex!("0122222222333333334444444455000000fa0000000000000fb0"), 0x0176a1), + (hex!("0122222222333333334444444455000000fa00000000000067b0"), 0x017741), + (hex!("0122222222333333334444444455000000fb0000000000000fc0"), 0x0177e1), + (hex!("0122222222333333334444444455000000fb0000000000004eb0"), 0x017881), + (hex!("0122222222333333334444444455000000fb0000000000006ef0"), 0x017921), + (hex!("0122222222333333334444444455000000fc0000000000000fd0"), 0x0179c1), + (hex!("0122222222333333334444444455000000fc0000000000004470"), 0x017a61), + (hex!("0122222222333333334444444455000000fc0000000000005940"), 0x017b01), + (hex!("0122222222333333334444444455000000fd0000000000000fe0"), 0x017ba1), + (hex!("0122222222333333334444444455000000fe0000000000000ff0"), 0x017c41), + (hex!("0122222222333333334444444455000000ff0000000000001000"), 0x017ce1), + (hex!("0122222222333333334444444455000000ff0000000000005690"), 0x017d81), + (hex!("0122222222333333334444444455000001000000000000001010"), 0x017e21), + (hex!("0122222222333333334444444455000001000000000000005210"), 0x017ec1), + (hex!("01222222223333333344444444550000010000000000000070a0"), 0x017f61), + (hex!("0122222222333333334444444455000001010000000000001020"), 0x018001), + (hex!("0122222222333333334444444455000001010000000000006b80"), 0x0180a1), + (hex!("0122222222333333334444444455000001020000000000001030"), 0x018141), + (hex!("0122222222333333334444444455000001030000000000001040"), 0x0181e1), + (hex!("0122222222333333334444444455000001030000000000004c80"), 0x018281), + (hex!("0122222222333333334444444455000001040000000000001050"), 0x018321), + (hex!("0122222222333333334444444455000001040000000000004850"), 0x0183c1), + (hex!("01222222223333333344444444550000010400000000000057b0"), 0x018461), + (hex!("0122222222333333334444444455000001050000000000001060"), 0x018501), + (hex!("01222222223333333344444444550000010500000000000048d0"), 0x0185a1), + (hex!("0122222222333333334444444455000001050000000000007870"), 0x018641), + (hex!("0122222222333333334444444455000001060000000000001070"), 0x0186e1), + (hex!("0122222222333333334444444455000001060000000000004f90"), 0x018781), + (hex!("0122222222333333334444444455000001060000000000006270"), 0x018821), + (hex!("0122222222333333334444444455000001070000000000001080"), 0x0188c1), + (hex!("01222222223333333344444444550000010700000000000063b0"), 0x018961), + (hex!("0122222222333333334444444455000001080000000000001090"), 0x018a01), + (hex!("01222222223333333344444444550000010900000000000010a0"), 0x018aa1), + (hex!("0122222222333333334444444455000001090000000000006f40"), 0x018b41), + (hex!("01222222223333333344444444550000010a00000000000010b0"), 0x018be1), + (hex!("01222222223333333344444444550000010a0000000000006640"), 0x018c81), + (hex!("01222222223333333344444444550000010b00000000000010c0"), 0x018d21), + (hex!("01222222223333333344444444550000010c00000000000010d0"), 0x018dc1), + (hex!("01222222223333333344444444550000010d00000000000010e0"), 0x018e61), + (hex!("01222222223333333344444444550000010e00000000000010f0"), 0x018f01), + (hex!("01222222223333333344444444550000010e0000000000005c40"), 0x018fa1), + (hex!("01222222223333333344444444550000010e0000000000007ba0"), 0x019041), + (hex!("01222222223333333344444444550000010f0000000000001100"), 0x0190e1), + (hex!("01222222223333333344444444550000010f0000000000005c30"), 0x019181), + (hex!("0122222222333333334444444455000001100000000000001110"), 0x019221), + (hex!("0122222222333333334444444455000001100000000000007640"), 0x0192c1), + (hex!("0122222222333333334444444455000001110000000000001120"), 0x019361), + (hex!("01222222223333333344444444550000011100000000000052c0"), 0x019401), + (hex!("0122222222333333334444444455000001110000000000005710"), 0x0194a1), + (hex!("0122222222333333334444444455000001110000000000006a00"), 0x019541), + (hex!("0122222222333333334444444455000001120000000000001130"), 0x0195e1), + (hex!("0122222222333333334444444455000001130000000000001140"), 0x019681), + (hex!("0122222222333333334444444455000001140000000000001150"), 0x019721), + (hex!("0122222222333333334444444455000001140000000000003fa0"), 0x0197c1), + (hex!("01222222223333333344444444550000011400000000000054b0"), 0x019861), + (hex!("0122222222333333334444444455000001140000000000006070"), 0x019901), + (hex!("0122222222333333334444444455000001150000000000001160"), 0x0199a1), + (hex!("0122222222333333334444444455000001150000000000005320"), 0x019a41), + (hex!("0122222222333333334444444455000001150000000000006600"), 0x019ae1), + (hex!("0122222222333333334444444455000001150000000000006df0"), 0x019b81), + (hex!("01222222223333333344444444550000011500000000000079c0"), 0x019c21), + (hex!("0122222222333333334444444455000001160000000000001170"), 0x019cc1), + (hex!("0122222222333333334444444455000001170000000000001180"), 0x019d61), + (hex!("0122222222333333334444444455000001170000000000004a60"), 0x019e01), + (hex!("01222222223333333344444444550000011700000000000063c0"), 0x019ea1), + (hex!("0122222222333333334444444455000001180000000000001190"), 0x019f41), + (hex!("0122222222333333334444444455000001180000000000004530"), 0x019fe1), + (hex!("01222222223333333344444444550000011800000000000077c0"), 0x01a081), + (hex!("01222222223333333344444444550000011900000000000011a0"), 0x01a121), + (hex!("01222222223333333344444444550000011a00000000000011b0"), 0x01a1c1), + (hex!("01222222223333333344444444550000011a00000000000041c0"), 0x01a261), + (hex!("01222222223333333344444444550000011a00000000000061e0"), 0x01a301), + (hex!("01222222223333333344444444550000011b00000000000011c0"), 0x01a3a1), + (hex!("01222222223333333344444444550000011c00000000000011d0"), 0x01a441), + (hex!("01222222223333333344444444550000011c0000000000005f90"), 0x01a4e1), + (hex!("01222222223333333344444444550000011d00000000000011e0"), 0x01a581), + (hex!("01222222223333333344444444550000011d0000000000004160"), 0x01a621), + (hex!("01222222223333333344444444550000011e00000000000011f0"), 0x01a6c1), + (hex!("01222222223333333344444444550000011e00000000000056d0"), 0x01a761), + (hex!("01222222223333333344444444550000011f0000000000001200"), 0x01a801), + (hex!("01222222223333333344444444550000011f0000000000004510"), 0x01a8a1), + (hex!("0122222222333333334444444455000001200000000000001210"), 0x01a941), + (hex!("0122222222333333334444444455000001210000000000001220"), 0x01a9e1), + (hex!("0122222222333333334444444455000001210000000000005140"), 0x01aa81), + (hex!("0122222222333333334444444455000001210000000000006710"), 0x01ab21), + (hex!("0122222222333333334444444455000001210000000000006f50"), 0x01abc1), + (hex!("0122222222333333334444444455000001220000000000001230"), 0x01ac61), + (hex!("0122222222333333334444444455000001220000000000005570"), 0x01ad01), + (hex!("0122222222333333334444444455000001220000000000007ac0"), 0x01ada1), + (hex!("0122222222333333334444444455000001230000000000001240"), 0x01ae41), + (hex!("0122222222333333334444444455000001240000000000001250"), 0x01aee1), + (hex!("0122222222333333334444444455000001240000000000006cd0"), 0x01af81), + (hex!("0122222222333333334444444455000001250000000000001260"), 0x01b021), + (hex!("01222222223333333344444444550000012500000000000046b0"), 0x01b0c1), + (hex!("0122222222333333334444444455000001250000000000005eb0"), 0x01b161), + (hex!("0122222222333333334444444455000001260000000000001270"), 0x01b201), + (hex!("0122222222333333334444444455000001260000000000004630"), 0x01b2a1), + (hex!("0122222222333333334444444455000001270000000000001280"), 0x01b341), + (hex!("0122222222333333334444444455000001270000000000004ff0"), 0x01b3e1), + (hex!("0122222222333333334444444455000001270000000000006ec0"), 0x01b481), + (hex!("0122222222333333334444444455000001280000000000001290"), 0x01b521), + (hex!("01222222223333333344444444550000012900000000000012a0"), 0x01b5c1), + (hex!("0122222222333333334444444455000001290000000000005f60"), 0x01b661), + (hex!("01222222223333333344444444550000012a00000000000012b0"), 0x01b701), + (hex!("01222222223333333344444444550000012a0000000000005480"), 0x01b7a1), + (hex!("01222222223333333344444444550000012b00000000000012c0"), 0x01b841), + (hex!("01222222223333333344444444550000012b00000000000065a0"), 0x01b8e1), + (hex!("01222222223333333344444444550000012b00000000000066c0"), 0x01b981), + (hex!("01222222223333333344444444550000012c00000000000012d0"), 0x01ba21), + (hex!("01222222223333333344444444550000012c00000000000064b0"), 0x01bac1), + (hex!("01222222223333333344444444550000012d00000000000012e0"), 0x01bb61), + (hex!("01222222223333333344444444550000012d00000000000049c0"), 0x01bc01), + (hex!("01222222223333333344444444550000012d0000000000004bf0"), 0x01bca1), + (hex!("01222222223333333344444444550000012e00000000000012f0"), 0x01bd41), + (hex!("01222222223333333344444444550000012e0000000000005ed0"), 0x01bde1), + (hex!("01222222223333333344444444550000012f0000000000001300"), 0x01be81), + (hex!("01222222223333333344444444550000012f00000000000049a0"), 0x01bf21), + (hex!("0122222222333333334444444455000001300000000000001310"), 0x01bfc1), + (hex!("0122222222333333334444444455000001300000000000007840"), 0x01c061), + (hex!("0122222222333333334444444455000001310000000000001320"), 0x01c101), + (hex!("0122222222333333334444444455000001310000000000005f70"), 0x01c1a1), + (hex!("0122222222333333334444444455000001320000000000001330"), 0x01c241), + (hex!("0122222222333333334444444455000001320000000000005a00"), 0x01c2e1), + (hex!("0122222222333333334444444455000001330000000000001340"), 0x01c381), + (hex!("0122222222333333334444444455000001330000000000006c70"), 0x01c421), + (hex!("0122222222333333334444444455000001340000000000001350"), 0x01c4c1), + (hex!("0122222222333333334444444455000001340000000000005c60"), 0x01c561), + (hex!("0122222222333333334444444455000001350000000000001360"), 0x01c601), + (hex!("0122222222333333334444444455000001350000000000004f10"), 0x01c6a1), + (hex!("0122222222333333334444444455000001360000000000001370"), 0x01c741), + (hex!("0122222222333333334444444455000001360000000000004c60"), 0x01c7e1), + (hex!("0122222222333333334444444455000001370000000000001380"), 0x01c881), + (hex!("0122222222333333334444444455000001380000000000001390"), 0x01c921), + (hex!("01222222223333333344444444550000013900000000000013a0"), 0x01c9c1), + (hex!("0122222222333333334444444455000001390000000000004ea0"), 0x01ca61), + (hex!("01222222223333333344444444550000013a00000000000013b0"), 0x01cb01), + (hex!("01222222223333333344444444550000013a0000000000007350"), 0x01cba1), + (hex!("01222222223333333344444444550000013b00000000000013c0"), 0x01cc41), + (hex!("01222222223333333344444444550000013c00000000000013d0"), 0x01cce1), + (hex!("01222222223333333344444444550000013c0000000000007050"), 0x01cd81), + (hex!("01222222223333333344444444550000013d00000000000013e0"), 0x01ce21), + (hex!("01222222223333333344444444550000013d0000000000006bd0"), 0x01cec1), + (hex!("01222222223333333344444444550000013e00000000000013f0"), 0x01cf61), + (hex!("01222222223333333344444444550000013e00000000000058e0"), 0x01d001), + (hex!("01222222223333333344444444550000013f0000000000001400"), 0x01d0a1), + (hex!("01222222223333333344444444550000013f0000000000004740"), 0x01d141), + (hex!("0122222222333333334444444455000001400000000000001410"), 0x01d1e1), + (hex!("0122222222333333334444444455000001400000000000003f10"), 0x01d281), + (hex!("0122222222333333334444444455000001400000000000006d40"), 0x01d321), + (hex!("01222222223333333344444444550000014000000000000072d0"), 0x01d3c1), + (hex!("0122222222333333334444444455000001410000000000001420"), 0x01d461), + (hex!("0122222222333333334444444455000001420000000000001430"), 0x01d501), + (hex!("0122222222333333334444444455000001430000000000001440"), 0x01d5a1), + (hex!("0122222222333333334444444455000001440000000000001450"), 0x01d641), + (hex!("0122222222333333334444444455000001450000000000001460"), 0x01d6e1), + (hex!("0122222222333333334444444455000001460000000000001470"), 0x01d781), + (hex!("01222222223333333344444444550000014600000000000055c0"), 0x01d821), + (hex!("0122222222333333334444444455000001470000000000001480"), 0x01d8c1), + (hex!("0122222222333333334444444455000001470000000000004570"), 0x01d961), + (hex!("0122222222333333334444444455000001470000000000004be0"), 0x01da01), + (hex!("0122222222333333334444444455000001480000000000001490"), 0x01daa1), + (hex!("0122222222333333334444444455000001480000000000005360"), 0x01db41), + (hex!("01222222223333333344444444550000014900000000000014a0"), 0x01dbe1), + (hex!("01222222223333333344444444550000014a00000000000014b0"), 0x01dc81), + (hex!("01222222223333333344444444550000014a00000000000053d0"), 0x01dd21), + (hex!("01222222223333333344444444550000014b00000000000014c0"), 0x01ddc1), + (hex!("01222222223333333344444444550000014b0000000000005950"), 0x01de61), + (hex!("01222222223333333344444444550000014c00000000000014d0"), 0x01df01), + (hex!("01222222223333333344444444550000014c0000000000004f60"), 0x01dfa1), + (hex!("01222222223333333344444444550000014d00000000000014e0"), 0x01e041), + (hex!("01222222223333333344444444550000014d0000000000004520"), 0x01e0e1), + (hex!("01222222223333333344444444550000014d0000000000005200"), 0x01e181), + (hex!("01222222223333333344444444550000014e00000000000014f0"), 0x01e221), + (hex!("01222222223333333344444444550000014e0000000000005bd0"), 0x01e2c1), + (hex!("01222222223333333344444444550000014f0000000000001500"), 0x01e361), + (hex!("01222222223333333344444444550000014f00000000000060d0"), 0x01e401), + (hex!("0122222222333333334444444455000001500000000000001510"), 0x01e4a1), + (hex!("01222222223333333344444444550000015000000000000075e0"), 0x01e541), + (hex!("0122222222333333334444444455000001510000000000001520"), 0x01e5e1), + (hex!("0122222222333333334444444455000001510000000000005c00"), 0x01e681), + (hex!("0122222222333333334444444455000001510000000000006af0"), 0x01e721), + (hex!("0122222222333333334444444455000001510000000000007b80"), 0x01e7c1), + (hex!("0122222222333333334444444455000001520000000000001530"), 0x01e861), + (hex!("0122222222333333334444444455000001520000000000004c70"), 0x01e901), + (hex!("0122222222333333334444444455000001530000000000001540"), 0x01e9a1), + (hex!("0122222222333333334444444455000001540000000000001550"), 0x01ea41), + (hex!("0122222222333333334444444455000001540000000000007cd0"), 0x01eae1), + (hex!("0122222222333333334444444455000001550000000000001560"), 0x01eb81), + (hex!("0122222222333333334444444455000001550000000000004ae0"), 0x01ec21), + (hex!("01222222223333333344444444550000015500000000000068c0"), 0x01ecc1), + (hex!("0122222222333333334444444455000001560000000000001570"), 0x01ed61), + (hex!("01222222223333333344444444550000015600000000000064a0"), 0x01ee01), + (hex!("0122222222333333334444444455000001570000000000001580"), 0x01eea1), + (hex!("0122222222333333334444444455000001580000000000001590"), 0x01ef41), + (hex!("0122222222333333334444444455000001580000000000006d30"), 0x01efe1), + (hex!("01222222223333333344444444550000015800000000000074f0"), 0x01f081), + (hex!("01222222223333333344444444550000015900000000000015a0"), 0x01f121), + (hex!("01222222223333333344444444550000015900000000000053a0"), 0x01f1c1), + (hex!("01222222223333333344444444550000015900000000000055e0"), 0x01f261), + (hex!("0122222222333333334444444455000001590000000000006210"), 0x01f301), + (hex!("01222222223333333344444444550000015900000000000067c0"), 0x01f3a1), + (hex!("01222222223333333344444444550000015a00000000000015b0"), 0x01f441), + (hex!("01222222223333333344444444550000015b00000000000015c0"), 0x01f4e1), + (hex!("01222222223333333344444444550000015c00000000000015d0"), 0x01f581), + (hex!("01222222223333333344444444550000015c0000000000004d80"), 0x01f621), + (hex!("01222222223333333344444444550000015c00000000000073f0"), 0x01f6c1), + (hex!("01222222223333333344444444550000015d00000000000015e0"), 0x01f761), + (hex!("01222222223333333344444444550000015e00000000000015f0"), 0x01f801), + (hex!("01222222223333333344444444550000015e0000000000004120"), 0x01f8a1), + (hex!("01222222223333333344444444550000015e0000000000004350"), 0x01f941), + (hex!("01222222223333333344444444550000015e0000000000007c50"), 0x01f9e1), + (hex!("01222222223333333344444444550000015f0000000000001600"), 0x01fa81), + (hex!("0122222222333333334444444455000001600000000000001610"), 0x01fb21), + (hex!("0122222222333333334444444455000001600000000000004840"), 0x01fbc1), + (hex!("0122222222333333334444444455000001600000000000004b10"), 0x01fc61), + (hex!("0122222222333333334444444455000001600000000000007060"), 0x01fd01), + (hex!("0122222222333333334444444455000001610000000000001620"), 0x01fda1), + (hex!("0122222222333333334444444455000001610000000000005300"), 0x01fe41), + (hex!("0122222222333333334444444455000001620000000000001630"), 0x01fee1), + (hex!("0122222222333333334444444455000001620000000000006530"), 0x01ff81), + (hex!("0122222222333333334444444455000001630000000000001640"), 0x020021), + (hex!("0122222222333333334444444455000001640000000000001650"), 0x0200c1), + (hex!("0122222222333333334444444455000001650000000000001660"), 0x020161), + (hex!("0122222222333333334444444455000001660000000000001670"), 0x020201), + (hex!("0122222222333333334444444455000001670000000000001680"), 0x0202a1), + (hex!("0122222222333333334444444455000001670000000000007310"), 0x020341), + (hex!("0122222222333333334444444455000001680000000000001690"), 0x0203e1), + (hex!("0122222222333333334444444455000001680000000000007b50"), 0x020481), + (hex!("01222222223333333344444444550000016900000000000016a0"), 0x020521), + (hex!("01222222223333333344444444550000016900000000000049d0"), 0x0205c1), + (hex!("01222222223333333344444444550000016a00000000000016b0"), 0x020661), + (hex!("01222222223333333344444444550000016a00000000000078b0"), 0x020701), + (hex!("01222222223333333344444444550000016b00000000000016c0"), 0x0207a1), + (hex!("01222222223333333344444444550000016b0000000000004100"), 0x020841), + (hex!("01222222223333333344444444550000016c00000000000016d0"), 0x0208e1), + (hex!("01222222223333333344444444550000016c0000000000006e00"), 0x020981), + (hex!("01222222223333333344444444550000016d00000000000016e0"), 0x020a21), + (hex!("01222222223333333344444444550000016e00000000000016f0"), 0x020ac1), + (hex!("01222222223333333344444444550000016e0000000000004ac0"), 0x020b61), + (hex!("01222222223333333344444444550000016e0000000000007820"), 0x020c01), + (hex!("01222222223333333344444444550000016f0000000000001700"), 0x020ca1), + (hex!("0122222222333333334444444455000001700000000000001710"), 0x020d41), + (hex!("0122222222333333334444444455000001700000000000005830"), 0x020de1), + (hex!("0122222222333333334444444455000001710000000000001720"), 0x020e81), + (hex!("01222222223333333344444444550000017100000000000072f0"), 0x020f21), + (hex!("0122222222333333334444444455000001720000000000001730"), 0x020fc1), + (hex!("0122222222333333334444444455000001720000000000004870"), 0x021061), + (hex!("01222222223333333344444444550000017200000000000070b0"), 0x021101), + (hex!("0122222222333333334444444455000001730000000000001740"), 0x0211a1), + (hex!("0122222222333333334444444455000001740000000000001750"), 0x021241), + (hex!("0122222222333333334444444455000001750000000000001760"), 0x0212e1), + (hex!("0122222222333333334444444455000001750000000000005670"), 0x021381), + (hex!("0122222222333333334444444455000001750000000000005870"), 0x021421), + (hex!("0122222222333333334444444455000001760000000000001770"), 0x0214c1), + (hex!("0122222222333333334444444455000001770000000000001780"), 0x021561), + (hex!("0122222222333333334444444455000001770000000000005000"), 0x021601), + (hex!("0122222222333333334444444455000001770000000000007090"), 0x0216a1), + (hex!("0122222222333333334444444455000001780000000000001790"), 0x021741), + (hex!("01222222223333333344444444550000017800000000000048a0"), 0x0217e1), + (hex!("0122222222333333334444444455000001780000000000006bf0"), 0x021881), + (hex!("01222222223333333344444444550000017900000000000017a0"), 0x021921), + (hex!("01222222223333333344444444550000017900000000000057d0"), 0x0219c1), + (hex!("0122222222333333334444444455000001790000000000006660"), 0x021a61), + (hex!("01222222223333333344444444550000017a00000000000017b0"), 0x021b01), + (hex!("01222222223333333344444444550000017a0000000000004970"), 0x021ba1), + (hex!("01222222223333333344444444550000017a0000000000005dc0"), 0x021c41), + (hex!("01222222223333333344444444550000017b00000000000017c0"), 0x021ce1), + (hex!("01222222223333333344444444550000017b0000000000004ee0"), 0x021d81), + (hex!("01222222223333333344444444550000017b00000000000054c0"), 0x021e21), + (hex!("01222222223333333344444444550000017c00000000000017d0"), 0x021ec1), + (hex!("01222222223333333344444444550000017c0000000000003fc0"), 0x021f61), + (hex!("01222222223333333344444444550000017c00000000000063e0"), 0x022001), + (hex!("01222222223333333344444444550000017c0000000000006520"), 0x0220a1), + (hex!("01222222223333333344444444550000017d00000000000017e0"), 0x022141), + (hex!("01222222223333333344444444550000017d0000000000006220"), 0x0221e1), + (hex!("01222222223333333344444444550000017d0000000000007120"), 0x022281), + (hex!("01222222223333333344444444550000017e00000000000017f0"), 0x022321), + (hex!("01222222223333333344444444550000017f0000000000001800"), 0x0223c1), + (hex!("0122222222333333334444444455000001800000000000001810"), 0x022461), + (hex!("0122222222333333334444444455000001810000000000001820"), 0x022501), + (hex!("01222222223333333344444444550000018100000000000041f0"), 0x0225a1), + (hex!("0122222222333333334444444455000001810000000000007590"), 0x022641), + (hex!("0122222222333333334444444455000001820000000000001830"), 0x0226e1), + (hex!("0122222222333333334444444455000001820000000000004ce0"), 0x022781), + (hex!("0122222222333333334444444455000001830000000000001840"), 0x022821), + (hex!("01222222223333333344444444550000018300000000000042c0"), 0x0228c1), + (hex!("0122222222333333334444444455000001840000000000001850"), 0x022961), + (hex!("0122222222333333334444444455000001840000000000004f70"), 0x022a01), + (hex!("0122222222333333334444444455000001850000000000001860"), 0x022aa1), + (hex!("0122222222333333334444444455000001850000000000006470"), 0x022b41), + (hex!("0122222222333333334444444455000001850000000000007500"), 0x022be1), + (hex!("0122222222333333334444444455000001860000000000001870"), 0x022c81), + (hex!("0122222222333333334444444455000001860000000000004770"), 0x022d21), + (hex!("0122222222333333334444444455000001870000000000001880"), 0x022dc1), + (hex!("0122222222333333334444444455000001870000000000006a30"), 0x022e61), + (hex!("0122222222333333334444444455000001880000000000001890"), 0x022f01), + (hex!("0122222222333333334444444455000001880000000000007410"), 0x022fa1), + (hex!("01222222223333333344444444550000018900000000000018a0"), 0x023041), + (hex!("01222222223333333344444444550000018900000000000044d0"), 0x0230e1), + (hex!("0122222222333333334444444455000001890000000000005ac0"), 0x023181), + (hex!("01222222223333333344444444550000018a00000000000018b0"), 0x023221), + (hex!("01222222223333333344444444550000018a0000000000006260"), 0x0232c1), + (hex!("01222222223333333344444444550000018a0000000000006d70"), 0x023361), + (hex!("01222222223333333344444444550000018b00000000000018c0"), 0x023401), + (hex!("01222222223333333344444444550000018b0000000000004aa0"), 0x0234a1), + (hex!("01222222223333333344444444550000018b0000000000006fd0"), 0x023541), + (hex!("01222222223333333344444444550000018c00000000000018d0"), 0x0235e1), + (hex!("01222222223333333344444444550000018c00000000000051b0"), 0x023681), + (hex!("01222222223333333344444444550000018c0000000000006650"), 0x023721), + (hex!("01222222223333333344444444550000018d00000000000018e0"), 0x0237c1), + (hex!("01222222223333333344444444550000018e00000000000018f0"), 0x023861), + (hex!("01222222223333333344444444550000018e00000000000041d0"), 0x023901), + (hex!("01222222223333333344444444550000018f0000000000001900"), 0x0239a1), + (hex!("01222222223333333344444444550000018f0000000000007600"), 0x023a41), + (hex!("0122222222333333334444444455000001900000000000001910"), 0x023ae1), + (hex!("0122222222333333334444444455000001900000000000005410"), 0x023b81), + (hex!("0122222222333333334444444455000001900000000000006760"), 0x023c21), + (hex!("0122222222333333334444444455000001910000000000001920"), 0x023cc1), + (hex!("0122222222333333334444444455000001920000000000001930"), 0x023d61), + (hex!("0122222222333333334444444455000001920000000000004ca0"), 0x023e01), + (hex!("0122222222333333334444444455000001920000000000005d80"), 0x023ea1), + (hex!("0122222222333333334444444455000001920000000000005fd0"), 0x023f41), + (hex!("01222222223333333344444444550000019200000000000070d0"), 0x023fe1), + (hex!("0122222222333333334444444455000001930000000000001940"), 0x024081), + (hex!("0122222222333333334444444455000001930000000000004010"), 0x024121), + (hex!("0122222222333333334444444455000001930000000000007ca0"), 0x0241c1), + (hex!("0122222222333333334444444455000001940000000000001950"), 0x024261), + (hex!("0122222222333333334444444455000001950000000000001960"), 0x024301), + (hex!("0122222222333333334444444455000001950000000000005380"), 0x0243a1), + (hex!("0122222222333333334444444455000001960000000000001970"), 0x024441), + (hex!("0122222222333333334444444455000001960000000000006de0"), 0x0244e1), + (hex!("0122222222333333334444444455000001970000000000001980"), 0x024581), + (hex!("01222222223333333344444444550000019700000000000048f0"), 0x024621), + (hex!("0122222222333333334444444455000001980000000000001990"), 0x0246c1), + (hex!("0122222222333333334444444455000001980000000000006510"), 0x024761), + (hex!("01222222223333333344444444550000019900000000000019a0"), 0x024801), + (hex!("0122222222333333334444444455000001990000000000007570"), 0x0248a1), + (hex!("0122222222333333334444444455000001990000000000007580"), 0x024941), + (hex!("01222222223333333344444444550000019a00000000000019b0"), 0x0249e1), + (hex!("01222222223333333344444444550000019a0000000000004050"), 0x024a81), + (hex!("01222222223333333344444444550000019a0000000000004ba0"), 0x024b21), + (hex!("01222222223333333344444444550000019a0000000000005540"), 0x024bc1), + (hex!("01222222223333333344444444550000019a00000000000061c0"), 0x024c61), + (hex!("01222222223333333344444444550000019a0000000000007c60"), 0x024d01), + (hex!("01222222223333333344444444550000019b00000000000019c0"), 0x024da1), + (hex!("01222222223333333344444444550000019b0000000000006240"), 0x024e41), + (hex!("01222222223333333344444444550000019c00000000000019d0"), 0x024ee1), + (hex!("01222222223333333344444444550000019d00000000000019e0"), 0x024f81), + (hex!("01222222223333333344444444550000019d0000000000004640"), 0x025021), + (hex!("01222222223333333344444444550000019d00000000000052a0"), 0x0250c1), + (hex!("01222222223333333344444444550000019d00000000000052b0"), 0x025161), + (hex!("01222222223333333344444444550000019e00000000000019f0"), 0x025201), + (hex!("01222222223333333344444444550000019f0000000000001a00"), 0x0252a1), + (hex!("01222222223333333344444444550000019f0000000000006b20"), 0x025341), + (hex!("0122222222333333334444444455000001a00000000000001a10"), 0x0253e1), + (hex!("0122222222333333334444444455000001a10000000000001a20"), 0x025481), + (hex!("0122222222333333334444444455000001a10000000000005460"), 0x025521), + (hex!("0122222222333333334444444455000001a10000000000005d20"), 0x0255c1), + (hex!("0122222222333333334444444455000001a100000000000068f0"), 0x025661), + (hex!("0122222222333333334444444455000001a20000000000001a30"), 0x025701), + (hex!("0122222222333333334444444455000001a20000000000007170"), 0x0257a1), + (hex!("0122222222333333334444444455000001a30000000000001a40"), 0x025841), + (hex!("0122222222333333334444444455000001a40000000000001a50"), 0x0258e1), + (hex!("0122222222333333334444444455000001a50000000000001a60"), 0x025981), + (hex!("0122222222333333334444444455000001a60000000000001a70"), 0x025a21), + (hex!("0122222222333333334444444455000001a70000000000001a80"), 0x025ac1), + (hex!("0122222222333333334444444455000001a70000000000005a90"), 0x025b61), + (hex!("0122222222333333334444444455000001a70000000000006440"), 0x025c01), + (hex!("0122222222333333334444444455000001a80000000000001a90"), 0x025ca1), + (hex!("0122222222333333334444444455000001a80000000000004800"), 0x025d41), + (hex!("0122222222333333334444444455000001a90000000000001aa0"), 0x025de1), + (hex!("0122222222333333334444444455000001aa0000000000001ab0"), 0x025e81), + (hex!("0122222222333333334444444455000001aa0000000000005b60"), 0x025f21), + (hex!("0122222222333333334444444455000001ab0000000000001ac0"), 0x025fc1), + (hex!("0122222222333333334444444455000001ab0000000000006700"), 0x026061), + (hex!("0122222222333333334444444455000001ab00000000000071d0"), 0x026101), + (hex!("0122222222333333334444444455000001ac0000000000001ad0"), 0x0261a1), + (hex!("0122222222333333334444444455000001ac0000000000007380"), 0x026241), + (hex!("0122222222333333334444444455000001ad0000000000001ae0"), 0x0262e1), + (hex!("0122222222333333334444444455000001ad0000000000006350"), 0x026381), + (hex!("0122222222333333334444444455000001ae0000000000001af0"), 0x026421), + (hex!("0122222222333333334444444455000001af0000000000001b00"), 0x0264c1), + (hex!("0122222222333333334444444455000001af0000000000007390"), 0x026561), + (hex!("0122222222333333334444444455000001b00000000000001b10"), 0x026601), + (hex!("0122222222333333334444444455000001b10000000000001b20"), 0x0266a1), + (hex!("0122222222333333334444444455000001b10000000000005cc0"), 0x026741), + (hex!("0122222222333333334444444455000001b20000000000001b30"), 0x0267e1), + (hex!("0122222222333333334444444455000001b20000000000004fb0"), 0x026881), + (hex!("0122222222333333334444444455000001b30000000000001b40"), 0x026921), + (hex!("0122222222333333334444444455000001b40000000000001b50"), 0x0269c1), + (hex!("0122222222333333334444444455000001b50000000000001b60"), 0x026a61), + (hex!("0122222222333333334444444455000001b60000000000001b70"), 0x026b01), + (hex!("0122222222333333334444444455000001b600000000000048e0"), 0x026ba1), + (hex!("0122222222333333334444444455000001b70000000000001b80"), 0x026c41), + (hex!("0122222222333333334444444455000001b70000000000005ca0"), 0x026ce1), + (hex!("0122222222333333334444444455000001b70000000000007900"), 0x026d81), + (hex!("0122222222333333334444444455000001b80000000000001b90"), 0x026e21), + (hex!("0122222222333333334444444455000001b80000000000004d90"), 0x026ec1), + (hex!("0122222222333333334444444455000001b90000000000001ba0"), 0x026f61), + (hex!("0122222222333333334444444455000001b90000000000003f40"), 0x027001), + (hex!("0122222222333333334444444455000001ba0000000000001bb0"), 0x0270a1), + (hex!("0122222222333333334444444455000001ba00000000000042a0"), 0x027141), + (hex!("0122222222333333334444444455000001ba00000000000067f0"), 0x0271e1), + (hex!("0122222222333333334444444455000001ba00000000000073a0"), 0x027281), + (hex!("0122222222333333334444444455000001bb0000000000001bc0"), 0x027321), + (hex!("0122222222333333334444444455000001bb0000000000004a00"), 0x0273c1), + (hex!("0122222222333333334444444455000001bb0000000000005e00"), 0x027461), + (hex!("0122222222333333334444444455000001bc0000000000001bd0"), 0x027501), + (hex!("0122222222333333334444444455000001bc0000000000004230"), 0x0275a1), + (hex!("0122222222333333334444444455000001bc0000000000005860"), 0x027641), + (hex!("0122222222333333334444444455000001bd0000000000001be0"), 0x0276e1), + (hex!("0122222222333333334444444455000001bd0000000000007c70"), 0x027781), + (hex!("0122222222333333334444444455000001be0000000000001bf0"), 0x027821), + (hex!("0122222222333333334444444455000001be0000000000007770"), 0x0278c1), + (hex!("0122222222333333334444444455000001be0000000000007cf0"), 0x027961), + (hex!("0122222222333333334444444455000001bf0000000000001c00"), 0x027a01), + (hex!("0122222222333333334444444455000001bf0000000000006490"), 0x027aa1), + (hex!("0122222222333333334444444455000001c00000000000001c10"), 0x027b41), + (hex!("0122222222333333334444444455000001c10000000000001c20"), 0x027be1), + (hex!("0122222222333333334444444455000001c10000000000004600"), 0x027c81), + (hex!("0122222222333333334444444455000001c20000000000001c30"), 0x027d21), + (hex!("0122222222333333334444444455000001c20000000000006e30"), 0x027dc1), + (hex!("0122222222333333334444444455000001c30000000000001c40"), 0x027e61), + (hex!("0122222222333333334444444455000001c40000000000001c50"), 0x027f01), + (hex!("0122222222333333334444444455000001c50000000000001c60"), 0x027fa1), + (hex!("0122222222333333334444444455000001c60000000000001c70"), 0x028041), + (hex!("0122222222333333334444444455000001c60000000000004240"), 0x0280e1), + (hex!("0122222222333333334444444455000001c60000000000005bb0"), 0x028181), + (hex!("0122222222333333334444444455000001c70000000000001c80"), 0x028221), + (hex!("0122222222333333334444444455000001c80000000000001c90"), 0x0282c1), + (hex!("0122222222333333334444444455000001c90000000000001ca0"), 0x028361), + (hex!("0122222222333333334444444455000001c90000000000006730"), 0x028401), + (hex!("0122222222333333334444444455000001ca0000000000001cb0"), 0x0284a1), + (hex!("0122222222333333334444444455000001ca00000000000070f0"), 0x028541), + (hex!("0122222222333333334444444455000001cb0000000000001cc0"), 0x0285e1), + (hex!("0122222222333333334444444455000001cb00000000000071a0"), 0x028681), + (hex!("0122222222333333334444444455000001cc0000000000001cd0"), 0x028721), + (hex!("0122222222333333334444444455000001cc0000000000005280"), 0x0287c1), + (hex!("0122222222333333334444444455000001cc0000000000005d90"), 0x028861), + (hex!("0122222222333333334444444455000001cd0000000000001ce0"), 0x028901), + (hex!("0122222222333333334444444455000001cd00000000000069b0"), 0x0289a1), + (hex!("0122222222333333334444444455000001ce0000000000001cf0"), 0x028a41), + (hex!("0122222222333333334444444455000001ce0000000000004540"), 0x028ae1), + (hex!("0122222222333333334444444455000001cf0000000000001d00"), 0x028b81), + (hex!("0122222222333333334444444455000001cf00000000000076a0"), 0x028c21), + (hex!("0122222222333333334444444455000001d00000000000001d10"), 0x028cc1), + (hex!("0122222222333333334444444455000001d000000000000060a0"), 0x028d61), + (hex!("0122222222333333334444444455000001d10000000000001d20"), 0x028e01), + (hex!("0122222222333333334444444455000001d20000000000001d30"), 0x028ea1), + (hex!("0122222222333333334444444455000001d30000000000001d40"), 0x028f41), + (hex!("0122222222333333334444444455000001d30000000000004000"), 0x028fe1), + (hex!("0122222222333333334444444455000001d30000000000004140"), 0x029081), + (hex!("0122222222333333334444444455000001d30000000000006790"), 0x029121), + (hex!("0122222222333333334444444455000001d40000000000001d50"), 0x0291c1), + (hex!("0122222222333333334444444455000001d50000000000001d60"), 0x029261), + (hex!("0122222222333333334444444455000001d60000000000001d70"), 0x029301), + (hex!("0122222222333333334444444455000001d60000000000004b50"), 0x0293a1), + (hex!("0122222222333333334444444455000001d60000000000007430"), 0x029441), + (hex!("0122222222333333334444444455000001d70000000000001d80"), 0x0294e1), + (hex!("0122222222333333334444444455000001d70000000000006920"), 0x029581), + (hex!("0122222222333333334444444455000001d80000000000001d90"), 0x029621), + (hex!("0122222222333333334444444455000001d80000000000005b30"), 0x0296c1), + (hex!("0122222222333333334444444455000001d90000000000001da0"), 0x029761), + (hex!("0122222222333333334444444455000001da0000000000001db0"), 0x029801), + (hex!("0122222222333333334444444455000001da0000000000004af0"), 0x0298a1), + (hex!("0122222222333333334444444455000001da0000000000007240"), 0x029941), + (hex!("0122222222333333334444444455000001da0000000000007470"), 0x0299e1), + (hex!("0122222222333333334444444455000001db0000000000001dc0"), 0x029a81), + (hex!("0122222222333333334444444455000001db00000000000045d0"), 0x029b21), + (hex!("0122222222333333334444444455000001dc0000000000001dd0"), 0x029bc1), + (hex!("0122222222333333334444444455000001dd0000000000001de0"), 0x029c61), + (hex!("0122222222333333334444444455000001dd0000000000004bb0"), 0x029d01), + (hex!("0122222222333333334444444455000001dd0000000000004cd0"), 0x029da1), + (hex!("0122222222333333334444444455000001dd0000000000006100"), 0x029e41), + (hex!("0122222222333333334444444455000001dd0000000000007bb0"), 0x029ee1), + (hex!("0122222222333333334444444455000001de0000000000001df0"), 0x029f81), + (hex!("0122222222333333334444444455000001de0000000000004260"), 0x02a021), + (hex!("0122222222333333334444444455000001de0000000000006040"), 0x02a0c1), + (hex!("0122222222333333334444444455000001df0000000000001e00"), 0x02a161), + (hex!("0122222222333333334444444455000001df0000000000005fa0"), 0x02a201), + (hex!("0122222222333333334444444455000001df0000000000006a70"), 0x02a2a1), + (hex!("0122222222333333334444444455000001df0000000000006dc0"), 0x02a341), + (hex!("0122222222333333334444444455000001e00000000000001e10"), 0x02a3e1), + (hex!("0122222222333333334444444455000001e00000000000007010"), 0x02a481), + (hex!("0122222222333333334444444455000001e10000000000001e20"), 0x02a521), + (hex!("0122222222333333334444444455000001e10000000000005720"), 0x02a5c1), + (hex!("0122222222333333334444444455000001e10000000000006830"), 0x02a661), + (hex!("0122222222333333334444444455000001e20000000000001e30"), 0x02a701), + (hex!("0122222222333333334444444455000001e20000000000005100"), 0x02a7a1), + (hex!("0122222222333333334444444455000001e30000000000001e40"), 0x02a841), + (hex!("0122222222333333334444444455000001e40000000000001e50"), 0x02a8e1), + (hex!("0122222222333333334444444455000001e40000000000003f30"), 0x02a981), + (hex!("0122222222333333334444444455000001e40000000000005220"), 0x02aa21), + (hex!("0122222222333333334444444455000001e50000000000001e60"), 0x02aac1), + (hex!("0122222222333333334444444455000001e50000000000006f60"), 0x02ab61), + (hex!("0122222222333333334444444455000001e60000000000001e70"), 0x02ac01), + (hex!("0122222222333333334444444455000001e60000000000006c80"), 0x02aca1), + (hex!("0122222222333333334444444455000001e70000000000001e80"), 0x02ad41), + (hex!("0122222222333333334444444455000001e80000000000001e90"), 0x02ade1), + (hex!("0122222222333333334444444455000001e80000000000004e30"), 0x02ae81), + (hex!("0122222222333333334444444455000001e90000000000001ea0"), 0x02af21), + (hex!("0122222222333333334444444455000001e90000000000005470"), 0x02afc1), + (hex!("0122222222333333334444444455000001ea0000000000001eb0"), 0x02b061), + (hex!("0122222222333333334444444455000001ea0000000000007980"), 0x02b101), + (hex!("0122222222333333334444444455000001eb0000000000001ec0"), 0x02b1a1), + (hex!("0122222222333333334444444455000001eb0000000000004390"), 0x02b241), + (hex!("0122222222333333334444444455000001eb0000000000005970"), 0x02b2e1), + (hex!("0122222222333333334444444455000001ec0000000000001ed0"), 0x02b381), + (hex!("0122222222333333334444444455000001ec0000000000005d50"), 0x02b421), + (hex!("0122222222333333334444444455000001ec00000000000076e0"), 0x02b4c1), + (hex!("0122222222333333334444444455000001ed0000000000001ee0"), 0x02b561), + (hex!("0122222222333333334444444455000001ed0000000000006190"), 0x02b601), + (hex!("0122222222333333334444444455000001ee0000000000001ef0"), 0x02b6a1), + (hex!("0122222222333333334444444455000001ee0000000000004900"), 0x02b741), + (hex!("0122222222333333334444444455000001ef0000000000001f00"), 0x02b7e1), + (hex!("0122222222333333334444444455000001ef0000000000006c60"), 0x02b881), + (hex!("0122222222333333334444444455000001f00000000000001f10"), 0x02b921), + (hex!("0122222222333333334444444455000001f00000000000006950"), 0x02b9c1), + (hex!("0122222222333333334444444455000001f10000000000001f20"), 0x02ba61), + (hex!("0122222222333333334444444455000001f10000000000006400"), 0x02bb01), + (hex!("0122222222333333334444444455000001f20000000000001f30"), 0x02bba1), + (hex!("0122222222333333334444444455000001f20000000000006f00"), 0x02bc41), + (hex!("0122222222333333334444444455000001f20000000000007b10"), 0x02bce1), + (hex!("0122222222333333334444444455000001f30000000000001f40"), 0x02bd81), + (hex!("0122222222333333334444444455000001f40000000000001f50"), 0x02be21), + (hex!("0122222222333333334444444455000001f50000000000001f60"), 0x02bec1), + (hex!("0122222222333333334444444455000001f500000000000044f0"), 0x02bf61), + (hex!("0122222222333333334444444455000001f60000000000001f70"), 0x02c001), + (hex!("0122222222333333334444444455000001f70000000000001f80"), 0x02c0a1), + (hex!("0122222222333333334444444455000001f70000000000004ad0"), 0x02c141), + (hex!("0122222222333333334444444455000001f80000000000001f90"), 0x02c1e1), + (hex!("0122222222333333334444444455000001f90000000000001fa0"), 0x02c281), + (hex!("0122222222333333334444444455000001f90000000000003f60"), 0x02c321), + (hex!("0122222222333333334444444455000001f90000000000004a80"), 0x02c3c1), + (hex!("0122222222333333334444444455000001fa0000000000001fb0"), 0x02c461), + (hex!("0122222222333333334444444455000001fa0000000000006f90"), 0x02c501), + (hex!("0122222222333333334444444455000001fb0000000000001fc0"), 0x02c5a1), + (hex!("0122222222333333334444444455000001fc0000000000001fd0"), 0x02c641), + (hex!("0122222222333333334444444455000001fc0000000000004a90"), 0x02c6e1), + (hex!("0122222222333333334444444455000001fd0000000000001fe0"), 0x02c781), + (hex!("0122222222333333334444444455000001fd0000000000005f50"), 0x02c821), + (hex!("0122222222333333334444444455000001fe0000000000001ff0"), 0x02c8c1), + (hex!("0122222222333333334444444455000001ff0000000000002000"), 0x02c961), + (hex!("0122222222333333334444444455000002000000000000002010"), 0x02ca01), + (hex!("0122222222333333334444444455000002000000000000005f00"), 0x02caa1), + (hex!("0122222222333333334444444455000002000000000000006840"), 0x02cb41), + (hex!("0122222222333333334444444455000002010000000000002020"), 0x02cbe1), + (hex!("0122222222333333334444444455000002020000000000002030"), 0x02cc81), + (hex!("0122222222333333334444444455000002030000000000002040"), 0x02cd21), + (hex!("0122222222333333334444444455000002040000000000002050"), 0x02cdc1), + (hex!("01222222223333333344444444550000020400000000000051f0"), 0x02ce61), + (hex!("0122222222333333334444444455000002050000000000002060"), 0x02cf01), + (hex!("0122222222333333334444444455000002060000000000002070"), 0x02cfa1), + (hex!("0122222222333333334444444455000002060000000000005c80"), 0x02d041), + (hex!("01222222223333333344444444550000020600000000000061d0"), 0x02d0e1), + (hex!("01222222223333333344444444550000020600000000000078c0"), 0x02d181), + (hex!("0122222222333333334444444455000002070000000000002080"), 0x02d221), + (hex!("0122222222333333334444444455000002070000000000006ba0"), 0x02d2c1), + (hex!("0122222222333333334444444455000002080000000000002090"), 0x02d361), + (hex!("01222222223333333344444444550000020900000000000020a0"), 0x02d401), + (hex!("01222222223333333344444444550000020900000000000067a0"), 0x02d4a1), + (hex!("01222222223333333344444444550000020a00000000000020b0"), 0x02d541), + (hex!("01222222223333333344444444550000020a0000000000004950"), 0x02d5e1), + (hex!("01222222223333333344444444550000020a0000000000004de0"), 0x02d681), + (hex!("01222222223333333344444444550000020b00000000000020c0"), 0x02d721), + (hex!("01222222223333333344444444550000020b0000000000004b00"), 0x02d7c1), + (hex!("01222222223333333344444444550000020c00000000000020d0"), 0x02d861), + (hex!("01222222223333333344444444550000020d00000000000020e0"), 0x02d901), + (hex!("01222222223333333344444444550000020e00000000000020f0"), 0x02d9a1), + (hex!("01222222223333333344444444550000020f0000000000002100"), 0x02da41), + (hex!("0122222222333333334444444455000002100000000000002110"), 0x02dae1), + (hex!("0122222222333333334444444455000002110000000000002120"), 0x02db81), + (hex!("0122222222333333334444444455000002110000000000004490"), 0x02dc21), + (hex!("0122222222333333334444444455000002120000000000002130"), 0x02dcc1), + (hex!("0122222222333333334444444455000002130000000000002140"), 0x02dd61), + (hex!("01222222223333333344444444550000021300000000000046d0"), 0x02de01), + (hex!("01222222223333333344444444550000021300000000000046e0"), 0x02dea1), + (hex!("0122222222333333334444444455000002130000000000004b70"), 0x02df41), + (hex!("0122222222333333334444444455000002140000000000002150"), 0x02dfe1), + (hex!("0122222222333333334444444455000002140000000000006c50"), 0x02e081), + (hex!("0122222222333333334444444455000002150000000000002160"), 0x02e121), + (hex!("01222222223333333344444444550000021500000000000043c0"), 0x02e1c1), + (hex!("0122222222333333334444444455000002160000000000002170"), 0x02e261), + (hex!("01222222223333333344444444550000021600000000000055b0"), 0x02e301), + (hex!("0122222222333333334444444455000002160000000000006150"), 0x02e3a1), + (hex!("0122222222333333334444444455000002170000000000002180"), 0x02e441), + (hex!("01222222223333333344444444550000021700000000000053b0"), 0x02e4e1), + (hex!("0122222222333333334444444455000002170000000000007460"), 0x02e581), + (hex!("0122222222333333334444444455000002180000000000002190"), 0x02e621), + (hex!("01222222223333333344444444550000021900000000000021a0"), 0x02e6c1), + (hex!("01222222223333333344444444550000021a00000000000021b0"), 0x02e761), + (hex!("01222222223333333344444444550000021a0000000000007650"), 0x02e801), + (hex!("01222222223333333344444444550000021b00000000000021c0"), 0x02e8a1), + (hex!("01222222223333333344444444550000021b0000000000004b20"), 0x02e941), + (hex!("01222222223333333344444444550000021c00000000000021d0"), 0x02e9e1), + (hex!("01222222223333333344444444550000021c0000000000007610"), 0x02ea81), + (hex!("01222222223333333344444444550000021d00000000000021e0"), 0x02eb21), + (hex!("01222222223333333344444444550000021d0000000000005f40"), 0x02ebc1), + (hex!("01222222223333333344444444550000021e00000000000021f0"), 0x02ec61), + (hex!("01222222223333333344444444550000021e0000000000005a50"), 0x02ed01), + (hex!("01222222223333333344444444550000021e0000000000005ff0"), 0x02eda1), + (hex!("01222222223333333344444444550000021f0000000000002200"), 0x02ee41), + (hex!("01222222223333333344444444550000021f00000000000043a0"), 0x02eee1), + (hex!("01222222223333333344444444550000021f0000000000004cb0"), 0x02ef81), + (hex!("01222222223333333344444444550000021f0000000000004e00"), 0x02f021), + (hex!("0122222222333333334444444455000002200000000000002210"), 0x02f0c1), + (hex!("0122222222333333334444444455000002210000000000002220"), 0x02f161), + (hex!("0122222222333333334444444455000002210000000000006290"), 0x02f201), + (hex!("0122222222333333334444444455000002210000000000007230"), 0x02f2a1), + (hex!("0122222222333333334444444455000002220000000000002230"), 0x02f341), + (hex!("0122222222333333334444444455000002220000000000006ea0"), 0x02f3e1), + (hex!("0122222222333333334444444455000002230000000000002240"), 0x02f481), + (hex!("0122222222333333334444444455000002230000000000004710"), 0x02f521), + (hex!("0122222222333333334444444455000002240000000000002250"), 0x02f5c1), + (hex!("0122222222333333334444444455000002250000000000002260"), 0x02f661), + (hex!("0122222222333333334444444455000002260000000000002270"), 0x02f701), + (hex!("0122222222333333334444444455000002260000000000005b40"), 0x02f7a1), + (hex!("0122222222333333334444444455000002260000000000006300"), 0x02f841), + (hex!("0122222222333333334444444455000002270000000000002280"), 0x02f8e1), + (hex!("0122222222333333334444444455000002270000000000005b80"), 0x02f981), + (hex!("0122222222333333334444444455000002280000000000002290"), 0x02fa21), + (hex!("0122222222333333334444444455000002280000000000003ed0"), 0x02fac1), + (hex!("0122222222333333334444444455000002280000000000004550"), 0x02fb61), + (hex!("01222222223333333344444444550000022800000000000077d0"), 0x02fc01), + (hex!("01222222223333333344444444550000022900000000000022a0"), 0x02fca1), + (hex!("0122222222333333334444444455000002290000000000006480"), 0x02fd41), + (hex!("01222222223333333344444444550000022a00000000000022b0"), 0x02fde1), + (hex!("01222222223333333344444444550000022a0000000000005450"), 0x02fe81), + (hex!("01222222223333333344444444550000022b00000000000022c0"), 0x02ff21), + (hex!("01222222223333333344444444550000022b0000000000006dd0"), 0x02ffc1), + (hex!("01222222223333333344444444550000022c00000000000022d0"), 0x030061), + (hex!("01222222223333333344444444550000022c0000000000006890"), 0x030101), + (hex!("01222222223333333344444444550000022d00000000000022e0"), 0x0301a1), + (hex!("01222222223333333344444444550000022e00000000000022f0"), 0x030241), + (hex!("01222222223333333344444444550000022e0000000000004f20"), 0x0302e1), + (hex!("01222222223333333344444444550000022f0000000000002300"), 0x030381), + (hex!("01222222223333333344444444550000022f0000000000005260"), 0x030421), + (hex!("01222222223333333344444444550000022f00000000000053f0"), 0x0304c1), + (hex!("0122222222333333334444444455000002300000000000002310"), 0x030561), + (hex!("01222222223333333344444444550000023000000000000050e0"), 0x030601), + (hex!("0122222222333333334444444455000002310000000000002320"), 0x0306a1), + (hex!("0122222222333333334444444455000002310000000000007800"), 0x030741), + (hex!("0122222222333333334444444455000002320000000000002330"), 0x0307e1), + (hex!("0122222222333333334444444455000002330000000000002340"), 0x030881), + (hex!("0122222222333333334444444455000002330000000000004d70"), 0x030921), + (hex!("0122222222333333334444444455000002330000000000005cf0"), 0x0309c1), + (hex!("0122222222333333334444444455000002340000000000002350"), 0x030a61), + (hex!("0122222222333333334444444455000002350000000000002360"), 0x030b01), + (hex!("0122222222333333334444444455000002350000000000006970"), 0x030ba1), + (hex!("0122222222333333334444444455000002360000000000002370"), 0x030c41), + (hex!("0122222222333333334444444455000002360000000000005270"), 0x030ce1), + (hex!("0122222222333333334444444455000002370000000000002380"), 0x030d81), + (hex!("0122222222333333334444444455000002370000000000005d70"), 0x030e21), + (hex!("0122222222333333334444444455000002380000000000002390"), 0x030ec1), + (hex!("01222222223333333344444444550000023800000000000069a0"), 0x030f61), + (hex!("01222222223333333344444444550000023900000000000023a0"), 0x031001), + (hex!("01222222223333333344444444550000023900000000000052e0"), 0x0310a1), + (hex!("0122222222333333334444444455000002390000000000005a10"), 0x031141), + (hex!("0122222222333333334444444455000002390000000000007440"), 0x0311e1), + (hex!("01222222223333333344444444550000023a00000000000023b0"), 0x031281), + (hex!("01222222223333333344444444550000023a0000000000003f00"), 0x031321), + (hex!("01222222223333333344444444550000023a0000000000004430"), 0x0313c1), + (hex!("01222222223333333344444444550000023a0000000000007070"), 0x031461), + (hex!("01222222223333333344444444550000023a00000000000074a0"), 0x031501), + (hex!("01222222223333333344444444550000023b00000000000023c0"), 0x0315a1), + (hex!("01222222223333333344444444550000023b0000000000004730"), 0x031641), + (hex!("01222222223333333344444444550000023b00000000000068b0"), 0x0316e1), + (hex!("01222222223333333344444444550000023c00000000000023d0"), 0x031781), + (hex!("01222222223333333344444444550000023c0000000000004680"), 0x031821), + (hex!("01222222223333333344444444550000023d00000000000023e0"), 0x0318c1), + (hex!("01222222223333333344444444550000023d00000000000059a0"), 0x031961), + (hex!("01222222223333333344444444550000023e00000000000023f0"), 0x031a01), + (hex!("01222222223333333344444444550000023f0000000000002400"), 0x031aa1), + (hex!("0122222222333333334444444455000002400000000000002410"), 0x031b41), + (hex!("0122222222333333334444444455000002400000000000004920"), 0x031be1), + (hex!("01222222223333333344444444550000024000000000000066e0"), 0x031c81), + (hex!("01222222223333333344444444550000024000000000000076f0"), 0x031d21), + (hex!("01222222223333333344444444550000024000000000000078e0"), 0x031dc1), + (hex!("0122222222333333334444444455000002410000000000002420"), 0x031e61), + (hex!("0122222222333333334444444455000002420000000000002430"), 0x031f01), + (hex!("0122222222333333334444444455000002420000000000006590"), 0x031fa1), + (hex!("0122222222333333334444444455000002430000000000002440"), 0x032041), + (hex!("0122222222333333334444444455000002430000000000004d00"), 0x0320e1), + (hex!("0122222222333333334444444455000002440000000000002450"), 0x032181), + (hex!("0122222222333333334444444455000002440000000000005f80"), 0x032221), + (hex!("0122222222333333334444444455000002450000000000002460"), 0x0322c1), + (hex!("0122222222333333334444444455000002450000000000004940"), 0x032361), + (hex!("0122222222333333334444444455000002460000000000002470"), 0x032401), + (hex!("0122222222333333334444444455000002470000000000002480"), 0x0324a1), + (hex!("0122222222333333334444444455000002470000000000004dd0"), 0x032541), + (hex!("0122222222333333334444444455000002470000000000005930"), 0x0325e1), + (hex!("01222222223333333344444444550000024700000000000061b0"), 0x032681), + (hex!("0122222222333333334444444455000002470000000000007740"), 0x032721), + (hex!("0122222222333333334444444455000002480000000000002490"), 0x0327c1), + (hex!("0122222222333333334444444455000002480000000000004890"), 0x032861), + (hex!("01222222223333333344444444550000024900000000000024a0"), 0x032901), + (hex!("01222222223333333344444444550000024a00000000000024b0"), 0x0329a1), + (hex!("01222222223333333344444444550000024b00000000000024c0"), 0x032a41), + (hex!("01222222223333333344444444550000024c00000000000024d0"), 0x032ae1), + (hex!("01222222223333333344444444550000024d00000000000024e0"), 0x032b81), + (hex!("01222222223333333344444444550000024d0000000000004070"), 0x032c21), + (hex!("01222222223333333344444444550000024e00000000000024f0"), 0x032cc1), + (hex!("01222222223333333344444444550000024e00000000000066a0"), 0x032d61), + (hex!("01222222223333333344444444550000024e0000000000006ab0"), 0x032e01), + (hex!("01222222223333333344444444550000024f0000000000002500"), 0x032ea1), + (hex!("0122222222333333334444444455000002500000000000002510"), 0x032f41), + (hex!("0122222222333333334444444455000002510000000000002520"), 0x032fe1), + (hex!("0122222222333333334444444455000002510000000000007320"), 0x033081), + (hex!("0122222222333333334444444455000002520000000000002530"), 0x033121), + (hex!("0122222222333333334444444455000002520000000000006410"), 0x0331c1), + (hex!("0122222222333333334444444455000002530000000000002540"), 0x033261), + (hex!("0122222222333333334444444455000002530000000000005110"), 0x033301), + (hex!("0122222222333333334444444455000002540000000000002550"), 0x0333a1), + (hex!("01222222223333333344444444550000025400000000000040c0"), 0x033441), + (hex!("0122222222333333334444444455000002540000000000006a40"), 0x0334e1), + (hex!("0122222222333333334444444455000002550000000000002560"), 0x033581), + (hex!("0122222222333333334444444455000002550000000000005190"), 0x033621), + (hex!("0122222222333333334444444455000002560000000000002570"), 0x0336c1), + (hex!("01222222223333333344444444550000025600000000000061f0"), 0x033761), + (hex!("0122222222333333334444444455000002570000000000002580"), 0x033801), + (hex!("0122222222333333334444444455000002580000000000002590"), 0x0338a1), + (hex!("01222222223333333344444444550000025800000000000043d0"), 0x033941), + (hex!("01222222223333333344444444550000025900000000000025a0"), 0x0339e1), + (hex!("0122222222333333334444444455000002590000000000006bb0"), 0x033a81), + (hex!("01222222223333333344444444550000025a00000000000025b0"), 0x033b21), + (hex!("01222222223333333344444444550000025a0000000000005fb0"), 0x033bc1), + (hex!("01222222223333333344444444550000025a00000000000064c0"), 0x033c61), + (hex!("01222222223333333344444444550000025b00000000000025c0"), 0x033d01), + (hex!("01222222223333333344444444550000025b0000000000005c10"), 0x033da1), + (hex!("01222222223333333344444444550000025c00000000000025d0"), 0x033e41), + (hex!("01222222223333333344444444550000025c0000000000007d00"), 0x033ee1), + (hex!("01222222223333333344444444550000025d00000000000025e0"), 0x033f81), + (hex!("01222222223333333344444444550000025e00000000000025f0"), 0x034021), + (hex!("01222222223333333344444444550000025e00000000000045e0"), 0x0340c1), + (hex!("01222222223333333344444444550000025e0000000000006ee0"), 0x034161), + (hex!("01222222223333333344444444550000025f0000000000002600"), 0x034201), + (hex!("01222222223333333344444444550000025f00000000000050b0"), 0x0342a1), + (hex!("01222222223333333344444444550000025f0000000000007690"), 0x034341), + (hex!("0122222222333333334444444455000002600000000000002610"), 0x0343e1), + (hex!("0122222222333333334444444455000002600000000000007b60"), 0x034481), + (hex!("0122222222333333334444444455000002610000000000002620"), 0x034521), + (hex!("0122222222333333334444444455000002620000000000002630"), 0x0345c1), + (hex!("0122222222333333334444444455000002630000000000002640"), 0x034661), + (hex!("0122222222333333334444444455000002640000000000002650"), 0x034701), + (hex!("0122222222333333334444444455000002650000000000002660"), 0x0347a1), + (hex!("0122222222333333334444444455000002650000000000006180"), 0x034841), + (hex!("0122222222333333334444444455000002660000000000002670"), 0x0348e1), + (hex!("0122222222333333334444444455000002660000000000005430"), 0x034981), + (hex!("0122222222333333334444444455000002660000000000007a60"), 0x034a21), + (hex!("0122222222333333334444444455000002670000000000002680"), 0x034ac1), + (hex!("01222222223333333344444444550000026700000000000077f0"), 0x034b61), + (hex!("0122222222333333334444444455000002680000000000002690"), 0x034c01), + (hex!("01222222223333333344444444550000026900000000000026a0"), 0x034ca1), + (hex!("01222222223333333344444444550000026a00000000000026b0"), 0x034d41), + (hex!("01222222223333333344444444550000026a0000000000007530"), 0x034de1), + (hex!("01222222223333333344444444550000026b00000000000026c0"), 0x034e81), + (hex!("01222222223333333344444444550000026b00000000000058b0"), 0x034f21), + (hex!("01222222223333333344444444550000026b00000000000066b0"), 0x034fc1), + (hex!("01222222223333333344444444550000026b0000000000006b10"), 0x035061), + (hex!("01222222223333333344444444550000026c00000000000026d0"), 0x035101), + (hex!("01222222223333333344444444550000026d00000000000026e0"), 0x0351a1), + (hex!("01222222223333333344444444550000026d0000000000004210"), 0x035241), + (hex!("01222222223333333344444444550000026d0000000000005490"), 0x0352e1), + (hex!("01222222223333333344444444550000026d0000000000005e60"), 0x035381), + (hex!("01222222223333333344444444550000026d00000000000068e0"), 0x035421), + (hex!("01222222223333333344444444550000026d0000000000007020"), 0x0354c1), + (hex!("01222222223333333344444444550000026d0000000000007300"), 0x035561), + (hex!("01222222223333333344444444550000026e00000000000026f0"), 0x035601), + (hex!("01222222223333333344444444550000026f0000000000002700"), 0x0356a1), + (hex!("01222222223333333344444444550000026f0000000000004910"), 0x035741), + (hex!("0122222222333333334444444455000002700000000000002710"), 0x0357e1), + (hex!("0122222222333333334444444455000002710000000000002720"), 0x035881), + (hex!("01222222223333333344444444550000027100000000000050c0"), 0x035921), + (hex!("0122222222333333334444444455000002720000000000002730"), 0x0359c1), + (hex!("0122222222333333334444444455000002730000000000002740"), 0x035a61), + (hex!("0122222222333333334444444455000002740000000000002750"), 0x035b01), + (hex!("0122222222333333334444444455000002740000000000007490"), 0x035ba1), + (hex!("0122222222333333334444444455000002750000000000002760"), 0x035c41), + (hex!("0122222222333333334444444455000002760000000000002770"), 0x035ce1), + (hex!("0122222222333333334444444455000002760000000000004790"), 0x035d81), + (hex!("0122222222333333334444444455000002770000000000002780"), 0x035e21), + (hex!("01222222223333333344444444550000027700000000000050a0"), 0x035ec1), + (hex!("0122222222333333334444444455000002780000000000002790"), 0x035f61), + (hex!("0122222222333333334444444455000002780000000000004330"), 0x036001), + (hex!("0122222222333333334444444455000002780000000000006b00"), 0x0360a1), + (hex!("01222222223333333344444444550000027900000000000027a0"), 0x036141), + (hex!("01222222223333333344444444550000027a00000000000027b0"), 0x0361e1), + (hex!("01222222223333333344444444550000027b00000000000027c0"), 0x036281), + (hex!("01222222223333333344444444550000027b0000000000004930"), 0x036321), + (hex!("01222222223333333344444444550000027b0000000000006250"), 0x0363c1), + (hex!("01222222223333333344444444550000027c00000000000027d0"), 0x036461), + (hex!("01222222223333333344444444550000027d00000000000027e0"), 0x036501), + (hex!("01222222223333333344444444550000027d0000000000005ce0"), 0x0365a1), + (hex!("01222222223333333344444444550000027d0000000000005fe0"), 0x036641), + (hex!("01222222223333333344444444550000027e00000000000027f0"), 0x0366e1), + (hex!("01222222223333333344444444550000027f0000000000002800"), 0x036781), + (hex!("01222222223333333344444444550000027f0000000000003e90"), 0x036821), + (hex!("01222222223333333344444444550000027f0000000000007910"), 0x0368c1), + (hex!("0122222222333333334444444455000002800000000000002810"), 0x036961), + (hex!("0122222222333333334444444455000002800000000000004990"), 0x036a01), + (hex!("0122222222333333334444444455000002800000000000006160"), 0x036aa1), + (hex!("0122222222333333334444444455000002800000000000006740"), 0x036b41), + (hex!("0122222222333333334444444455000002810000000000002820"), 0x036be1), + (hex!("0122222222333333334444444455000002820000000000002830"), 0x036c81), + (hex!("0122222222333333334444444455000002820000000000005170"), 0x036d21), + (hex!("0122222222333333334444444455000002830000000000002840"), 0x036dc1), + (hex!("0122222222333333334444444455000002840000000000002850"), 0x036e61), + (hex!("0122222222333333334444444455000002840000000000004810"), 0x036f01), + (hex!("0122222222333333334444444455000002840000000000006aa0"), 0x036fa1), + (hex!("0122222222333333334444444455000002850000000000002860"), 0x037041), + (hex!("0122222222333333334444444455000002860000000000002870"), 0x0370e1), + (hex!("0122222222333333334444444455000002860000000000005080"), 0x037181), + (hex!("0122222222333333334444444455000002870000000000002880"), 0x037221), + (hex!("0122222222333333334444444455000002870000000000004e60"), 0x0372c1), + (hex!("0122222222333333334444444455000002880000000000002890"), 0x037361), + (hex!("0122222222333333334444444455000002880000000000005060"), 0x037401), + (hex!("0122222222333333334444444455000002880000000000006f20"), 0x0374a1), + (hex!("01222222223333333344444444550000028900000000000028a0"), 0x037541), + (hex!("01222222223333333344444444550000028900000000000047e0"), 0x0375e1), + (hex!("01222222223333333344444444550000028a00000000000028b0"), 0x037681), + (hex!("01222222223333333344444444550000028a0000000000005ab0"), 0x037721), + (hex!("01222222223333333344444444550000028a0000000000007130"), 0x0377c1), + (hex!("01222222223333333344444444550000028a0000000000007660"), 0x037861), + (hex!("01222222223333333344444444550000028b00000000000028c0"), 0x037901), + (hex!("01222222223333333344444444550000028b00000000000054e0"), 0x0379a1), + (hex!("01222222223333333344444444550000028c00000000000028d0"), 0x037a41), + (hex!("01222222223333333344444444550000028c00000000000046f0"), 0x037ae1), + (hex!("01222222223333333344444444550000028c00000000000061a0"), 0x037b81), + (hex!("01222222223333333344444444550000028d00000000000028e0"), 0x037c21), + (hex!("01222222223333333344444444550000028e00000000000028f0"), 0x037cc1), + (hex!("01222222223333333344444444550000028e0000000000004130"), 0x037d61), + (hex!("01222222223333333344444444550000028f0000000000002900"), 0x037e01), + (hex!("01222222223333333344444444550000028f0000000000007510"), 0x037ea1), + (hex!("0122222222333333334444444455000002900000000000002910"), 0x037f41), + (hex!("0122222222333333334444444455000002900000000000004a40"), 0x037fe1), + (hex!("0122222222333333334444444455000002910000000000002920"), 0x038081), + (hex!("0122222222333333334444444455000002920000000000002930"), 0x038121), + (hex!("0122222222333333334444444455000002920000000000004e90"), 0x0381c1), + (hex!("0122222222333333334444444455000002930000000000002940"), 0x038261), + (hex!("0122222222333333334444444455000002930000000000006880"), 0x038301), + (hex!("0122222222333333334444444455000002940000000000002950"), 0x0383a1), + (hex!("0122222222333333334444444455000002940000000000007bc0"), 0x038441), + (hex!("0122222222333333334444444455000002950000000000002960"), 0x0384e1), + (hex!("0122222222333333334444444455000002960000000000002970"), 0x038581), + (hex!("01222222223333333344444444550000029600000000000059d0"), 0x038621), + (hex!("0122222222333333334444444455000002970000000000002980"), 0x0386c1), + (hex!("0122222222333333334444444455000002970000000000004a50"), 0x038761), + (hex!("0122222222333333334444444455000002970000000000005f20"), 0x038801), + (hex!("01222222223333333344444444550000029700000000000068d0"), 0x0388a1), + (hex!("0122222222333333334444444455000002980000000000002990"), 0x038941), + (hex!("0122222222333333334444444455000002980000000000004370"), 0x0389e1), + (hex!("0122222222333333334444444455000002980000000000004420"), 0x038a81), + (hex!("01222222223333333344444444550000029900000000000029a0"), 0x038b21), + (hex!("01222222223333333344444444550000029a00000000000029b0"), 0x038bc1), + (hex!("01222222223333333344444444550000029a0000000000006010"), 0x038c61), + (hex!("01222222223333333344444444550000029a0000000000006980"), 0x038d01), + (hex!("01222222223333333344444444550000029b00000000000029c0"), 0x038da1), + (hex!("01222222223333333344444444550000029c00000000000029d0"), 0x038e41), + (hex!("01222222223333333344444444550000029c0000000000007480"), 0x038ee1), + (hex!("01222222223333333344444444550000029d00000000000029e0"), 0x038f81), + (hex!("01222222223333333344444444550000029d0000000000005030"), 0x039021), + (hex!("01222222223333333344444444550000029d0000000000007780"), 0x0390c1), + (hex!("01222222223333333344444444550000029d0000000000007a50"), 0x039161), + (hex!("01222222223333333344444444550000029e00000000000029f0"), 0x039201), + (hex!("01222222223333333344444444550000029e00000000000074b0"), 0x0392a1), + (hex!("01222222223333333344444444550000029f0000000000002a00"), 0x039341), + (hex!("0122222222333333334444444455000002a00000000000002a10"), 0x0393e1), + (hex!("0122222222333333334444444455000002a10000000000002a20"), 0x039481), + (hex!("0122222222333333334444444455000002a20000000000002a30"), 0x039521), + (hex!("0122222222333333334444444455000002a20000000000004c50"), 0x0395c1), + (hex!("0122222222333333334444444455000002a20000000000006f10"), 0x039661), + (hex!("0122222222333333334444444455000002a30000000000002a40"), 0x039701), + (hex!("0122222222333333334444444455000002a40000000000002a50"), 0x0397a1), + (hex!("0122222222333333334444444455000002a40000000000005d60"), 0x039841), + (hex!("0122222222333333334444444455000002a50000000000002a60"), 0x0398e1), + (hex!("0122222222333333334444444455000002a50000000000005440"), 0x039981), + (hex!("0122222222333333334444444455000002a50000000000005890"), 0x039a21), + (hex!("0122222222333333334444444455000002a60000000000002a70"), 0x039ac1), + (hex!("0122222222333333334444444455000002a70000000000002a80"), 0x039b61), + (hex!("0122222222333333334444444455000002a700000000000054a0"), 0x039c01), + (hex!("0122222222333333334444444455000002a70000000000007280"), 0x039ca1), + (hex!("0122222222333333334444444455000002a80000000000002a90"), 0x039d41), + (hex!("0122222222333333334444444455000002a90000000000002aa0"), 0x039de1), + (hex!("0122222222333333334444444455000002aa0000000000002ab0"), 0x039e81), + (hex!("0122222222333333334444444455000002ab0000000000002ac0"), 0x039f21), + (hex!("0122222222333333334444444455000002ab0000000000006c90"), 0x039fc1), + (hex!("0122222222333333334444444455000002ac0000000000002ad0"), 0x03a061), + (hex!("0122222222333333334444444455000002ac0000000000006db0"), 0x03a101), + (hex!("0122222222333333334444444455000002ad0000000000002ae0"), 0x03a1a1), + (hex!("0122222222333333334444444455000002ad00000000000065e0"), 0x03a241), + (hex!("0122222222333333334444444455000002ad0000000000007b40"), 0x03a2e1), + (hex!("0122222222333333334444444455000002ae0000000000002af0"), 0x03a381), + (hex!("0122222222333333334444444455000002ae0000000000004d20"), 0x03a421), + (hex!("0122222222333333334444444455000002ae0000000000006f30"), 0x03a4c1), + (hex!("0122222222333333334444444455000002af0000000000002b00"), 0x03a561), + (hex!("0122222222333333334444444455000002b00000000000002b10"), 0x03a601), + (hex!("0122222222333333334444444455000002b00000000000004560"), 0x03a6a1), + (hex!("0122222222333333334444444455000002b00000000000005800"), 0x03a741), + (hex!("0122222222333333334444444455000002b00000000000005a60"), 0x03a7e1), + (hex!("0122222222333333334444444455000002b10000000000002b20"), 0x03a881), + (hex!("0122222222333333334444444455000002b10000000000007b30"), 0x03a921), + (hex!("0122222222333333334444444455000002b20000000000002b30"), 0x03a9c1), + (hex!("0122222222333333334444444455000002b20000000000004440"), 0x03aa61), + (hex!("0122222222333333334444444455000002b20000000000004f80"), 0x03ab01), + (hex!("0122222222333333334444444455000002b20000000000005020"), 0x03aba1), + (hex!("0122222222333333334444444455000002b30000000000002b40"), 0x03ac41), + (hex!("0122222222333333334444444455000002b40000000000002b50"), 0x03ace1), + (hex!("0122222222333333334444444455000002b50000000000002b60"), 0x03ad81), + (hex!("0122222222333333334444444455000002b500000000000059e0"), 0x03ae21), + (hex!("0122222222333333334444444455000002b60000000000002b70"), 0x03aec1), + (hex!("0122222222333333334444444455000002b70000000000002b80"), 0x03af61), + (hex!("0122222222333333334444444455000002b80000000000002b90"), 0x03b001), + (hex!("0122222222333333334444444455000002b80000000000004590"), 0x03b0a1), + (hex!("0122222222333333334444444455000002b800000000000047d0"), 0x03b141), + (hex!("0122222222333333334444444455000002b80000000000006030"), 0x03b1e1), + (hex!("0122222222333333334444444455000002b80000000000006a20"), 0x03b281), + (hex!("0122222222333333334444444455000002b80000000000006a90"), 0x03b321), + (hex!("0122222222333333334444444455000002b90000000000002ba0"), 0x03b3c1), + (hex!("0122222222333333334444444455000002ba0000000000002bb0"), 0x03b461), + (hex!("0122222222333333334444444455000002ba0000000000006e80"), 0x03b501), + (hex!("0122222222333333334444444455000002bb0000000000002bc0"), 0x03b5a1), + (hex!("0122222222333333334444444455000002bc0000000000002bd0"), 0x03b641), + (hex!("0122222222333333334444444455000002bc0000000000004b30"), 0x03b6e1), + (hex!("0122222222333333334444444455000002bd0000000000002be0"), 0x03b781), + (hex!("0122222222333333334444444455000002bd0000000000005e10"), 0x03b821), + (hex!("0122222222333333334444444455000002be0000000000002bf0"), 0x03b8c1), + (hex!("0122222222333333334444444455000002bf0000000000002c00"), 0x03b961), + (hex!("0122222222333333334444444455000002c00000000000002c10"), 0x03ba01), + (hex!("0122222222333333334444444455000002c10000000000002c20"), 0x03baa1), + (hex!("0122222222333333334444444455000002c10000000000003ef0"), 0x03bb41), + (hex!("0122222222333333334444444455000002c20000000000002c30"), 0x03bbe1), + (hex!("0122222222333333334444444455000002c200000000000056e0"), 0x03bc81), + (hex!("0122222222333333334444444455000002c30000000000002c40"), 0x03bd21), + (hex!("0122222222333333334444444455000002c30000000000004b60"), 0x03bdc1), + (hex!("0122222222333333334444444455000002c40000000000002c50"), 0x03be61), + (hex!("0122222222333333334444444455000002c400000000000045f0"), 0x03bf01), + (hex!("0122222222333333334444444455000002c40000000000005290"), 0x03bfa1), + (hex!("0122222222333333334444444455000002c50000000000002c60"), 0x03c041), + (hex!("0122222222333333334444444455000002c60000000000002c70"), 0x03c0e1), + (hex!("0122222222333333334444444455000002c60000000000006ae0"), 0x03c181), + (hex!("0122222222333333334444444455000002c70000000000002c80"), 0x03c221), + (hex!("0122222222333333334444444455000002c70000000000005680"), 0x03c2c1), + (hex!("0122222222333333334444444455000002c70000000000006e10"), 0x03c361), + (hex!("0122222222333333334444444455000002c80000000000002c90"), 0x03c401), + (hex!("0122222222333333334444444455000002c90000000000002ca0"), 0x03c4a1), + (hex!("0122222222333333334444444455000002ca0000000000002cb0"), 0x03c541), + (hex!("0122222222333333334444444455000002cb0000000000002cc0"), 0x03c5e1), + (hex!("0122222222333333334444444455000002cc0000000000002cd0"), 0x03c681), + (hex!("0122222222333333334444444455000002cc0000000000005b50"), 0x03c721), + (hex!("0122222222333333334444444455000002cd0000000000002ce0"), 0x03c7c1), + (hex!("0122222222333333334444444455000002ce0000000000002cf0"), 0x03c861), + (hex!("0122222222333333334444444455000002ce00000000000043f0"), 0x03c901), + (hex!("0122222222333333334444444455000002ce0000000000006420"), 0x03c9a1), + (hex!("0122222222333333334444444455000002cf0000000000002d00"), 0x03ca41), + (hex!("0122222222333333334444444455000002d00000000000002d10"), 0x03cae1), + (hex!("0122222222333333334444444455000002d10000000000002d20"), 0x03cb81), + (hex!("0122222222333333334444444455000002d10000000000005370"), 0x03cc21), + (hex!("0122222222333333334444444455000002d20000000000002d30"), 0x03ccc1), + (hex!("0122222222333333334444444455000002d20000000000005ef0"), 0x03cd61), + (hex!("0122222222333333334444444455000002d20000000000006570"), 0x03ce01), + (hex!("0122222222333333334444444455000002d30000000000002d40"), 0x03cea1), + (hex!("0122222222333333334444444455000002d30000000000007360"), 0x03cf41), + (hex!("0122222222333333334444444455000002d40000000000002d50"), 0x03cfe1), + (hex!("0122222222333333334444444455000002d400000000000079a0"), 0x03d081), + (hex!("0122222222333333334444444455000002d50000000000002d60"), 0x03d121), + (hex!("0122222222333333334444444455000002d50000000000004250"), 0x03d1c1), + (hex!("0122222222333333334444444455000002d50000000000006050"), 0x03d261), + (hex!("0122222222333333334444444455000002d60000000000002d70"), 0x03d301), + (hex!("0122222222333333334444444455000002d60000000000007080"), 0x03d3a1), + (hex!("0122222222333333334444444455000002d70000000000002d80"), 0x03d441), + (hex!("0122222222333333334444444455000002d80000000000002d90"), 0x03d4e1), + (hex!("0122222222333333334444444455000002d80000000000007110"), 0x03d581), + (hex!("0122222222333333334444444455000002d800000000000073c0"), 0x03d621), + (hex!("0122222222333333334444444455000002d800000000000075a0"), 0x03d6c1), + (hex!("0122222222333333334444444455000002d90000000000002da0"), 0x03d761), + (hex!("0122222222333333334444444455000002d90000000000004860"), 0x03d801), + (hex!("0122222222333333334444444455000002d90000000000006b60"), 0x03d8a1), + (hex!("0122222222333333334444444455000002da0000000000002db0"), 0x03d941), + (hex!("0122222222333333334444444455000002da0000000000006630"), 0x03d9e1), + (hex!("0122222222333333334444444455000002db0000000000002dc0"), 0x03da81), + (hex!("0122222222333333334444444455000002dc0000000000002dd0"), 0x03db21), + (hex!("0122222222333333334444444455000002dc0000000000004830"), 0x03dbc1), + (hex!("0122222222333333334444444455000002dd0000000000002de0"), 0x03dc61), + (hex!("0122222222333333334444444455000002de0000000000002df0"), 0x03dd01), + (hex!("0122222222333333334444444455000002de0000000000004f00"), 0x03dda1), + (hex!("0122222222333333334444444455000002df0000000000002e00"), 0x03de41), + (hex!("0122222222333333334444444455000002e00000000000002e10"), 0x03dee1), + (hex!("0122222222333333334444444455000002e10000000000002e20"), 0x03df81), + (hex!("0122222222333333334444444455000002e10000000000006e90"), 0x03e021), + (hex!("0122222222333333334444444455000002e20000000000002e30"), 0x03e0c1), + (hex!("0122222222333333334444444455000002e200000000000053e0"), 0x03e161), + (hex!("0122222222333333334444444455000002e30000000000002e40"), 0x03e201), + (hex!("0122222222333333334444444455000002e30000000000006020"), 0x03e2a1), + (hex!("0122222222333333334444444455000002e30000000000006540"), 0x03e341), + (hex!("0122222222333333334444444455000002e40000000000002e50"), 0x03e3e1), + (hex!("0122222222333333334444444455000002e50000000000002e60"), 0x03e481), + (hex!("0122222222333333334444444455000002e50000000000005180"), 0x03e521), + (hex!("0122222222333333334444444455000002e50000000000007bf0"), 0x03e5c1), + (hex!("0122222222333333334444444455000002e60000000000002e70"), 0x03e661), + (hex!("0122222222333333334444444455000002e60000000000005350"), 0x03e701), + (hex!("0122222222333333334444444455000002e60000000000007960"), 0x03e7a1), + (hex!("0122222222333333334444444455000002e70000000000002e80"), 0x03e841), + (hex!("0122222222333333334444444455000002e80000000000002e90"), 0x03e8e1), + (hex!("0122222222333333334444444455000002e90000000000002ea0"), 0x03e981), + (hex!("0122222222333333334444444455000002ea0000000000002eb0"), 0x03ea21), + (hex!("0122222222333333334444444455000002eb0000000000002ec0"), 0x03eac1), + (hex!("0122222222333333334444444455000002ec0000000000002ed0"), 0x03eb61), + (hex!("0122222222333333334444444455000002ec0000000000006c10"), 0x03ec01), + (hex!("0122222222333333334444444455000002ed0000000000002ee0"), 0x03eca1), + (hex!("0122222222333333334444444455000002ed0000000000005590"), 0x03ed41), + (hex!("0122222222333333334444444455000002ed0000000000005cd0"), 0x03ede1), + (hex!("0122222222333333334444444455000002ed0000000000006910"), 0x03ee81), + (hex!("0122222222333333334444444455000002ee0000000000002ef0"), 0x03ef21), + (hex!("0122222222333333334444444455000002ef0000000000002f00"), 0x03efc1), + (hex!("0122222222333333334444444455000002ef0000000000004ed0"), 0x03f061), + (hex!("0122222222333333334444444455000002f00000000000002f10"), 0x03f101), + (hex!("0122222222333333334444444455000002f00000000000004cf0"), 0x03f1a1), + (hex!("0122222222333333334444444455000002f00000000000005d10"), 0x03f241), + (hex!("0122222222333333334444444455000002f00000000000006860"), 0x03f2e1), + (hex!("0122222222333333334444444455000002f00000000000006b50"), 0x03f381), + (hex!("0122222222333333334444444455000002f00000000000007100"), 0x03f421), + (hex!("0122222222333333334444444455000002f00000000000007aa0"), 0x03f4c1), + (hex!("0122222222333333334444444455000002f10000000000002f20"), 0x03f561), + (hex!("0122222222333333334444444455000002f20000000000002f30"), 0x03f601), + (hex!("0122222222333333334444444455000002f200000000000044b0"), 0x03f6a1), + (hex!("0122222222333333334444444455000002f30000000000002f40"), 0x03f741), + (hex!("0122222222333333334444444455000002f300000000000075b0"), 0x03f7e1), + (hex!("0122222222333333334444444455000002f40000000000002f50"), 0x03f881), + (hex!("0122222222333333334444444455000002f400000000000060f0"), 0x03f921), + (hex!("0122222222333333334444444455000002f50000000000002f60"), 0x03f9c1), + (hex!("0122222222333333334444444455000002f50000000000007210"), 0x03fa61), + (hex!("0122222222333333334444444455000002f60000000000002f70"), 0x03fb01), + (hex!("0122222222333333334444444455000002f60000000000006610"), 0x03fba1), + (hex!("0122222222333333334444444455000002f70000000000002f80"), 0x03fc41), + (hex!("0122222222333333334444444455000002f70000000000007560"), 0x03fce1), + (hex!("0122222222333333334444444455000002f80000000000002f90"), 0x03fd81), + (hex!("0122222222333333334444444455000002f80000000000006320"), 0x03fe21), + (hex!("0122222222333333334444444455000002f90000000000002fa0"), 0x03fec1), + (hex!("0122222222333333334444444455000002f90000000000006e50"), 0x03ff61), + (hex!("0122222222333333334444444455000002fa0000000000002fb0"), 0x040001), + (hex!("0122222222333333334444444455000002fb0000000000002fc0"), 0x0400a1), + (hex!("0122222222333333334444444455000002fb0000000000004780"), 0x040141), + (hex!("0122222222333333334444444455000002fc0000000000002fd0"), 0x0401e1), + (hex!("0122222222333333334444444455000002fd0000000000002fe0"), 0x040281), + (hex!("0122222222333333334444444455000002fd0000000000005600"), 0x040321), + (hex!("0122222222333333334444444455000002fd0000000000006c00"), 0x0403c1), + (hex!("0122222222333333334444444455000002fe0000000000002ff0"), 0x040461), + (hex!("0122222222333333334444444455000002ff0000000000003000"), 0x040501), + (hex!("0122222222333333334444444455000003000000000000003010"), 0x0405a1), + (hex!("0122222222333333334444444455000003000000000000004080"), 0x040641), + (hex!("0122222222333333334444444455000003010000000000003020"), 0x0406e1), + (hex!("0122222222333333334444444455000003010000000000006340"), 0x040781), + (hex!("0122222222333333334444444455000003020000000000003030"), 0x040821), + (hex!("0122222222333333334444444455000003020000000000005b00"), 0x0408c1), + (hex!("0122222222333333334444444455000003020000000000007b20"), 0x040961), + (hex!("0122222222333333334444444455000003030000000000003040"), 0x040a01), + (hex!("01222222223333333344444444550000030300000000000056b0"), 0x040aa1), + (hex!("0122222222333333334444444455000003030000000000006280"), 0x040b41), + (hex!("0122222222333333334444444455000003030000000000007ad0"), 0x040be1), + (hex!("0122222222333333334444444455000003040000000000003050"), 0x040c81), + (hex!("0122222222333333334444444455000003040000000000005c50"), 0x040d21), + (hex!("0122222222333333334444444455000003050000000000003060"), 0x040dc1), + (hex!("01222222223333333344444444550000030500000000000072e0"), 0x040e61), + (hex!("0122222222333333334444444455000003060000000000003070"), 0x040f01), + (hex!("0122222222333333334444444455000003060000000000004360"), 0x040fa1), + (hex!("0122222222333333334444444455000003060000000000004380"), 0x041041), + (hex!("0122222222333333334444444455000003060000000000004820"), 0x0410e1), + (hex!("0122222222333333334444444455000003060000000000006d10"), 0x041181), + (hex!("0122222222333333334444444455000003070000000000003080"), 0x041221), + (hex!("0122222222333333334444444455000003070000000000004450"), 0x0412c1), + (hex!("0122222222333333334444444455000003080000000000003090"), 0x041361), + (hex!("0122222222333333334444444455000003080000000000005ad0"), 0x041401), + (hex!("01222222223333333344444444550000030900000000000030a0"), 0x0414a1), + (hex!("01222222223333333344444444550000030a00000000000030b0"), 0x041541), + (hex!("01222222223333333344444444550000030a0000000000007760"), 0x0415e1), + (hex!("01222222223333333344444444550000030b00000000000030c0"), 0x041681), + (hex!("01222222223333333344444444550000030b0000000000007a80"), 0x041721), + (hex!("01222222223333333344444444550000030c00000000000030d0"), 0x0417c1), + (hex!("01222222223333333344444444550000030d00000000000030e0"), 0x041861), + (hex!("01222222223333333344444444550000030d0000000000003eb0"), 0x041901), + (hex!("01222222223333333344444444550000030e00000000000030f0"), 0x0419a1), + (hex!("01222222223333333344444444550000030f0000000000003100"), 0x041a41), + (hex!("01222222223333333344444444550000030f0000000000004690"), 0x041ae1), + (hex!("01222222223333333344444444550000030f0000000000006900"), 0x041b81), + (hex!("0122222222333333334444444455000003100000000000003110"), 0x041c21), + (hex!("01222222223333333344444444550000031000000000000058a0"), 0x041cc1), + (hex!("0122222222333333334444444455000003110000000000003120"), 0x041d61), + (hex!("0122222222333333334444444455000003110000000000004200"), 0x041e01), + (hex!("0122222222333333334444444455000003120000000000003130"), 0x041ea1), + (hex!("0122222222333333334444444455000003130000000000003140"), 0x041f41), + (hex!("0122222222333333334444444455000003130000000000004d50"), 0x041fe1), + (hex!("0122222222333333334444444455000003130000000000005400"), 0x042081), + (hex!("0122222222333333334444444455000003130000000000005520"), 0x042121), + (hex!("0122222222333333334444444455000003140000000000003150"), 0x0421c1), + (hex!("0122222222333333334444444455000003140000000000006450"), 0x042261), + (hex!("0122222222333333334444444455000003150000000000003160"), 0x042301), + (hex!("01222222223333333344444444550000031500000000000062d0"), 0x0423a1), + (hex!("0122222222333333334444444455000003160000000000003170"), 0x042441), + (hex!("0122222222333333334444444455000003160000000000004c40"), 0x0424e1), + (hex!("0122222222333333334444444455000003160000000000007c80"), 0x042581), + (hex!("0122222222333333334444444455000003170000000000003180"), 0x042621), + (hex!("0122222222333333334444444455000003170000000000004400"), 0x0426c1), + (hex!("0122222222333333334444444455000003170000000000005090"), 0x042761), + (hex!("0122222222333333334444444455000003170000000000006cb0"), 0x042801), + (hex!("0122222222333333334444444455000003180000000000003190"), 0x0428a1), + (hex!("0122222222333333334444444455000003180000000000006560"), 0x042941), + (hex!("01222222223333333344444444550000031900000000000031a0"), 0x0429e1), + (hex!("01222222223333333344444444550000031900000000000052d0"), 0x042a81), + (hex!("01222222223333333344444444550000031900000000000057e0"), 0x042b21), + (hex!("01222222223333333344444444550000031a00000000000031b0"), 0x042bc1), + (hex!("01222222223333333344444444550000031a00000000000071e0"), 0x042c61), + (hex!("01222222223333333344444444550000031b00000000000031c0"), 0x042d01), + (hex!("01222222223333333344444444550000031c00000000000031d0"), 0x042da1), + (hex!("01222222223333333344444444550000031c0000000000004480"), 0x042e41), + (hex!("01222222223333333344444444550000031c0000000000005790"), 0x042ee1), + (hex!("01222222223333333344444444550000031c0000000000007be0"), 0x042f81), + (hex!("01222222223333333344444444550000031d00000000000031e0"), 0x043021), + (hex!("01222222223333333344444444550000031d0000000000005560"), 0x0430c1), + (hex!("01222222223333333344444444550000031e00000000000031f0"), 0x043161), + (hex!("01222222223333333344444444550000031f0000000000003200"), 0x043201), + (hex!("01222222223333333344444444550000031f0000000000004190"), 0x0432a1), + (hex!("0122222222333333334444444455000003200000000000003210"), 0x043341), + (hex!("0122222222333333334444444455000003210000000000003220"), 0x0433e1), + (hex!("0122222222333333334444444455000003220000000000003230"), 0x043481), + (hex!("0122222222333333334444444455000003230000000000003240"), 0x043521), + (hex!("01222222223333333344444444550000032300000000000069d0"), 0x0435c1), + (hex!("0122222222333333334444444455000003240000000000003250"), 0x043661), + (hex!("0122222222333333334444444455000003250000000000003260"), 0x043701), + (hex!("01222222223333333344444444550000032500000000000042b0"), 0x0437a1), + (hex!("01222222223333333344444444550000032500000000000064e0"), 0x043841), + (hex!("0122222222333333334444444455000003260000000000003270"), 0x0438e1), + (hex!("0122222222333333334444444455000003270000000000003280"), 0x043981), + (hex!("0122222222333333334444444455000003270000000000005b20"), 0x043a21), + (hex!("0122222222333333334444444455000003270000000000006330"), 0x043ac1), + (hex!("0122222222333333334444444455000003270000000000006810"), 0x043b61), + (hex!("0122222222333333334444444455000003280000000000003290"), 0x043c01), + (hex!("01222222223333333344444444550000032900000000000032a0"), 0x043ca1), + (hex!("01222222223333333344444444550000032900000000000056f0"), 0x043d41), + (hex!("0122222222333333334444444455000003290000000000005e20"), 0x043de1), + (hex!("0122222222333333334444444455000003290000000000005e70"), 0x043e81), + (hex!("01222222223333333344444444550000032a00000000000032b0"), 0x043f21), + (hex!("01222222223333333344444444550000032b00000000000032c0"), 0x043fc1), + (hex!("01222222223333333344444444550000032b0000000000005500"), 0x044061), + (hex!("01222222223333333344444444550000032b0000000000005a20"), 0x044101), + (hex!("01222222223333333344444444550000032c00000000000032d0"), 0x0441a1), + (hex!("01222222223333333344444444550000032c0000000000004060"), 0x044241), + (hex!("01222222223333333344444444550000032c0000000000004760"), 0x0442e1), + (hex!("01222222223333333344444444550000032d00000000000032e0"), 0x044381), + (hex!("01222222223333333344444444550000032d00000000000068a0"), 0x044421), + (hex!("01222222223333333344444444550000032e00000000000032f0"), 0x0444c1), + (hex!("01222222223333333344444444550000032f0000000000003300"), 0x044561), + (hex!("0122222222333333334444444455000003300000000000003310"), 0x044601), + (hex!("0122222222333333334444444455000003300000000000006e40"), 0x0446a1), + (hex!("0122222222333333334444444455000003310000000000003320"), 0x044741), + (hex!("0122222222333333334444444455000003310000000000004620"), 0x0447e1), + (hex!("0122222222333333334444444455000003320000000000003330"), 0x044881), + (hex!("0122222222333333334444444455000003330000000000003340"), 0x044921), + (hex!("0122222222333333334444444455000003330000000000004b80"), 0x0449c1), + (hex!("0122222222333333334444444455000003340000000000003350"), 0x044a61), + (hex!("0122222222333333334444444455000003350000000000003360"), 0x044b01), + (hex!("0122222222333333334444444455000003360000000000003370"), 0x044ba1), + (hex!("0122222222333333334444444455000003370000000000003380"), 0x044c41), + (hex!("0122222222333333334444444455000003380000000000003390"), 0x044ce1), + (hex!("01222222223333333344444444550000033900000000000033a0"), 0x044d81), + (hex!("0122222222333333334444444455000003390000000000006b90"), 0x044e21), + (hex!("01222222223333333344444444550000033a00000000000033b0"), 0x044ec1), + (hex!("01222222223333333344444444550000033a0000000000007420"), 0x044f61), + (hex!("01222222223333333344444444550000033b00000000000033c0"), 0x045001), + (hex!("01222222223333333344444444550000033b0000000000007620"), 0x0450a1), + (hex!("01222222223333333344444444550000033c00000000000033d0"), 0x045141), + (hex!("01222222223333333344444444550000033c0000000000006b30"), 0x0451e1), + (hex!("01222222223333333344444444550000033d00000000000033e0"), 0x045281), + (hex!("01222222223333333344444444550000033e00000000000033f0"), 0x045321), + (hex!("01222222223333333344444444550000033e00000000000048b0"), 0x0453c1), + (hex!("01222222223333333344444444550000033e0000000000004e70"), 0x045461), + (hex!("01222222223333333344444444550000033f0000000000003400"), 0x045501), + (hex!("01222222223333333344444444550000033f0000000000006380"), 0x0455a1), + (hex!("0122222222333333334444444455000003400000000000003410"), 0x045641), + (hex!("0122222222333333334444444455000003410000000000003420"), 0x0456e1), + (hex!("0122222222333333334444444455000003410000000000006090"), 0x045781), + (hex!("0122222222333333334444444455000003420000000000003430"), 0x045821), + (hex!("01222222223333333344444444550000034200000000000073d0"), 0x0458c1), + (hex!("0122222222333333334444444455000003430000000000003440"), 0x045961), + (hex!("0122222222333333334444444455000003430000000000006370"), 0x045a01), + (hex!("01222222223333333344444444550000034300000000000075c0"), 0x045aa1), + (hex!("0122222222333333334444444455000003440000000000003450"), 0x045b41), + (hex!("0122222222333333334444444455000003450000000000003460"), 0x045be1), + (hex!("0122222222333333334444444455000003460000000000003470"), 0x045c81), + (hex!("01222222223333333344444444550000034600000000000055f0"), 0x045d21), + (hex!("0122222222333333334444444455000003470000000000003480"), 0x045dc1), + (hex!("0122222222333333334444444455000003470000000000003fe0"), 0x045e61), + (hex!("0122222222333333334444444455000003480000000000003490"), 0x045f01), + (hex!("0122222222333333334444444455000003480000000000007990"), 0x045fa1), + (hex!("01222222223333333344444444550000034900000000000034a0"), 0x046041), + (hex!("0122222222333333334444444455000003490000000000004410"), 0x0460e1), + (hex!("01222222223333333344444444550000034a00000000000034b0"), 0x046181), + (hex!("01222222223333333344444444550000034a00000000000062a0"), 0x046221), + (hex!("01222222223333333344444444550000034a0000000000007260"), 0x0462c1), + (hex!("01222222223333333344444444550000034b00000000000034c0"), 0x046361), + (hex!("01222222223333333344444444550000034b0000000000005760"), 0x046401), + (hex!("01222222223333333344444444550000034b0000000000006200"), 0x0464a1), + (hex!("01222222223333333344444444550000034c00000000000034d0"), 0x046541), + (hex!("01222222223333333344444444550000034d00000000000034e0"), 0x0465e1), + (hex!("01222222223333333344444444550000034e00000000000034f0"), 0x046681), + (hex!("01222222223333333344444444550000034e0000000000007790"), 0x046721), + (hex!("01222222223333333344444444550000034f0000000000003500"), 0x0467c1), + (hex!("0122222222333333334444444455000003500000000000003510"), 0x046861), + (hex!("0122222222333333334444444455000003510000000000003520"), 0x046901), + (hex!("0122222222333333334444444455000003520000000000003530"), 0x0469a1), + (hex!("01222222223333333344444444550000035200000000000056a0"), 0x046a41), + (hex!("0122222222333333334444444455000003530000000000003540"), 0x046ae1), + (hex!("0122222222333333334444444455000003540000000000003550"), 0x046b81), + (hex!("01222222223333333344444444550000035400000000000047b0"), 0x046c21), + (hex!("0122222222333333334444444455000003550000000000003560"), 0x046cc1), + (hex!("0122222222333333334444444455000003550000000000004500"), 0x046d61), + (hex!("0122222222333333334444444455000003560000000000003570"), 0x046e01), + (hex!("0122222222333333334444444455000003560000000000004fc0"), 0x046ea1), + (hex!("0122222222333333334444444455000003560000000000007160"), 0x046f41), + (hex!("0122222222333333334444444455000003560000000000007400"), 0x046fe1), + (hex!("0122222222333333334444444455000003570000000000003580"), 0x047081), + (hex!("0122222222333333334444444455000003580000000000003590"), 0x047121), + (hex!("0122222222333333334444444455000003580000000000005a80"), 0x0471c1), + (hex!("01222222223333333344444444550000035900000000000035a0"), 0x047261), + (hex!("01222222223333333344444444550000035900000000000073b0"), 0x047301), + (hex!("01222222223333333344444444550000035a00000000000035b0"), 0x0473a1), + (hex!("01222222223333333344444444550000035a0000000000004c20"), 0x047441), + (hex!("01222222223333333344444444550000035b00000000000035c0"), 0x0474e1), + (hex!("01222222223333333344444444550000035b0000000000005120"), 0x047581), + (hex!("01222222223333333344444444550000035c00000000000035d0"), 0x047621), + (hex!("01222222223333333344444444550000035c0000000000004300"), 0x0476c1), + (hex!("01222222223333333344444444550000035c0000000000005a40"), 0x047761), + (hex!("01222222223333333344444444550000035c0000000000006620"), 0x047801), + (hex!("01222222223333333344444444550000035c0000000000006ed0"), 0x0478a1), + (hex!("01222222223333333344444444550000035d00000000000035e0"), 0x047941), + (hex!("01222222223333333344444444550000035d0000000000005df0"), 0x0479e1), + (hex!("01222222223333333344444444550000035e00000000000035f0"), 0x047a81), + (hex!("01222222223333333344444444550000035f0000000000003600"), 0x047b21), + (hex!("01222222223333333344444444550000035f00000000000058d0"), 0x047bc1), + (hex!("0122222222333333334444444455000003600000000000003610"), 0x047c61), + (hex!("0122222222333333334444444455000003600000000000007b90"), 0x047d01), + (hex!("0122222222333333334444444455000003610000000000003620"), 0x047da1), + (hex!("0122222222333333334444444455000003610000000000006ad0"), 0x047e41), + (hex!("0122222222333333334444444455000003620000000000003630"), 0x047ee1), + (hex!("01222222223333333344444444550000036200000000000063a0"), 0x047f81), + (hex!("0122222222333333334444444455000003630000000000003640"), 0x048021), + (hex!("0122222222333333334444444455000003630000000000007250"), 0x0480c1), + (hex!("0122222222333333334444444455000003640000000000003650"), 0x048161), + (hex!("0122222222333333334444444455000003640000000000005510"), 0x048201), + (hex!("0122222222333333334444444455000003640000000000007850"), 0x0482a1), + (hex!("0122222222333333334444444455000003650000000000003660"), 0x048341), + (hex!("0122222222333333334444444455000003660000000000003670"), 0x0483e1), + (hex!("0122222222333333334444444455000003660000000000004650"), 0x048481), + (hex!("01222222223333333344444444550000036600000000000050d0"), 0x048521), + (hex!("0122222222333333334444444455000003660000000000006eb0"), 0x0485c1), + (hex!("0122222222333333334444444455000003670000000000003680"), 0x048661), + (hex!("01222222223333333344444444550000036700000000000071f0"), 0x048701), + (hex!("0122222222333333334444444455000003680000000000003690"), 0x0487a1), + (hex!("01222222223333333344444444550000036900000000000036a0"), 0x048841), + (hex!("0122222222333333334444444455000003690000000000005c70"), 0x0488e1), + (hex!("01222222223333333344444444550000036a00000000000036b0"), 0x048981), + (hex!("01222222223333333344444444550000036a00000000000071b0"), 0x048a21), + (hex!("01222222223333333344444444550000036b00000000000036c0"), 0x048ac1), + (hex!("01222222223333333344444444550000036b0000000000004670"), 0x048b61), + (hex!("01222222223333333344444444550000036c00000000000036d0"), 0x048c01), + (hex!("01222222223333333344444444550000036c0000000000004750"), 0x048ca1), + (hex!("01222222223333333344444444550000036c0000000000006fa0"), 0x048d41), + (hex!("01222222223333333344444444550000036d00000000000036e0"), 0x048de1), + (hex!("01222222223333333344444444550000036d0000000000003f70"), 0x048e81), + (hex!("01222222223333333344444444550000036d0000000000004b90"), 0x048f21), + (hex!("01222222223333333344444444550000036d00000000000057a0"), 0x048fc1), + (hex!("01222222223333333344444444550000036e00000000000036f0"), 0x049061), + (hex!("01222222223333333344444444550000036e00000000000075d0"), 0x049101), + (hex!("01222222223333333344444444550000036f0000000000003700"), 0x0491a1), + (hex!("0122222222333333334444444455000003700000000000003710"), 0x049241), + (hex!("0122222222333333334444444455000003700000000000005aa0"), 0x0492e1), + (hex!("0122222222333333334444444455000003710000000000003720"), 0x049381), + (hex!("0122222222333333334444444455000003710000000000005130"), 0x049421), + (hex!("0122222222333333334444444455000003710000000000006fc0"), 0x0494c1), + (hex!("0122222222333333334444444455000003710000000000007b00"), 0x049561), + (hex!("0122222222333333334444444455000003720000000000003730"), 0x049601), + (hex!("01222222223333333344444444550000037200000000000054d0"), 0x0496a1), + (hex!("0122222222333333334444444455000003730000000000003740"), 0x049741), + (hex!("0122222222333333334444444455000003730000000000004220"), 0x0497e1), + (hex!("0122222222333333334444444455000003740000000000003750"), 0x049881), + (hex!("0122222222333333334444444455000003740000000000004720"), 0x049921), + (hex!("0122222222333333334444444455000003750000000000003760"), 0x0499c1), + (hex!("0122222222333333334444444455000003750000000000004110"), 0x049a61), + (hex!("0122222222333333334444444455000003760000000000003770"), 0x049b01), + (hex!("0122222222333333334444444455000003770000000000003780"), 0x049ba1), + (hex!("0122222222333333334444444455000003780000000000003790"), 0x049c41), + (hex!("0122222222333333334444444455000003780000000000004b40"), 0x049ce1), + (hex!("0122222222333333334444444455000003780000000000005660"), 0x049d81), + (hex!("0122222222333333334444444455000003780000000000005ea0"), 0x049e21), + (hex!("01222222223333333344444444550000037900000000000037a0"), 0x049ec1), + (hex!("01222222223333333344444444550000037a00000000000037b0"), 0x049f61), + (hex!("01222222223333333344444444550000037b00000000000037c0"), 0x04a001), + (hex!("01222222223333333344444444550000037c00000000000037d0"), 0x04a0a1), + (hex!("01222222223333333344444444550000037c0000000000004340"), 0x04a141), + (hex!("01222222223333333344444444550000037c0000000000005230"), 0x04a1e1), + (hex!("01222222223333333344444444550000037d00000000000037e0"), 0x04a281), + (hex!("01222222223333333344444444550000037d00000000000051e0"), 0x04a321), + (hex!("01222222223333333344444444550000037e00000000000037f0"), 0x04a3c1), + (hex!("01222222223333333344444444550000037e0000000000004090"), 0x04a461), + (hex!("01222222223333333344444444550000037e0000000000005c20"), 0x04a501), + (hex!("01222222223333333344444444550000037f0000000000003800"), 0x04a5a1), + (hex!("0122222222333333334444444455000003800000000000003810"), 0x04a641), + (hex!("0122222222333333334444444455000003800000000000007630"), 0x04a6e1), + (hex!("0122222222333333334444444455000003810000000000003820"), 0x04a781), + (hex!("0122222222333333334444444455000003820000000000003830"), 0x04a821), + (hex!("0122222222333333334444444455000003820000000000004170"), 0x04a8c1), + (hex!("0122222222333333334444444455000003830000000000003840"), 0x04a961), + (hex!("0122222222333333334444444455000003840000000000003850"), 0x04aa01), + (hex!("0122222222333333334444444455000003850000000000003860"), 0x04aaa1), + (hex!("0122222222333333334444444455000003850000000000004180"), 0x04ab41), + (hex!("0122222222333333334444444455000003850000000000005c90"), 0x04abe1), + (hex!("0122222222333333334444444455000003850000000000005da0"), 0x04ac81), + (hex!("0122222222333333334444444455000003850000000000006ff0"), 0x04ad21), + (hex!("0122222222333333334444444455000003860000000000003870"), 0x04adc1), + (hex!("01222222223333333344444444550000038600000000000065c0"), 0x04ae61), + (hex!("0122222222333333334444444455000003870000000000003880"), 0x04af01), + (hex!("0122222222333333334444444455000003870000000000007cc0"), 0x04afa1), + (hex!("0122222222333333334444444455000003880000000000003890"), 0x04b041), + (hex!("01222222223333333344444444550000038900000000000038a0"), 0x04b0e1), + (hex!("01222222223333333344444444550000038a00000000000038b0"), 0x04b181), + (hex!("01222222223333333344444444550000038a00000000000073e0"), 0x04b221), + (hex!("01222222223333333344444444550000038b00000000000038c0"), 0x04b2c1), + (hex!("01222222223333333344444444550000038c00000000000038d0"), 0x04b361), + (hex!("01222222223333333344444444550000038d00000000000038e0"), 0x04b401), + (hex!("01222222223333333344444444550000038d00000000000069f0"), 0x04b4a1), + (hex!("01222222223333333344444444550000038d0000000000007680"), 0x04b541), + (hex!("01222222223333333344444444550000038e00000000000038f0"), 0x04b5e1), + (hex!("01222222223333333344444444550000038f0000000000003900"), 0x04b681), + (hex!("01222222223333333344444444550000038f00000000000045b0"), 0x04b721), + (hex!("01222222223333333344444444550000038f0000000000007180"), 0x04b7c1), + (hex!("0122222222333333334444444455000003900000000000003910"), 0x04b861), + (hex!("0122222222333333334444444455000003910000000000003920"), 0x04b901), + (hex!("0122222222333333334444444455000003910000000000004a20"), 0x04b9a1), + (hex!("0122222222333333334444444455000003920000000000003930"), 0x04ba41), + (hex!("01222222223333333344444444550000039200000000000059b0"), 0x04bae1), + (hex!("0122222222333333334444444455000003930000000000003940"), 0x04bb81), + (hex!("0122222222333333334444444455000003930000000000006cc0"), 0x04bc21), + (hex!("0122222222333333334444444455000003940000000000003950"), 0x04bcc1), + (hex!("01222222223333333344444444550000039400000000000056c0"), 0x04bd61), + (hex!("0122222222333333334444444455000003950000000000003960"), 0x04be01), + (hex!("0122222222333333334444444455000003950000000000004cc0"), 0x04bea1), + (hex!("0122222222333333334444444455000003950000000000007720"), 0x04bf41), + (hex!("0122222222333333334444444455000003960000000000003970"), 0x04bfe1), + (hex!("0122222222333333334444444455000003960000000000004da0"), 0x04c081), + (hex!("0122222222333333334444444455000003960000000000004df0"), 0x04c121), + (hex!("0122222222333333334444444455000003960000000000004f30"), 0x04c1c1), + (hex!("01222222223333333344444444550000039600000000000050f0"), 0x04c261), + (hex!("0122222222333333334444444455000003960000000000007940"), 0x04c301), + (hex!("0122222222333333334444444455000003970000000000003980"), 0x04c3a1), + (hex!("0122222222333333334444444455000003970000000000005850"), 0x04c441), + (hex!("0122222222333333334444444455000003970000000000007bd0"), 0x04c4e1), + (hex!("0122222222333333334444444455000003980000000000003990"), 0x04c581), + (hex!("0122222222333333334444444455000003980000000000004c00"), 0x04c621), + (hex!("0122222222333333334444444455000003980000000000005580"), 0x04c6c1), + (hex!("01222222223333333344444444550000039900000000000039a0"), 0x04c761), + (hex!("0122222222333333334444444455000003990000000000005820"), 0x04c801), + (hex!("01222222223333333344444444550000039a00000000000039b0"), 0x04c8a1), + (hex!("01222222223333333344444444550000039b00000000000039c0"), 0x04c941), + (hex!("01222222223333333344444444550000039b0000000000004c10"), 0x04c9e1), + (hex!("01222222223333333344444444550000039b0000000000006460"), 0x04ca81), + (hex!("01222222223333333344444444550000039c00000000000039d0"), 0x04cb21), + (hex!("01222222223333333344444444550000039d00000000000039e0"), 0x04cbc1), + (hex!("01222222223333333344444444550000039d00000000000044c0"), 0x04cc61), + (hex!("01222222223333333344444444550000039d00000000000049e0"), 0x04cd01), + (hex!("01222222223333333344444444550000039e00000000000039f0"), 0x04cda1), + (hex!("01222222223333333344444444550000039f0000000000003a00"), 0x04ce41), + (hex!("0122222222333333334444444455000003a00000000000003a10"), 0x04cee1), + (hex!("0122222222333333334444444455000003a10000000000003a20"), 0x04cf81), + (hex!("0122222222333333334444444455000003a10000000000006a80"), 0x04d021), + (hex!("0122222222333333334444444455000003a20000000000003a30"), 0x04d0c1), + (hex!("0122222222333333334444444455000003a200000000000062b0"), 0x04d161), + (hex!("0122222222333333334444444455000003a30000000000003a40"), 0x04d201), + (hex!("0122222222333333334444444455000003a30000000000006ce0"), 0x04d2a1), + (hex!("0122222222333333334444444455000003a40000000000003a50"), 0x04d341), + (hex!("0122222222333333334444444455000003a50000000000003a60"), 0x04d3e1), + (hex!("0122222222333333334444444455000003a60000000000003a70"), 0x04d481), + (hex!("0122222222333333334444444455000003a60000000000007750"), 0x04d521), + (hex!("0122222222333333334444444455000003a70000000000003a80"), 0x04d5c1), + (hex!("0122222222333333334444444455000003a70000000000005b10"), 0x04d661), + (hex!("0122222222333333334444444455000003a80000000000003a90"), 0x04d701), + (hex!("0122222222333333334444444455000003a80000000000006c20"), 0x04d7a1), + (hex!("0122222222333333334444444455000003a90000000000003aa0"), 0x04d841), + (hex!("0122222222333333334444444455000003a90000000000005b70"), 0x04d8e1), + (hex!("0122222222333333334444444455000003a900000000000070e0"), 0x04d981), + (hex!("0122222222333333334444444455000003aa0000000000003ab0"), 0x04da21), + (hex!("0122222222333333334444444455000003aa00000000000049f0"), 0x04dac1), + (hex!("0122222222333333334444444455000003aa0000000000004d60"), 0x04db61), + (hex!("0122222222333333334444444455000003ab0000000000003ac0"), 0x04dc01), + (hex!("0122222222333333334444444455000003ac0000000000003ad0"), 0x04dca1), + (hex!("0122222222333333334444444455000003ac0000000000004580"), 0x04dd41), + (hex!("0122222222333333334444444455000003ad0000000000003ae0"), 0x04dde1), + (hex!("0122222222333333334444444455000003ae0000000000003af0"), 0x04de81), + (hex!("0122222222333333334444444455000003af0000000000003b00"), 0x04df21), + (hex!("0122222222333333334444444455000003b00000000000003b10"), 0x04dfc1), + (hex!("0122222222333333334444444455000003b10000000000003b20"), 0x04e061), + (hex!("0122222222333333334444444455000003b10000000000003fd0"), 0x04e101), + (hex!("0122222222333333334444444455000003b20000000000003b30"), 0x04e1a1), + (hex!("0122222222333333334444444455000003b30000000000003b40"), 0x04e241), + (hex!("0122222222333333334444444455000003b40000000000003b50"), 0x04e2e1), + (hex!("0122222222333333334444444455000003b40000000000007450"), 0x04e381), + (hex!("0122222222333333334444444455000003b50000000000003b60"), 0x04e421), + (hex!("0122222222333333334444444455000003b60000000000003b70"), 0x04e4c1), + (hex!("0122222222333333334444444455000003b70000000000003b80"), 0x04e561), + (hex!("0122222222333333334444444455000003b70000000000006d50"), 0x04e601), + (hex!("0122222222333333334444444455000003b80000000000003b90"), 0x04e6a1), + (hex!("0122222222333333334444444455000003b800000000000057c0"), 0x04e741), + (hex!("0122222222333333334444444455000003b800000000000078a0"), 0x04e7e1), + (hex!("0122222222333333334444444455000003b90000000000003ba0"), 0x04e881), + (hex!("0122222222333333334444444455000003b90000000000006750"), 0x04e921), + (hex!("0122222222333333334444444455000003ba0000000000003bb0"), 0x04e9c1), + (hex!("0122222222333333334444444455000003ba0000000000007a10"), 0x04ea61), + (hex!("0122222222333333334444444455000003ba0000000000007a20"), 0x04eb01), + (hex!("0122222222333333334444444455000003bb0000000000003bc0"), 0x04eba1), + (hex!("0122222222333333334444444455000003bb0000000000005bc0"), 0x04ec41), + (hex!("0122222222333333334444444455000003bc0000000000003bd0"), 0x04ece1), + (hex!("0122222222333333334444444455000003bc0000000000005e80"), 0x04ed81), + (hex!("0122222222333333334444444455000003bc0000000000007ab0"), 0x04ee21), + (hex!("0122222222333333334444444455000003bd0000000000003be0"), 0x04eec1), + (hex!("0122222222333333334444444455000003bd00000000000049b0"), 0x04ef61), + (hex!("0122222222333333334444444455000003be0000000000003bf0"), 0x04f001), + (hex!("0122222222333333334444444455000003be0000000000005780"), 0x04f0a1), + (hex!("0122222222333333334444444455000003be0000000000007930"), 0x04f141), + (hex!("0122222222333333334444444455000003bf0000000000003c00"), 0x04f1e1), + (hex!("0122222222333333334444444455000003bf0000000000005de0"), 0x04f281), + (hex!("0122222222333333334444444455000003bf00000000000060b0"), 0x04f321), + (hex!("0122222222333333334444444455000003bf00000000000060c0"), 0x04f3c1), + (hex!("0122222222333333334444444455000003bf0000000000006a50"), 0x04f461), + (hex!("0122222222333333334444444455000003c00000000000003c10"), 0x04f501), + (hex!("0122222222333333334444444455000003c00000000000004030"), 0x04f5a1), + (hex!("0122222222333333334444444455000003c10000000000003c20"), 0x04f641), + (hex!("0122222222333333334444444455000003c20000000000003c30"), 0x04f6e1), + (hex!("0122222222333333334444444455000003c200000000000040b0"), 0x04f781), + (hex!("0122222222333333334444444455000003c30000000000003c40"), 0x04f821), + (hex!("0122222222333333334444444455000003c40000000000003c50"), 0x04f8c1), + (hex!("0122222222333333334444444455000003c40000000000005ba0"), 0x04f961), + (hex!("0122222222333333334444444455000003c50000000000003c60"), 0x04fa01), + (hex!("0122222222333333334444444455000003c60000000000003c70"), 0x04faa1), + (hex!("0122222222333333334444444455000003c70000000000003c80"), 0x04fb41), + (hex!("0122222222333333334444444455000003c70000000000004270"), 0x04fbe1), + (hex!("0122222222333333334444444455000003c80000000000003c90"), 0x04fc81), + (hex!("0122222222333333334444444455000003c80000000000006e70"), 0x04fd21), + (hex!("0122222222333333334444444455000003c90000000000003ca0"), 0x04fdc1), + (hex!("0122222222333333334444444455000003ca0000000000003cb0"), 0x04fe61), + (hex!("0122222222333333334444444455000003ca0000000000006e20"), 0x04ff01), + (hex!("0122222222333333334444444455000003ca0000000000007c20"), 0x04ffa1), + (hex!("0122222222333333334444444455000003cb0000000000003cc0"), 0x050041), + (hex!("0122222222333333334444444455000003cc0000000000003cd0"), 0x0500e1), + (hex!("0122222222333333334444444455000003cc0000000000006120"), 0x050181), + (hex!("0122222222333333334444444455000003cc0000000000007950"), 0x050221), + (hex!("0122222222333333334444444455000003cd0000000000003ce0"), 0x0502c1), + (hex!("0122222222333333334444444455000003ce0000000000003cf0"), 0x050361), + (hex!("0122222222333333334444444455000003cf0000000000003d00"), 0x050401), + (hex!("0122222222333333334444444455000003d00000000000003d10"), 0x0504a1), + (hex!("0122222222333333334444444455000003d10000000000003d20"), 0x050541), + (hex!("0122222222333333334444444455000003d10000000000005e50"), 0x0505e1), + (hex!("0122222222333333334444444455000003d10000000000007880"), 0x050681), + (hex!("0122222222333333334444444455000003d20000000000003d30"), 0x050721), + (hex!("0122222222333333334444444455000003d20000000000005d00"), 0x0507c1), + (hex!("0122222222333333334444444455000003d30000000000003d40"), 0x050861), + (hex!("0122222222333333334444444455000003d30000000000005d40"), 0x050901), + (hex!("0122222222333333334444444455000003d300000000000063f0"), 0x0509a1), + (hex!("0122222222333333334444444455000003d40000000000003d50"), 0x050a41), + (hex!("0122222222333333334444444455000003d40000000000005700"), 0x050ae1), + (hex!("0122222222333333334444444455000003d400000000000078f0"), 0x050b81), + (hex!("0122222222333333334444444455000003d50000000000003d60"), 0x050c21), + (hex!("0122222222333333334444444455000003d60000000000003d70"), 0x050cc1), + (hex!("0122222222333333334444444455000003d70000000000003d80"), 0x050d61), + (hex!("0122222222333333334444444455000003d80000000000003d90"), 0x050e01), + (hex!("0122222222333333334444444455000003d80000000000006690"), 0x050ea1), + (hex!("0122222222333333334444444455000003d90000000000003da0"), 0x050f41), + (hex!("0122222222333333334444444455000003d900000000000076d0"), 0x050fe1), + (hex!("0122222222333333334444444455000003da0000000000003db0"), 0x051081), + (hex!("0122222222333333334444444455000003db0000000000003dc0"), 0x051121), + (hex!("0122222222333333334444444455000003db0000000000004a30"), 0x0511c1), + (hex!("0122222222333333334444444455000003db0000000000005390"), 0x051261), + (hex!("0122222222333333334444444455000003dc0000000000003dd0"), 0x051301), + (hex!("0122222222333333334444444455000003dc0000000000006d60"), 0x0513a1), + (hex!("0122222222333333334444444455000003dd0000000000003de0"), 0x051441), + (hex!("0122222222333333334444444455000003de0000000000003df0"), 0x0514e1), + (hex!("0122222222333333334444444455000003df0000000000003e00"), 0x051581), + (hex!("0122222222333333334444444455000003df0000000000005240"), 0x051621), + (hex!("0122222222333333334444444455000003df0000000000005610"), 0x0516c1), + (hex!("0122222222333333334444444455000003e00000000000003e10"), 0x051761), + (hex!("0122222222333333334444444455000003e00000000000006500"), 0x051801), + (hex!("0122222222333333334444444455000003e10000000000003e20"), 0x0518a1), + (hex!("0122222222333333334444444455000003e10000000000006a10"), 0x051941), + (hex!("0122222222333333334444444455000003e10000000000007c10"), 0x0519e1), + (hex!("0122222222333333334444444455000003e20000000000003e30"), 0x051a81), + (hex!("0122222222333333334444444455000003e20000000000006310"), 0x051b21), + (hex!("0122222222333333334444444455000003e30000000000003e40"), 0x051bc1), + (hex!("0122222222333333334444444455000003e40000000000003e50"), 0x051c61), + (hex!("0122222222333333334444444455000003e40000000000006780"), 0x051d01), + (hex!("0122222222333333334444444455000003e40000000000007ce0"), 0x051da1), + (hex!("0122222222333333334444444455000003e50000000000003e60"), 0x051e41), + (hex!("0122222222333333334444444455000003e60000000000003e70"), 0x051ee1), + (hex!("0122222222333333334444444455000003e60000000000005040"), 0x051f81), + (hex!("0122222222333333334444444455000003e60000000000005bf0"), 0x052021), + (hex!("0122222222333333334444444455000003e70000000000003e80"), 0x0520c1), + (hex!("0122222222333333334444444455000003e70000000000003f50"), 0x052161), +]; diff --git a/pageserver/src/layered_repository/image_layer.rs b/pageserver/src/layered_repository/image_layer.rs index d0afce1549..08e635f073 100644 --- a/pageserver/src/layered_repository/image_layer.rs +++ b/pageserver/src/layered_repository/image_layer.rs @@ -16,40 +16,43 @@ //! Every image layer file consists of three parts: "summary", //! "index", and "values". The summary is a fixed size header at the //! beginning of the file, and it contains basic information about the -//! layer, and offsets to the other parts. The "index" is a serialized -//! HashMap, mapping from Key to an offset in the "values" part. The +//! layer, and offsets to the other parts. The "index" is a B-tree, +//! mapping from Key to an offset in the "values" part. The //! actual page images are stored in the "values" part. -//! -//! Only the "index" is loaded into memory by the load function. -//! When images are needed, they are read directly from disk. -//! use crate::config::PageServerConf; use crate::layered_repository::blob_io::{BlobCursor, BlobWriter, WriteBlobWriter}; -use crate::layered_repository::block_io::{BlockReader, FileBlockReader}; +use crate::layered_repository::block_io::{BlockBuf, BlockReader, FileBlockReader}; +use crate::layered_repository::disk_btree::{DiskBtreeBuilder, DiskBtreeReader, VisitDirection}; use crate::layered_repository::filename::{ImageFileName, PathOrConf}; use crate::layered_repository::storage_layer::{ Layer, ValueReconstructResult, ValueReconstructState, }; use crate::page_cache::PAGE_SZ; -use crate::repository::{Key, Value}; +use crate::repository::{Key, Value, KEY_SIZE}; use crate::virtual_file::VirtualFile; use crate::{ZTenantId, ZTimelineId}; use crate::{IMAGE_FILE_MAGIC, STORAGE_FORMAT_VERSION}; use anyhow::{bail, ensure, Context, Result}; use bytes::Bytes; +use hex; use log::*; use serde::{Deserialize, Serialize}; -use std::collections::HashMap; use std::fs; use std::io::Write; use std::io::{Seek, SeekFrom}; use std::ops::Range; use std::path::{Path, PathBuf}; -use std::sync::{RwLock, RwLockReadGuard, TryLockError}; +use std::sync::{RwLock, RwLockReadGuard}; use zenith_utils::bin_ser::BeSer; use zenith_utils::lsn::Lsn; +/// +/// Header stored in the beginning of the file +/// +/// After this comes the 'values' part, starting on block 1. After that, +/// the 'index' starts at the block indicated by 'index_start_blk' +/// #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] struct Summary { /// Magic value to identify this as a zenith image file. Always IMAGE_FILE_MAGIC. @@ -63,6 +66,9 @@ struct Summary { /// Block number where the 'index' part of the file begins. index_start_blk: u32, + /// Block within the 'index', where the B-tree root page is stored + index_root_blk: u32, + // the 'values' part starts after the summary header, on block 1. } impl From<&ImageLayer> for Summary { @@ -73,10 +79,10 @@ impl From<&ImageLayer> for Summary { tenantid: layer.tenantid, timelineid: layer.timelineid, key_range: layer.key_range.clone(), - lsn: layer.lsn, index_start_blk: 0, + index_root_blk: 0, } } } @@ -104,11 +110,9 @@ pub struct ImageLayerInner { /// If false, the 'index' has not been loaded into memory yet. loaded: bool, - /// offset of each value - index: HashMap, - // values copied from summary index_start_blk: u32, + index_root_blk: u32, /// Reader object for reading blocks from the file. (None if not loaded yet) file: Option>, @@ -147,21 +151,21 @@ impl Layer for ImageLayer { assert!(lsn_range.end >= self.lsn); let inner = self.load()?; - if let Some(&offset) = inner.index.get(&key) { - let buf = inner - .file - .as_ref() - .unwrap() - .block_cursor() - .read_blob(offset) - .with_context(|| { - format!( - "failed to read blob from data file {} at offset {}", - self.filename().display(), - offset - ) - })?; - let value = Bytes::from(buf); + + let file = inner.file.as_ref().unwrap(); + let tree_reader = DiskBtreeReader::new(inner.index_start_blk, inner.index_root_blk, file); + + let mut keybuf: [u8; KEY_SIZE] = [0u8; KEY_SIZE]; + key.write_to_byte_slice(&mut keybuf); + if let Some(offset) = tree_reader.get(&keybuf)? { + let blob = file.block_cursor().read_blob(offset).with_context(|| { + format!( + "failed to read value from data file {} at offset {}", + self.filename().display(), + offset + ) + })?; + let value = Bytes::from(blob); reconstruct_state.img = Some((self.lsn, value)); Ok(ValueReconstructResult::Complete) @@ -174,33 +178,6 @@ impl Layer for ImageLayer { todo!(); } - fn unload(&self) -> Result<()> { - // Unload the index. - // - // TODO: we should access the index directly from pages on the disk, - // using the buffer cache. This load/unload mechanism is really ad hoc. - - // FIXME: In debug mode, loading and unloading the index slows - // things down so much that you get timeout errors. At least - // with the test_parallel_copy test. So as an even more ad hoc - // stopgap fix for that, only unload every on average 10 - // checkpoint cycles. - use rand::RngCore; - if rand::thread_rng().next_u32() > (u32::MAX / 10) { - return Ok(()); - } - - let mut inner = match self.inner.try_write() { - Ok(inner) => inner, - Err(TryLockError::WouldBlock) => return Ok(()), - Err(TryLockError::Poisoned(_)) => panic!("ImageLayer lock was poisoned"), - }; - inner.index = HashMap::default(); - inner.loaded = false; - - Ok(()) - } - fn delete(&self) -> Result<()> { // delete underlying file fs::remove_file(self.path())?; @@ -227,10 +204,16 @@ impl Layer for ImageLayer { } let inner = self.load()?; + let file = inner.file.as_ref().unwrap(); + let tree_reader = + DiskBtreeReader::<_, KEY_SIZE>::new(inner.index_start_blk, inner.index_root_blk, file); - for (key, offset) in inner.index.iter() { - println!("key: {} offset {}", key, offset); - } + tree_reader.dump()?; + + tree_reader.visit(&[0u8; KEY_SIZE], VisitDirection::Forwards, |key, value| { + println!("key: {} offset {}", hex::encode(key), value); + true + })?; Ok(()) } @@ -300,6 +283,7 @@ impl ImageLayer { PathOrConf::Conf(_) => { let mut expected_summary = Summary::from(self); expected_summary.index_start_blk = actual_summary.index_start_blk; + expected_summary.index_root_blk = actual_summary.index_root_blk; if actual_summary != expected_summary { bail!("in-file summary does not match expected summary. actual = {:?} expected = {:?}", actual_summary, expected_summary); @@ -319,17 +303,8 @@ impl ImageLayer { } } - file.file.seek(SeekFrom::Start( - actual_summary.index_start_blk as u64 * PAGE_SZ as u64, - ))?; - let mut buf_reader = std::io::BufReader::new(&mut file.file); - let index = HashMap::des_from(&mut buf_reader)?; - inner.index_start_blk = actual_summary.index_start_blk; - - info!("loaded from {}", &path.display()); - - inner.index = index; + inner.index_root_blk = actual_summary.index_root_blk; inner.loaded = true; Ok(()) } @@ -348,10 +323,10 @@ impl ImageLayer { key_range: filename.key_range.clone(), lsn: filename.lsn, inner: RwLock::new(ImageLayerInner { - index: HashMap::new(), loaded: false, file: None, index_start_blk: 0, + index_root_blk: 0, }), } } @@ -376,9 +351,9 @@ impl ImageLayer { lsn: summary.lsn, inner: RwLock::new(ImageLayerInner { file: None, - index: HashMap::new(), loaded: false, index_start_blk: 0, + index_root_blk: 0, }), }) } @@ -420,9 +395,8 @@ pub struct ImageLayerWriter { key_range: Range, lsn: Lsn, - index: HashMap, - blob_writer: WriteBlobWriter, + tree: DiskBtreeBuilder, } impl ImageLayerWriter { @@ -447,9 +421,15 @@ impl ImageLayerWriter { }, ); info!("new image layer {}", path.display()); - let file = VirtualFile::create(&path)?; + let mut file = VirtualFile::create(&path)?; + // make room for the header block + file.seek(SeekFrom::Start(PAGE_SZ as u64))?; let blob_writer = WriteBlobWriter::new(file, PAGE_SZ as u64); + // Initialize the b-tree index builder + let block_buf = BlockBuf::new(); + let tree_builder = DiskBtreeBuilder::new(block_buf); + let writer = ImageLayerWriter { conf, _path: path, @@ -457,7 +437,7 @@ impl ImageLayerWriter { tenantid, key_range: key_range.clone(), lsn, - index: HashMap::new(), + tree: tree_builder, blob_writer, }; @@ -473,8 +453,9 @@ impl ImageLayerWriter { ensure!(self.key_range.contains(&key)); let off = self.blob_writer.write_blob(img)?; - let old = self.index.insert(key, off); - assert!(old.is_none()); + let mut keybuf: [u8; KEY_SIZE] = [0u8; KEY_SIZE]; + key.write_to_byte_slice(&mut keybuf); + self.tree.append(&keybuf, off)?; Ok(()) } @@ -486,9 +467,11 @@ impl ImageLayerWriter { let mut file = self.blob_writer.into_inner(); // Write out the index - let buf = HashMap::ser(&self.index)?; file.seek(SeekFrom::Start(index_start_blk as u64 * PAGE_SZ as u64))?; - file.write_all(&buf)?; + let (index_root_blk, block_buf) = self.tree.finish()?; + for buf in block_buf.blocks { + file.write_all(buf.as_ref())?; + } // Fill in the summary on blk 0 let summary = Summary { @@ -499,6 +482,7 @@ impl ImageLayerWriter { key_range: self.key_range.clone(), lsn: self.lsn, index_start_blk, + index_root_blk, }; file.seek(SeekFrom::Start(0))?; Summary::ser_into(&summary, &mut file)?; @@ -514,9 +498,9 @@ impl ImageLayerWriter { lsn: self.lsn, inner: RwLock::new(ImageLayerInner { loaded: false, - index: HashMap::new(), file: None, index_start_blk, + index_root_blk, }), }; trace!("created image layer {}", layer.path().display()); diff --git a/pageserver/src/layered_repository/inmemory_layer.rs b/pageserver/src/layered_repository/inmemory_layer.rs index 8a24528732..a45af51487 100644 --- a/pageserver/src/layered_repository/inmemory_layer.rs +++ b/pageserver/src/layered_repository/inmemory_layer.rs @@ -166,13 +166,6 @@ impl Layer for InMemoryLayer { todo!(); } - /// Cannot unload anything in an in-memory layer, since there's no backing - /// store. To release memory used by an in-memory layer, use 'freeze' to turn - /// it into an on-disk layer. - fn unload(&self) -> Result<()> { - Ok(()) - } - /// Nothing to do here. When you drop the last reference to the layer, it will /// be deallocated. fn delete(&self) -> Result<()> { diff --git a/pageserver/src/layered_repository/storage_layer.rs b/pageserver/src/layered_repository/storage_layer.rs index 5ad43182f6..e413f311c3 100644 --- a/pageserver/src/layered_repository/storage_layer.rs +++ b/pageserver/src/layered_repository/storage_layer.rs @@ -134,10 +134,6 @@ pub trait Layer: Send + Sync { /// Iterate through all keys and values stored in the layer fn iter(&self) -> Box> + '_>; - /// Release memory used by this layer. There is no corresponding 'load' - /// function, that's done implicitly when you call one of the get-functions. - fn unload(&self) -> Result<()>; - /// Permanently remove this layer from disk. fn delete(&self) -> Result<()>; diff --git a/pageserver/src/lib.rs b/pageserver/src/lib.rs index 6d2631b2b1..6dddef5f27 100644 --- a/pageserver/src/lib.rs +++ b/pageserver/src/lib.rs @@ -38,7 +38,7 @@ use pgdatadir_mapping::DatadirTimeline; /// This is embedded in the metadata file, and also in the header of all the /// layer files. If you make any backwards-incompatible changes to the storage /// format, bump this! -pub const STORAGE_FORMAT_VERSION: u16 = 2; +pub const STORAGE_FORMAT_VERSION: u16 = 3; // Magic constants used to identify different kinds of files pub const IMAGE_FILE_MAGIC: u16 = 0x5A60; diff --git a/pageserver/src/repository.rs b/pageserver/src/repository.rs index 7e998b0ebe..02334d3229 100644 --- a/pageserver/src/repository.rs +++ b/pageserver/src/repository.rs @@ -3,6 +3,7 @@ use crate::remote_storage::RemoteIndex; use crate::walrecord::ZenithWalRecord; use crate::CheckpointConfig; use anyhow::{bail, Result}; +use byteorder::{ByteOrder, BE}; use bytes::Bytes; use serde::{Deserialize, Serialize}; use std::fmt; @@ -27,6 +28,8 @@ pub struct Key { pub field6: u32, } +pub const KEY_SIZE: usize = 18; + impl Key { pub fn next(&self) -> Key { self.add(1) @@ -61,7 +64,7 @@ impl Key { key } - pub fn from_array(b: [u8; 18]) -> Self { + pub fn from_slice(b: &[u8]) -> Self { Key { field1: b[0], field2: u32::from_be_bytes(b[1..5].try_into().unwrap()), @@ -71,6 +74,15 @@ impl Key { field6: u32::from_be_bytes(b[14..18].try_into().unwrap()), } } + + pub fn write_to_byte_slice(&self, buf: &mut [u8]) { + buf[0] = self.field1; + BE::write_u32(&mut buf[1..5], self.field2); + BE::write_u32(&mut buf[5..9], self.field3); + BE::write_u32(&mut buf[9..13], self.field4); + buf[13] = self.field5; + BE::write_u32(&mut buf[14..18], self.field6); + } } pub fn key_range_size(key_range: &Range) -> u32 { @@ -569,7 +581,7 @@ mod tests { use lazy_static::lazy_static; lazy_static! { - static ref TEST_KEY: Key = Key::from_array(hex!("112222222233333333444444445500000001")); + static ref TEST_KEY: Key = Key::from_slice(&hex!("112222222233333333444444445500000001")); } #[test]