initial net code (#2)

* initial commit
not ready

* add new levin lib

* fix decoding multiple messages

* make the levin lib async

* saving progress

* saving progress

* init documention of levin and reorganise net

* update monero-wire

* remove p2p folder - this PR is just for net

* update Cargo.toml

* net: update links in cargo.toml
This commit is contained in:
Boog900 2023-03-07 22:37:55 +00:00 committed by GitHub
parent 649aac618c
commit 265fb3e895
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
17 changed files with 3563 additions and 3 deletions

3
.gitignore vendored
View file

@ -1,2 +1,3 @@
target/
Cargo.lock
Cargo.lock
.vscode

View file

@ -11,13 +11,16 @@ repository = "https://github.com/SyntheticBird45/cuprate"
# All Contributors on github
authors=[
"SyntheticBird45 <@someoneelse495495:matrix.org>"
"SyntheticBird45 <@someoneelse495495:matrix.org>",
"Boog900"
]
[workspace]
members = [
"blockchain_db"
"blockchain_db",
"net/levin",
"net/monero-wire"
]
[workspace.dependencies]

15
net/levin/Cargo.toml Normal file
View file

@ -0,0 +1,15 @@
[package]
name = "levin"
version = "0.1.0"
edition = "2021"
description = "A crate for working with the Levin protocol in Rust."
license = "MIT"
authors = ["Boog900"]
repository = "https://github.com/SyntheticBird45/cuprate/tree/main/net/levin"
[dependencies]
thiserror = "1.0.24"
byteorder = "1.4.3"
futures = "0.3"
bytes = "1"
pin-project = "1"

View file

@ -0,0 +1,82 @@
//! This module provides a `BucketSink` struct, which writes buckets to the
//! provided `AsyncWrite`. If you are a user of this library you should
//! probably use `MessageSink` instead.
use std::collections::VecDeque;
use std::pin::Pin;
use std::task::Poll;
use bytes::{Buf, BytesMut};
use futures::ready;
use futures::sink::Sink;
use futures::AsyncWrite;
use pin_project::pin_project;
use crate::{Bucket, BucketError};
/// A BucketSink writes Bucket instances to the provided AsyncWrite target.
#[pin_project]
pub struct BucketSink<W: AsyncWrite + std::marker::Unpin> {
#[pin]
writer: W,
buffer: VecDeque<BytesMut>,
}
impl<W: AsyncWrite + std::marker::Unpin> Sink<Bucket> for BucketSink<W> {
type Error = BucketError;
fn poll_ready(
self: std::pin::Pin<&mut Self>,
_: &mut std::task::Context<'_>,
) -> std::task::Poll<Result<(), Self::Error>> {
Poll::Ready(Ok(()))
}
fn start_send(mut self: Pin<&mut Self>, item: Bucket) -> Result<(), Self::Error> {
let buf = item.to_bytes();
self.buffer.push_back(BytesMut::from(&buf[..]));
Ok(())
}
fn poll_flush(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Result<(), Self::Error>> {
let this = self.project();
let mut w = this.writer;
let buffer = this.buffer;
loop {
match ready!(w.as_mut().poll_flush(cx)) {
Err(err) => return Poll::Ready(Err(err.into())),
Ok(()) => {
if let Some(buf) = buffer.front() {
match ready!(w.as_mut().poll_write(cx, buf)) {
Err(e) => match e.kind() {
std::io::ErrorKind::WouldBlock => return std::task::Poll::Pending,
_ => return Poll::Ready(Err(e.into())),
},
Ok(len) => {
if len == buffer[0].len() {
buffer.pop_front();
} else {
buffer[0].advance(len);
}
}
}
} else {
return Poll::Ready(Ok(()));
}
}
}
}
}
fn poll_close(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Result<(), Self::Error>> {
ready!(self.project().writer.poll_close(cx))?;
Poll::Ready(Ok(()))
}
}

View file

@ -0,0 +1,137 @@
//! This module provides a `BucketStream` struct, which is a stream of `Bucket`s,
//! where only the header is decoded. If you are a user of this library you should
//! probably use `MessageStream` instead.
use std::task::Poll;
use bytes::{Buf, BytesMut};
use futures::stream::Stream;
use futures::{ready, AsyncRead};
use pin_project::pin_project;
use super::{Bucket, BucketError, BucketHead};
/// An enum representing the decoding state of a `BucketStream`.
#[derive(Debug, Clone)]
enum BucketDecoder {
/// Waiting for the header of a `Bucket`.
WaitingForHeader,
/// Waiting for the body of a `Bucket` with the given header.
WaitingForBody(BucketHead),
}
impl BucketDecoder {
/// Returns the number of bytes needed to complete the current decoding state.
pub fn bytes_needed(&self) -> usize {
match self {
Self::WaitingForHeader => BucketHead::SIZE,
Self::WaitingForBody(bucket_head) => bucket_head.size as usize,
}
}
/// Tries to decode a `Bucket` from the given buffer, returning the decoded `Bucket` and the
/// number of bytes consumed from the buffer.
pub fn try_decode_bucket(
&mut self,
mut buf: &[u8],
) -> Result<(Option<Bucket>, usize), BucketError> {
let mut len = 0;
// first we decode header
if let BucketDecoder::WaitingForHeader = self {
if buf.len() < BucketHead::SIZE {
return Ok((None, 0));
}
let header = BucketHead::from_bytes(&mut buf)?;
len += BucketHead::SIZE;
*self = BucketDecoder::WaitingForBody(header);
};
// next we check we have enough bytes to fill the body
if let &mut Self::WaitingForBody(head) = self {
if buf.len() < head.size as usize {
return Ok((None, len));
}
*self = BucketDecoder::WaitingForHeader;
Ok((
Some(Bucket {
header: head,
body: buf.copy_to_bytes(buf.len()),
}),
len + head.size as usize,
))
} else {
unreachable!()
}
}
}
/// A stream of `Bucket`s, with only the header decoded.
#[pin_project]
#[derive(Debug, Clone)]
pub struct BucketStream<S> {
#[pin]
stream: S,
decoder: BucketDecoder,
buffer: BytesMut,
}
impl<S: AsyncRead> BucketStream<S> {
/// Creates a new `BucketStream` from the given `AsyncRead` stream.
pub fn new(stream: S) -> Self {
BucketStream {
stream,
decoder: BucketDecoder::WaitingForHeader,
buffer: BytesMut::with_capacity(1024),
}
}
}
impl<S: AsyncRead + std::marker::Unpin> Stream for BucketStream<S> {
type Item = Result<Bucket, BucketError>;
/// Attempt to read from the underlying stream into the buffer until enough bytes are received to construct a `Bucket`.
///
/// If enough bytes are received, return the decoded `Bucket`, if not enough bytes are received to construct a `Bucket`,
/// return `Poll::Pending`. This will never return `Poll::Ready(None)`.
///
fn poll_next(
self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
let this = self.project();
let mut stream = this.stream;
let decoder = this.decoder;
let buffer = this.buffer;
loop {
// this is a bit ugly but all we are doing is calculating the amount of bytes we
// need to build the rest of a bucket if this is zero it means we need to start
// reading a new bucket
let mut bytes_needed = buffer.len().saturating_sub(decoder.bytes_needed());
if bytes_needed == 0 {
bytes_needed = 1024
}
let mut buf = vec![0; bytes_needed];
match ready!(stream.as_mut().poll_read(cx, &mut buf)) {
Err(e) => match e.kind() {
std::io::ErrorKind::WouldBlock => return std::task::Poll::Pending,
std::io::ErrorKind::Interrupted => continue,
_ => return Poll::Ready(Some(Err(BucketError::IO(e)))),
},
Ok(len) => {
buffer.extend(&buf[..len]);
let (bucket, len) = decoder.try_decode_bucket(buffer)?;
buffer.advance(len);
if let Some(bucket) = bucket {
return Poll::Ready(Some(Ok(bucket)));
} else {
continue;
}
}
}
}
}
}

212
net/levin/src/header.rs Normal file
View file

@ -0,0 +1,212 @@
//! This module provides a struct BucketHead for the header of a levin protocol
//! message.
use std::io::Read;
use super::{BucketError, LEVIN_SIGNATURE, PROTOCOL_VERSION};
use byteorder::{LittleEndian, ReadBytesExt};
/// The Flags for the levin header
#[derive(Debug, Default, PartialEq, Eq, Clone, Copy)]
pub struct Flags(u32);
pub(crate) const REQUEST: Flags = Flags(0b0000_0001);
pub(crate) const RESPONSE: Flags = Flags(0b0000_0010);
const START_FRAGMENT: Flags = Flags(0b0000_0100);
const END_FRAGMENT: Flags = Flags(0b0000_1000);
const DUMMY: Flags = Flags(0b0000_1100); // both start and end fragment set
impl Flags {
fn contains_flag(&self, rhs: Self) -> bool {
self & &rhs == rhs
}
/// Converts the inner flags to little endian bytes
pub fn to_le_bytes(&self) -> [u8; 4] {
self.0.to_le_bytes()
}
/// Checks if the flags have the `REQUEST` flag set and
/// does not have the `RESPONSE` flag set, this does
/// not check for other flags
pub fn is_request(&self) -> bool {
self.contains_flag(REQUEST) && !self.contains_flag(RESPONSE)
}
/// Checks if the flags have the `RESPONSE` flag set and
/// does not have the `REQUEST` flag set, this does
/// not check for other flags
pub fn is_response(&self) -> bool {
self.contains_flag(RESPONSE) && !self.contains_flag(REQUEST)
}
/// Checks if the flags have the `START_FRAGMENT`and the
/// `END_FRAGMENT` flags set, this does
/// not check for other flags
pub fn is_dummy(&self) -> bool {
self.contains_flag(DUMMY)
}
/// Checks if the flags have the `START_FRAGMENT` flag
/// set and does not have the `END_FRAGMENT` flag set, this
/// does not check for other flags
pub fn is_start_fragment(&self) -> bool {
self.contains_flag(START_FRAGMENT) && !self.is_dummy()
}
/// Checks if the flags have the `END_FRAGMENT` flag
/// set and does not have the `START_FRAGMENT` flag set, this
/// does not check for other flags
pub fn is_end_fragment(&self) -> bool {
self.contains_flag(END_FRAGMENT) && !self.is_dummy()
}
/// Sets the `REQUEST` flag
pub fn set_flag_request(&mut self) {
*self |= REQUEST
}
/// Sets the `RESPONSE` flag
pub fn set_flag_response(&mut self) {
*self |= RESPONSE
}
/// Sets the `START_FRAGMENT` flag
pub fn set_flag_start_fragment(&mut self) {
*self |= START_FRAGMENT
}
/// Sets the `END_FRAGMENT` flag
pub fn set_flag_end_fragment(&mut self) {
*self |= END_FRAGMENT
}
/// Sets the `START_FRAGMENT` and `END_FRAGMENT` flag
pub fn set_flag_dummy(&mut self) {
self.set_flag_start_fragment();
self.set_flag_end_fragment();
}
}
impl From<u32> for Flags {
fn from(value: u32) -> Self {
Flags(value)
}
}
impl core::ops::BitAnd for &Flags {
type Output = Flags;
fn bitand(self, rhs: Self) -> Self::Output {
Flags(self.0 & rhs.0)
}
}
impl core::ops::BitOr for &Flags {
type Output = Flags;
fn bitor(self, rhs: Self) -> Self::Output {
Flags(self.0 | rhs.0)
}
}
impl core::ops::BitOrAssign for Flags {
fn bitor_assign(&mut self, rhs: Self) {
self.0 |= rhs.0
}
}
/// The Header of a Bucket. This contains
#[derive(Debug, PartialEq, Eq, Clone, Copy)]
pub struct BucketHead {
/// The network signature, should be `LEVIN_SIGNATURE` for Monero
pub signature: u64,
/// The size of the body
pub size: u64,
/// If the peer has to send data in the order of requests - some
/// messages require responses but don't have this set (some notifications)
pub have_to_return_data: bool,
/// Command
pub command: u32,
/// Return Code - will be 0 for requests and >0 for ok responses otherwise will be
/// a negative number corresponding to the error
pub return_code: i32,
/// The Flags of this header
pub flags: Flags,
/// The protocol version, for Monero this is currently 1
pub protocol_version: u32,
}
impl BucketHead {
/// The size of the header (in bytes)
pub const SIZE: usize = 33;
/// Builds the header in a Monero specific way
pub fn build(
payload_size: u64,
have_to_return_data: bool,
command: u32,
flags: Flags,
return_code: i32,
) -> BucketHead {
BucketHead {
signature: LEVIN_SIGNATURE,
size: payload_size,
have_to_return_data,
command,
return_code,
flags,
protocol_version: PROTOCOL_VERSION,
}
}
/// Builds the header from bytes, this function does not check any fields should
/// match the expected ones (signature, protocol_version)
pub fn from_bytes<R: Read + ?Sized>(r: &mut R) -> Result<BucketHead, BucketError> {
let header = BucketHead {
signature: r.read_u64::<LittleEndian>()?,
size: r.read_u64::<LittleEndian>()?,
have_to_return_data: r.read_u8()? != 0,
command: r.read_u32::<LittleEndian>()?,
return_code: r.read_i32::<LittleEndian>()?,
// this is incorrect an will not work for fragmented messages
flags: Flags::from(r.read_u32::<LittleEndian>()?),
protocol_version: r.read_u32::<LittleEndian>()?,
};
Ok(header)
}
/// Serializes the header
pub fn to_bytes(&self) -> Vec<u8> {
let mut out = Vec::with_capacity(BucketHead::SIZE);
out.extend_from_slice(&self.signature.to_le_bytes());
out.extend_from_slice(&self.size.to_le_bytes());
out.push(if self.have_to_return_data { 1 } else { 0 });
out.extend_from_slice(&self.command.to_le_bytes());
out.extend_from_slice(&self.return_code.to_le_bytes());
out.extend_from_slice(&self.flags.to_le_bytes());
out.extend_from_slice(&self.protocol_version.to_le_bytes());
out
}
}
#[cfg(test)]
mod tests {
use super::Flags;
#[test]
fn set_flags() {
macro_rules! set_and_check {
($set:ident, $check:ident) => {
let mut flag = Flags::default();
flag.$set();
assert!(flag.$check());
};
}
set_and_check!(set_flag_request, is_request);
set_and_check!(set_flag_response, is_response);
set_and_check!(set_flag_start_fragment, is_start_fragment);
set_and_check!(set_flag_end_fragment, is_end_fragment);
set_and_check!(set_flag_dummy, is_dummy);
}
}

134
net/levin/src/lib.rs Normal file
View file

@ -0,0 +1,134 @@
//! # Rust Levin
//!
//! A crate for working with the Levin protocol in Rust.
//!
//! The Levin protocol is a network protocol used in the Monero cryptocurrency. It is used for
//! peer-to-peer communication between nodes. This crate provides a Rust implementation of the Levin
//! header serialization and allows developers to define their own bucket bodies so this is not a
//! complete Monero networking crate.
//!
//! ## License
//!
//! This project is licensed under the MIT License.
// Coding conventions
#![forbid(unsafe_code)]
#![deny(non_upper_case_globals)]
#![deny(non_camel_case_types)]
#![deny(unused_mut)]
#![deny(missing_docs)]
pub mod bucket_sink;
pub mod bucket_stream;
pub mod header;
pub mod message_sink;
pub mod message_stream;
pub use header::BucketHead;
use std::fmt::Debug;
use bytes::Bytes;
use thiserror::Error;
/// Possible Errors when working with levin buckets
#[derive(Error, Debug)]
pub enum BucketError {
/// Unsupported p2p command.
#[error("Unsupported p2p command: {0}")]
UnsupportedP2pCommand(u32),
/// Revived header with incorrect signature.
#[error("Revived header with incorrect signature: {0}")]
IncorrectSignature(u64),
/// Header contains unknown flags.
#[error("Header contains unknown flags")]
UnknownFlags,
/// Revived header with unknown protocol version.
#[error("Revived header with unknown protocol version: {0}")]
UnknownProtocolVersion(u32),
/// More bytes needed to parse data.
#[error("More bytes needed to parse data")]
NotEnoughBytes,
/// Failed to decode bucket body.
#[error("Failed to decode bucket body: {0}")]
FailedToDecodeBucketBody(String),
/// Failed to encode bucket body.
#[error("Failed to encode bucket body: {0}")]
FailedToEncodeBucketBody(String),
/// IO Error.
#[error("IO Error: {0}")]
IO(#[from] std::io::Error),
/// Peer sent an error response code.
#[error("Peer sent an error response code: {0}")]
Error(i32),
}
const PROTOCOL_VERSION: u32 = 1;
const LEVIN_SIGNATURE: u64 = 0x0101010101012101;
/// A levin Bucket
#[derive(Debug)]
pub struct Bucket {
header: BucketHead,
body: Bytes,
}
impl Bucket {
fn to_bytes(&self) -> Bytes {
let mut buf = self.header.to_bytes();
buf.extend(self.body.iter());
buf.into()
}
}
/// An enum representing if the message is a request or response
#[derive(Debug)]
pub enum MessageType {
/// Request
Request,
/// Response
Response,
}
impl From<MessageType> for header::Flags {
fn from(val: MessageType) -> Self {
match val {
MessageType::Request => header::REQUEST,
MessageType::Response => header::RESPONSE,
}
}
}
impl TryInto<MessageType> for header::Flags {
type Error = BucketError;
fn try_into(self) -> Result<MessageType, Self::Error> {
if self.is_request() {
Ok(MessageType::Request)
} else if self.is_response() {
Ok(MessageType::Response)
} else {
Err(BucketError::UnknownFlags)
}
}
}
/// A levin body
pub trait LevinBody: Sized {
/// Decodes the message from the data in the header
fn decode_message(
buf: &[u8],
typ: MessageType,
have_to_return: bool,
command: u32,
) -> Result<Self, BucketError>;
/// Encodes the message
///
/// returns:
/// return_code: i32,
/// command: u32,
/// have_to_return: bool,
/// message_type: MessageType
/// bytes: Bytes
fn encode(&self) -> Result<(i32, u32, bool, MessageType, Bytes), BucketError>;
}

View file

@ -0,0 +1,64 @@
//! This module provides a `MessageSink` struct, which is a stream of user defined
//! messages.
use std::marker::PhantomData;
use std::pin::Pin;
use std::task::Poll;
use futures::AsyncWrite;
use futures::Sink;
use pin_project::pin_project;
use crate::bucket_sink::BucketSink;
use crate::Bucket;
use crate::BucketError;
use crate::BucketHead;
use crate::LevinBody;
/// A Sink that converts levin messages to buckets and passes them onto the `BucketSink`
#[pin_project]
pub struct MessageSink<W: AsyncWrite + std::marker::Unpin, E: LevinBody> {
#[pin]
bucket_sink: BucketSink<W>,
phantom: PhantomData<E>,
}
impl<W: AsyncWrite + std::marker::Unpin, E: LevinBody> Sink<E> for MessageSink<W, E> {
type Error = BucketError;
fn poll_ready(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Result<(), Self::Error>> {
self.project().bucket_sink.poll_ready(cx)
}
fn start_send(self: Pin<&mut Self>, item: E) -> Result<(), Self::Error> {
let (return_code, command, have_to_return_data, flags, body) = item.encode()?;
let header = BucketHead::build(
body.len() as u64,
have_to_return_data,
command,
flags.into(),
return_code,
);
let bucket = Bucket { header, body };
self.project().bucket_sink.start_send(bucket)
}
fn poll_flush(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Result<(), Self::Error>> {
self.project().bucket_sink.poll_flush(cx)
}
fn poll_close(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Result<(), Self::Error>> {
self.project().bucket_sink.poll_close(cx)
}
}

View file

@ -0,0 +1,79 @@
//! This contains
//!
use std::marker::PhantomData;
use std::task::Poll;
use futures::ready;
use futures::AsyncRead;
use futures::Stream;
use pin_project::pin_project;
use crate::bucket_stream::BucketStream;
use crate::BucketError;
use crate::LevinBody;
use crate::LEVIN_SIGNATURE;
use crate::PROTOCOL_VERSION;
/// A stream that reads from the underlying `BucketStream` and uses the the
/// methods on the `LevinBody` trait to decode the inner messages(bodies)
#[pin_project]
pub struct MessageStream<D: LevinBody, S: AsyncRead + std::marker::Unpin> {
#[pin]
bucket_stream: BucketStream<S>,
phantom: PhantomData<D>,
}
impl<D: LevinBody, S: AsyncRead + std::marker::Unpin> MessageStream<D, S> {
/// Creates a new stream from the provided `AsyncRead`
pub fn new(stream: S) -> Self {
MessageStream {
bucket_stream: BucketStream::new(stream),
phantom: PhantomData,
}
}
}
impl<D: LevinBody, S: AsyncRead + std::marker::Unpin> Stream for MessageStream<D, S> {
type Item = Result<D, BucketError>;
fn poll_next(
self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Option<Self::Item>> {
let this = self.project();
match ready!(this.bucket_stream.poll_next(cx)).expect("BucketStream will never return None")
{
Err(e) => Poll::Ready(Some(Err(e))),
Ok(bucket) => {
if bucket.header.signature != LEVIN_SIGNATURE {
return Err(BucketError::IncorrectSignature(bucket.header.signature))?;
}
if bucket.header.protocol_version != PROTOCOL_VERSION {
return Err(BucketError::UnknownProtocolVersion(
bucket.header.protocol_version,
))?;
}
if bucket.header.return_code < 0
|| (bucket.header.return_code == 0 && bucket.header.flags.is_response())
{
return Err(BucketError::Error(bucket.header.return_code))?;
}
if bucket.header.flags.is_dummy() {
cx.waker().wake_by_ref();
return Poll::Pending;
}
Poll::Ready(Some(D::decode_message(
&bucket.body,
bucket.header.flags.try_into()?,
bucket.header.have_to_return_data,
bucket.header.command,
)))
}
}
}
}

View file

@ -0,0 +1,19 @@
[package]
name = "monero-wire"
version = "0.1.0"
edition = "2021"
license = "MIT"
authors = ["Boog900"]
repository = "https://github.com/SyntheticBird45/cuprate/tree/main/net/monero-wire"
[dependencies]
serde = {version = "1.0", features =["derive"]}
serde_with = "2.2.0"
epee-serde = {git="https://github.com/Boog900/epee_serde.git"}
monero = "0.18.2"
levin = {path="../levin"}
byteorder = "1.4.3"
bytes = "1"
thiserror = "1.0.24"

View file

@ -0,0 +1,31 @@
macro_rules! get_field_from_map {
($map:ident, $field_name:expr) => {
$map.get($field_name)
.ok_or_else(|| serde::de::Error::missing_field($field_name))?
};
}
macro_rules! get_val_from_map {
($map:ident, $field_name:expr, $get_fn:ident, $expected_ty:expr) => {
$map.get($field_name)
.ok_or_else(|| serde::de::Error::missing_field($field_name))?
.$get_fn()
.ok_or_else(|| {
serde::de::Error::invalid_type($map.get_value_type_as_unexpected(), &$expected_ty)
})?
};
}
macro_rules! get_internal_val {
($value:ident, $get_fn:ident, $expected_ty:expr) => {
$value.$get_fn().ok_or_else(|| {
serde::de::Error::invalid_type($value.get_value_type_as_unexpected(), &$expected_ty)
})?
};
}
macro_rules! monero_decode_into_serde_err {
($ty:ty, $buf:ident) => {
monero::consensus::deserialize::<$ty>($buf).map_err(serde::de::Error::custom)?
};
}

109
net/monero-wire/src/lib.rs Normal file
View file

@ -0,0 +1,109 @@
//! # Monero Wire
//!
//! A crate defining Monero network messages and network addresses,
//! built on top of the levin crate.
//!
//! ## License
//!
//! This project is licensed under the MIT License.
// Coding conventions
#![forbid(unsafe_code)]
#![deny(non_upper_case_globals)]
#![deny(non_camel_case_types)]
#![deny(unused_mut)]
#![deny(missing_docs)]
#[macro_use]
mod internal_macros;
pub mod messages;
pub mod network_address;
pub use network_address::NetworkAddress;
// re-exports
pub use levin;
pub use levin::message_sink::MessageSink;
pub use levin::message_stream::MessageStream;
use levin::BucketError;
/// The possible commands that can be in a levin header
#[derive(Debug, PartialEq, Eq, Clone, Copy)]
pub enum P2pCommand {
// 100* admin commands
/// Handshake
Handshake,
/// TimedSync
TimedSync,
/// Ping
Ping,
/// SupportFlags
SupportFlags,
// 200* protocol commands
/// NewBlock
NewBlock,
/// NewTransactions
NewTransactions,
/// RequestGetObject
RequestGetObject,
/// ResponseGetObject
ResponseGetObject,
/// RequestChain
RequestChain,
/// ResponseChainEntry
ResponseChainEntry,
/// NewFluffyBlock
NewFluffyBlock,
/// RequestFluffyMissingTx
RequestFluffyMissingTx,
/// GetTxPoolComplement
GetTxPoolComplement,
}
impl TryFrom<u32> for P2pCommand {
type Error = BucketError;
fn try_from(value: u32) -> Result<Self, Self::Error> {
match value {
1001 => Ok(P2pCommand::Handshake),
1002 => Ok(P2pCommand::TimedSync),
1003 => Ok(P2pCommand::Ping),
1007 => Ok(P2pCommand::SupportFlags),
2001 => Ok(P2pCommand::NewBlock),
2002 => Ok(P2pCommand::NewTransactions),
2003 => Ok(P2pCommand::RequestGetObject),
2004 => Ok(P2pCommand::ResponseGetObject),
2006 => Ok(P2pCommand::RequestChain),
2007 => Ok(P2pCommand::ResponseChainEntry),
2008 => Ok(P2pCommand::NewFluffyBlock),
2009 => Ok(P2pCommand::RequestFluffyMissingTx),
2010 => Ok(P2pCommand::GetTxPoolComplement),
_ => Err(BucketError::UnsupportedP2pCommand(value)),
}
}
}
impl From<P2pCommand> for u32 {
fn from(val: P2pCommand) -> Self {
match val {
P2pCommand::Handshake => 1001,
P2pCommand::TimedSync => 1002,
P2pCommand::Ping => 1003,
P2pCommand::SupportFlags => 1007,
P2pCommand::NewBlock => 2001,
P2pCommand::NewTransactions => 2002,
P2pCommand::RequestGetObject => 2003,
P2pCommand::ResponseGetObject => 2004,
P2pCommand::RequestChain => 2006,
P2pCommand::ResponseChainEntry => 2007,
P2pCommand::NewFluffyBlock => 2008,
P2pCommand::RequestFluffyMissingTx => 2009,
P2pCommand::GetTxPoolComplement => 2010,
}
}
}

View file

@ -0,0 +1,323 @@
//! This module defines a Monero `Message` enum which contains
//! every possible Monero network message (levin body)
pub mod admin;
pub mod common;
pub mod protocol;
pub use common::{BasicNodeData, CoreSyncData, PeerID, PeerListEntryBase};
use bytes::Bytes;
use levin::BucketError;
use levin::MessageType;
use crate::P2pCommand;
fn zero_val<T: From<u8>>() -> T {
T::from(0_u8)
}
fn default_true() -> bool {
true
}
fn default_false() -> bool {
false
}
/// A message request
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum MessageRequest {
/// Handshake
Handshake(admin::HandshakeRequest),
/// TimedSync
TimedSync(admin::TimedSyncRequest),
/// Ping
Ping(admin::PingRequest),
/// SupportFlags
SupportFlags(admin::SupportFlagsRequest),
}
/// A message response
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum MessageResponse {
/// Handshake
Handshake(admin::HandshakeResponse),
/// TimedSync
TimedSync(admin::TimedSyncResponse),
/// Ping
Ping(admin::PingResponse),
/// SupportFlags
SupportFlags(admin::SupportFlagsResponse),
}
/// A messages notification
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum MessageNotification {
/// NewBlock
NewBlock(protocol::NewBlock),
/// NewTransactions
NewTransactions(protocol::NewTransactions),
/// RequestGetObject
RequestGetObject(protocol::GetObjectsRequest),
/// ResponseGetObject
ResponseGetObject(protocol::GetObjectsResponse),
/// RequestChain
RequestChain(protocol::ChainRequest),
/// ResponseChainEntry
ResponseChainEntry(protocol::ChainResponse),
/// NewFluffyBlock
NewFluffyBlock(protocol::NewFluffyBlock),
/// RequestFluffyMissingTx
RequestFluffyMissingTx(protocol::FluffyMissingTransactionsRequest),
/// GetTxPoolComplement
GetTxPoolComplement(protocol::TxPoolCompliment),
}
/// A Monero Message (levin body)
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Message {
/// Request
Request(MessageRequest),
/// Response
Response(MessageResponse),
/// Notification
Notification(Box<MessageNotification>), // check benefits/ drawbacks of doing this, im just boxing it for now to satisfy clippy
}
fn epee_encode_error_to_levin(err: epee_serde::Error) -> BucketError {
BucketError::FailedToEncodeBucketBody(err.to_string())
}
fn encode_message<T: serde::ser::Serialize>(message: &T) -> Result<Vec<u8>, BucketError> {
epee_serde::to_bytes(message).map_err(epee_encode_error_to_levin)
}
fn epee_decode_error_to_levin(err: epee_serde::Error) -> BucketError {
BucketError::FailedToDecodeBucketBody(err.to_string())
}
fn decode_message<T: serde::de::DeserializeOwned>(buf: &[u8]) -> Result<T, BucketError> {
epee_serde::from_bytes(buf).map_err(epee_decode_error_to_levin)
}
impl levin::LevinBody for Message {
fn decode_message(
buf: &[u8],
typ: MessageType,
have_to_return: bool,
command: u32,
) -> Result<Self, BucketError> {
let command = P2pCommand::try_from(command)?;
Ok(match typ {
MessageType::Response => Message::Response(match command {
P2pCommand::Handshake => MessageResponse::Handshake(decode_message(buf)?),
P2pCommand::TimedSync => MessageResponse::TimedSync(decode_message(buf)?),
P2pCommand::Ping => MessageResponse::Ping(decode_message(buf)?),
P2pCommand::SupportFlags => MessageResponse::SupportFlags(decode_message(buf)?),
_ => {
return Err(levin::BucketError::FailedToDecodeBucketBody(
"Invalid header flag/command/have_to_return combination".to_string(),
))
}
}),
MessageType::Request if have_to_return => Message::Request(match command {
P2pCommand::Handshake => MessageRequest::Handshake(decode_message(buf)?),
P2pCommand::TimedSync => MessageRequest::TimedSync(decode_message(buf)?),
P2pCommand::Ping => MessageRequest::Ping(admin::PingRequest),
P2pCommand::SupportFlags => {
MessageRequest::SupportFlags(admin::SupportFlagsRequest)
}
_ => {
return Err(levin::BucketError::FailedToDecodeBucketBody(
"Invalid header flag/command/have_to_return combination".to_string(),
))
}
}),
MessageType::Request if !have_to_return => {
Message::Notification(Box::new(match command {
P2pCommand::NewBlock => MessageNotification::NewBlock(decode_message(buf)?),
P2pCommand::NewTransactions => {
MessageNotification::NewTransactions(decode_message(buf)?)
}
P2pCommand::RequestGetObject => {
MessageNotification::RequestGetObject(decode_message(buf)?)
}
P2pCommand::ResponseGetObject => {
MessageNotification::ResponseGetObject(decode_message(buf)?)
}
P2pCommand::RequestChain => {
MessageNotification::RequestChain(decode_message(buf)?)
}
P2pCommand::ResponseChainEntry => {
MessageNotification::ResponseChainEntry(decode_message(buf)?)
}
P2pCommand::NewFluffyBlock => {
MessageNotification::NewFluffyBlock(decode_message(buf)?)
}
P2pCommand::RequestFluffyMissingTx => {
MessageNotification::RequestFluffyMissingTx(decode_message(buf)?)
}
P2pCommand::GetTxPoolComplement => {
MessageNotification::GetTxPoolComplement(decode_message(buf)?)
}
_ => {
return Err(levin::BucketError::FailedToDecodeBucketBody(
"Invalid header flag/command/have_to_return combination".to_string(),
))
}
}))
}
_ => unreachable!("All typs are handleded"),
})
}
fn encode(&self) -> Result<(i32, u32, bool, MessageType, Bytes), BucketError> {
let return_code;
let command;
let have_to_return_data;
let flag;
let bytes;
match self {
Message::Request(req) => {
return_code = 0;
have_to_return_data = true;
flag = MessageType::Request;
match req {
MessageRequest::Handshake(handshake) => {
command = P2pCommand::Handshake;
bytes = encode_message(handshake)?;
}
MessageRequest::TimedSync(timedsync) => {
command = P2pCommand::TimedSync;
bytes = encode_message(timedsync)?;
}
MessageRequest::Ping(_) => {
command = P2pCommand::Ping;
bytes = Vec::new();
}
MessageRequest::SupportFlags(_) => {
command = P2pCommand::SupportFlags;
bytes = Vec::new();
}
}
}
Message::Response(res) => {
return_code = 1;
have_to_return_data = false;
flag = MessageType::Response;
match res {
MessageResponse::Handshake(handshake) => {
command = P2pCommand::Handshake;
bytes = encode_message(handshake)?;
}
MessageResponse::TimedSync(timed_sync) => {
command = P2pCommand::TimedSync;
bytes = encode_message(timed_sync)?;
}
MessageResponse::Ping(ping) => {
command = P2pCommand::Ping;
bytes = encode_message(ping)?;
}
MessageResponse::SupportFlags(support_flags) => {
command = P2pCommand::SupportFlags;
bytes = encode_message(support_flags)?;
}
}
}
Message::Notification(noti) => {
return_code = 0;
have_to_return_data = false;
flag = MessageType::Response;
match noti.as_ref() {
MessageNotification::NewBlock(new_block) => {
command = P2pCommand::NewBlock;
bytes = encode_message(new_block)?;
}
MessageNotification::NewTransactions(new_txs) => {
command = P2pCommand::NewTransactions;
bytes = encode_message(new_txs)?;
}
MessageNotification::RequestGetObject(obj) => {
command = P2pCommand::RequestGetObject;
bytes = encode_message(obj)?;
}
MessageNotification::ResponseGetObject(obj) => {
command = P2pCommand::ResponseGetObject;
bytes = encode_message(obj)?;
}
MessageNotification::RequestChain(chain) => {
command = P2pCommand::RequestChain;
bytes = encode_message(chain)?;
}
MessageNotification::ResponseChainEntry(chain_entry) => {
command = P2pCommand::ResponseChainEntry;
bytes = encode_message(chain_entry)?;
}
MessageNotification::NewFluffyBlock(fluffy_block) => {
command = P2pCommand::NewFluffyBlock;
bytes = encode_message(fluffy_block)?;
}
MessageNotification::RequestFluffyMissingTx(txs) => {
command = P2pCommand::RequestFluffyMissingTx;
bytes = encode_message(txs)?;
}
MessageNotification::GetTxPoolComplement(txpool) => {
command = P2pCommand::GetTxPoolComplement;
bytes = encode_message(txpool)?;
}
}
}
}
return Ok((
return_code,
command.into(),
have_to_return_data,
flag,
bytes.into(),
));
}
}
#[cfg(test)]
mod tests {
use super::Message;
use levin::{LevinBody, MessageType};
#[test]
fn decode_handshake_request() {
let buf = [
1, 17, 1, 1, 1, 1, 2, 1, 1, 12, 9, 110, 111, 100, 101, 95, 100, 97, 116, 97, 12, 24, 7,
109, 121, 95, 112, 111, 114, 116, 6, 168, 70, 0, 0, 10, 110, 101, 116, 119, 111, 114,
107, 95, 105, 100, 10, 64, 18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161,
161, 16, 7, 112, 101, 101, 114, 95, 105, 100, 5, 153, 5, 227, 61, 188, 214, 159, 10,
13, 115, 117, 112, 112, 111, 114, 116, 95, 102, 108, 97, 103, 115, 6, 1, 0, 0, 0, 8,
114, 112, 99, 95, 112, 111, 114, 116, 7, 0, 0, 20, 114, 112, 99, 95, 99, 114, 101, 100,
105, 116, 115, 95, 112, 101, 114, 95, 104, 97, 115, 104, 6, 0, 0, 0, 0, 12, 112, 97,
121, 108, 111, 97, 100, 95, 100, 97, 116, 97, 12, 24, 21, 99, 117, 109, 117, 108, 97,
116, 105, 118, 101, 95, 100, 105, 102, 102, 105, 99, 117, 108, 116, 121, 5, 59, 90,
163, 153, 0, 0, 0, 0, 27, 99, 117, 109, 117, 108, 97, 116, 105, 118, 101, 95, 100, 105,
102, 102, 105, 99, 117, 108, 116, 121, 95, 116, 111, 112, 54, 52, 5, 0, 0, 0, 0, 0, 0,
0, 0, 14, 99, 117, 114, 114, 101, 110, 116, 95, 104, 101, 105, 103, 104, 116, 5, 190,
50, 0, 0, 0, 0, 0, 0, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6,
0, 0, 0, 0, 6, 116, 111, 112, 95, 105, 100, 10, 128, 230, 40, 186, 45, 79, 79, 224,
164, 117, 133, 84, 130, 185, 94, 4, 1, 57, 126, 74, 145, 238, 238, 122, 44, 214, 85,
129, 237, 230, 14, 67, 218, 11, 116, 111, 112, 95, 118, 101, 114, 115, 105, 111, 110,
8, 1, 18, 108, 111, 99, 97, 108, 95, 112, 101, 101, 114, 108, 105, 115, 116, 95, 110,
101, 119, 140, 4, 24, 3, 97, 100, 114, 12, 8, 4, 116, 121, 112, 101, 8, 1, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 225, 219, 21, 0, 6, 109, 95, 112, 111, 114,
116, 7, 0, 0, 2, 105, 100, 5, 0, 0, 0, 0, 0, 0, 0, 0, 9, 108, 97, 115, 116, 95, 115,
101, 101, 110, 1, 0, 0, 0, 0, 0, 0, 0, 0, 12, 112, 114, 117, 110, 105, 110, 103, 95,
115, 101, 101, 100, 6, 0, 0, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 0, 0,
20, 114, 112, 99, 95, 99, 114, 101, 100, 105, 116, 115, 95, 112, 101, 114, 95, 104, 97,
115, 104, 6, 0, 0, 0, 0,
];
let message = Message::decode_message(&buf, MessageType::Request, true, 1001);
println!("{:?}", message);
}
}

View file

@ -0,0 +1,939 @@
//! This module defines Monero admin messages
//!
//! Admin message requests must be responded to in order unlike
//! protocol messages.
use serde::{Deserialize, Serialize};
use super::{
common::{BasicNodeData, CoreSyncData, PeerListEntryBase},
PeerID,
};
/// A Handshake Request
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct HandshakeRequest {
/// Basic Node Data
pub node_data: BasicNodeData,
/// Core Sync Data
pub payload_data: CoreSyncData,
}
/// A Handshake Response
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct HandshakeResponse {
/// Basic Node Data
pub node_data: BasicNodeData,
/// Core Sync Data
pub payload_data: CoreSyncData,
/// PeerList
pub local_peerlist_new: Vec<PeerListEntryBase>,
}
/// A TimedSync Request
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct TimedSyncRequest {
/// Core Sync Data
pub payload_data: CoreSyncData,
}
/// A TimedSync Response
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct TimedSyncResponse {
/// Core Sync Data
pub payload_data: CoreSyncData,
/// PeerList
pub local_peerlist_new: Vec<PeerListEntryBase>,
}
/// The status field of an okay ping response
pub const PING_OK_RESPONSE_STATUS_TEXT: &str = "OK";
/// A Ping Request
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct PingRequest;
/// A Ping Response
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct PingResponse {
/// Status: should be `PING_OK_RESPONSE_STATUS_TEXT`
pub status: String,
/// Peer ID
pub peer_id: PeerID,
}
/// A Support Flags Request
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct SupportFlagsRequest;
/// A Support Flags Response
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct SupportFlagsResponse {
/// Support Flags
pub support_flags: u32,
}
#[cfg(test)]
mod tests {
use std::str::FromStr;
use monero::Hash;
use super::{BasicNodeData, CoreSyncData, HandshakeRequest, HandshakeResponse};
use crate::messages::common::PeerID;
#[test]
fn serde_handshake_req() {
let bytes = [
1, 17, 1, 1, 1, 1, 2, 1, 1, 8, 9, 110, 111, 100, 101, 95, 100, 97, 116, 97, 12, 16, 7,
109, 121, 95, 112, 111, 114, 116, 6, 0, 0, 0, 0, 10, 110, 101, 116, 119, 111, 114, 107,
95, 105, 100, 10, 64, 18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161, 161,
16, 7, 112, 101, 101, 114, 95, 105, 100, 5, 95, 135, 39, 132, 254, 187, 55, 134, 13,
115, 117, 112, 112, 111, 114, 116, 95, 102, 108, 97, 103, 115, 6, 1, 0, 0, 0, 12, 112,
97, 121, 108, 111, 97, 100, 95, 100, 97, 116, 97, 12, 16, 21, 99, 117, 109, 117, 108,
97, 116, 105, 118, 101, 95, 100, 105, 102, 102, 105, 99, 117, 108, 116, 121, 5, 1, 0,
0, 0, 0, 0, 0, 0, 14, 99, 117, 114, 114, 101, 110, 116, 95, 104, 101, 105, 103, 104,
116, 5, 0, 0, 0, 0, 0, 0, 0, 0, 6, 116, 111, 112, 95, 105, 100, 10, 128, 65, 128, 21,
187, 154, 233, 130, 161, 151, 93, 167, 215, 146, 119, 194, 112, 87, 39, 165, 104, 148,
186, 15, 178, 70, 173, 170, 187, 31, 70, 50, 227, 11, 116, 111, 112, 95, 118, 101, 114,
115, 105, 111, 110, 8, 1,
];
let handshake: HandshakeRequest = epee_serde::from_bytes(bytes).unwrap();
let basic_node_data = BasicNodeData {
my_port: 0,
network_id: [
18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161, 161, 16,
],
peer_id: PeerID(9671405426614699871),
support_flags: 1,
rpc_port: 0,
rpc_credits_per_hash: 0,
};
let core_sync_data = CoreSyncData {
cumulative_difficulty: 1,
cumulative_difficulty_top64: 0,
current_height: 0,
pruning_seed: 0,
top_id: Hash::from_str(
"0x418015bb9ae982a1975da7d79277c2705727a56894ba0fb246adaabb1f4632e3",
)
.unwrap(),
top_version: 1,
};
assert_eq!(basic_node_data, handshake.node_data);
assert_eq!(core_sync_data, handshake.payload_data);
let encoded_bytes = epee_serde::to_bytes(&handshake).unwrap();
let handshake_2: HandshakeRequest = epee_serde::from_bytes(encoded_bytes).unwrap();
assert_eq!(handshake, handshake_2);
}
#[test]
fn serde_handshake_res() {
let bytes = [
1, 17, 1, 1, 1, 1, 2, 1, 1, 12, 18, 108, 111, 99, 97, 108, 95, 112, 101, 101, 114, 108,
105, 115, 116, 95, 110, 101, 119, 140, 233, 3, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 107, 200, 125, 246, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 136, 104, 95, 61,
247, 215, 186, 88, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 184, 166, 147, 112, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 104, 35, 236, 206, 255, 98, 40, 247, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 94, 23, 147,
238, 6, 109, 95, 112, 111, 114, 116, 7, 64, 9, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 204, 213, 226, 217, 141, 205, 23, 171, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255,
65, 144, 135, 125, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 2, 2, 105, 100, 5, 142, 211, 149, 235, 68, 76, 58, 116, 8, 114, 112, 99, 95, 112,
111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 85, 25, 198, 233, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 239, 118, 5, 143, 245, 175, 79, 79, 12, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 128, 199,
45, 242, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 53, 123, 81, 158, 161, 118, 88, 49, 8, 114, 112, 99, 95, 112, 111, 114,
116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 49, 12, 239, 176, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 231, 115, 240, 106, 33, 156, 129, 168, 12, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 72, 83, 1, 29, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
163, 230, 52, 192, 29, 175, 184, 138, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115,
101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 24, 46, 131, 31,
6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100,
5, 254, 139, 52, 145, 46, 192, 114, 218, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 144,
76, 58, 247, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2,
2, 105, 100, 5, 22, 231, 179, 203, 218, 156, 64, 157, 12, 112, 114, 117, 110, 105, 110,
103, 95, 115, 101, 101, 100, 6, 128, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255,
46, 4, 27, 39, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2,
2, 105, 100, 5, 190, 246, 23, 85, 132, 101, 46, 120, 12, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 93, 95, 228, 51, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 76, 184, 195, 205,
119, 55, 205, 223, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 89, 36, 78, 249, 6,
109, 95, 112, 111, 114, 116, 7, 40, 230, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
146, 84, 128, 27, 45, 173, 110, 233, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 159, 203, 65, 163, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 36, 140, 127, 49, 80, 143, 75,
170, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 12, 3, 97, 100, 114, 12, 8,
4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 89, 163, 225, 22, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 202, 167,
239, 104, 191, 245, 206, 114, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101,
100, 6, 128, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 51, 38, 53, 106, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 26, 47, 180, 164, 45, 11, 4, 137, 12, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 98, 28, 252, 84, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
109, 204, 107, 31, 61, 229, 138, 61, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115,
101, 101, 100, 6, 130, 1, 0, 0, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 35, 142, 222, 141, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 74, 23, 146, 223, 157, 219, 198, 96,
12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 135, 1, 0, 0, 8, 114,
112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 168, 119, 134, 54, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 210, 66, 174, 96, 27,
169, 14, 141, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 24, 246, 38, 201, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 29, 41, 255, 104, 159, 199, 248, 152, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 255, 255, 5, 161, 65, 201, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 2, 2, 105, 100, 5, 97, 17, 31, 195, 85, 103, 254, 119, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 76, 89, 170, 52, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
191, 241, 183, 135, 6, 109, 186, 8, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 184, 153,
84, 228, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2,
105, 100, 5, 211, 11, 202, 189, 127, 164, 118, 13, 16, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 66, 42, 82, 58, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 209, 217, 109, 131,
173, 212, 169, 138, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6,
135, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 70, 93, 166, 98, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 35,
43, 193, 141, 6, 220, 30, 115, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 99, 17, 41, 20, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 56, 45, 55, 79, 80, 200, 73, 54, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 130, 180, 116,
214, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 101, 87, 57, 29, 16, 210, 163, 114, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 148, 163, 81, 34, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 118, 209, 9, 62, 203, 181,
144, 55, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 67, 85, 160, 39, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 8, 54, 90, 44, 129, 203, 30, 229, 8, 114, 112, 99, 95, 112, 111,
114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 162, 218, 65, 223, 6, 109, 95, 112, 111, 114, 116, 7, 163, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 8, 95, 132, 221, 20, 172, 44, 131, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 83, 78, 142, 101, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
53, 254, 109, 4, 238, 54, 246, 88, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 18, 169, 212, 248, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 239, 181, 136, 45, 48, 0, 23,
111, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 138, 201, 50, 228, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 100, 143, 145,
170, 221, 44, 134, 40, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 96, 2, 101, 49, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 175, 162, 146, 137, 96, 124, 179, 123, 8, 114,
112, 99, 95, 112, 111, 114, 116, 7, 161, 70, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 194, 55, 15, 70, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 200, 230, 46, 22, 61,
113, 146, 207, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 131,
1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 255, 255, 24, 144, 51, 151, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 2, 2, 105, 100, 5, 92, 225, 176, 105, 146, 76, 126, 246, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 51, 89, 43, 165,
6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100,
5, 227, 169, 44, 172, 218, 27, 134, 20, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 138,
201, 50, 11, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2,
2, 105, 100, 5, 77, 159, 65, 201, 154, 203, 137, 189, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 37, 19, 221, 165, 6, 109, 95, 112, 111,
114, 116, 7, 101, 225, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 16, 236, 61, 198,
179, 107, 122, 230, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 211, 104, 25, 222, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 141, 18, 233, 140, 158, 178, 117, 2, 12, 112, 114,
117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 79, 225, 232, 232, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 25,
41, 192, 83, 178, 152, 49, 39, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 45, 44, 224, 220, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 43, 192, 235, 1, 175, 51, 244, 209, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 44, 198, 160,
153, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 255, 168, 200, 48, 53, 200, 228, 34, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 85, 214, 100, 187, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 237, 118, 70, 209, 142,
252, 124, 105, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 132,
1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 104, 52, 169, 164, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 7, 180, 60, 132, 208, 115, 38, 78, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 255, 255, 188, 214, 129, 182, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 2, 2, 105, 100, 5, 6, 110, 8, 120, 108, 89, 215, 109, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 81, 79, 96, 58, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
29, 242, 12, 182, 32, 31, 164, 147, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 50, 246, 254, 202, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 200, 152, 38, 71, 152, 168, 148,
130, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
75, 168, 213, 32, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 89, 66, 99, 25, 23, 47, 236, 47, 12, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 80, 217, 6, 170, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 25, 65, 171, 45, 39,
65, 82, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 16, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 24, 27, 96, 242, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 11, 5,
248, 49, 46, 9, 138, 92, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100,
6, 133, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 72, 239, 241, 242, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
77, 192, 17, 161, 34, 186, 222, 123, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 144, 217, 70, 139, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 144, 82, 41, 18, 67, 173, 224,
251, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 129, 1, 0, 0,
12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 24,
216, 239, 98, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 135, 151, 188, 127, 59, 20, 95, 161, 12, 112, 114, 117, 110, 105, 110,
103, 95, 115, 101, 101, 100, 6, 134, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 91, 65, 190, 110, 6, 109, 95, 112, 111, 114,
116, 7, 169, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 96, 33, 127, 102, 27,
224, 111, 185, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100,
100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 134, 122, 61, 72, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 29, 223,
119, 202, 113, 165, 81, 7, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 37, 15, 95,
252, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 219, 64, 229, 115, 235, 126, 213, 173, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 178, 62, 9, 149, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 151, 68, 31, 187, 63,
163, 185, 18, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 23, 115, 236, 180, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 205,
12, 26, 50, 3, 165, 53, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 71, 78, 74, 138, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 206, 121, 181, 161, 29, 110, 53, 18, 8, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 73, 216,
104, 205, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 243, 216, 53, 122, 144, 182, 239, 255, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 85, 241, 6, 106, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 249, 166, 55, 155,
51, 129, 124, 12, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 79, 199, 36, 70, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 43, 26, 130, 68, 205, 92, 233, 62, 12, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 255, 255, 73, 113, 165, 163, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 125, 189, 171, 153, 83, 86, 245, 62, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 132, 1, 0, 0, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 137, 118, 214, 52, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
28, 94, 51, 157, 189, 105, 178, 114, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 75, 82, 185, 108, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 76, 196, 240, 134, 116, 222, 86,
142, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10,
64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 68, 118, 240, 19, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 213, 192, 104, 120,
74, 119, 165, 194, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 51, 83, 179, 106, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 27, 33, 186, 12, 194, 25, 75, 31, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 178, 162, 153, 157, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
4, 66, 90, 148, 24, 80, 49, 170, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 92, 170, 92, 127, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 213, 45, 208, 229, 123, 172, 173, 241,
8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 193,
72, 32, 187, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 50, 33, 217, 128, 43, 173, 61, 224, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 24, 218, 108, 30, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 115, 212, 228, 103,
43, 138, 93, 217, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 163, 172, 90, 168, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 135, 51, 116, 56, 95, 226, 184, 221, 12, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 255, 255, 5, 39, 91, 20, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 10, 81, 175, 58, 168, 40, 36, 112, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 131, 1, 0, 0, 12, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 255, 255, 45, 142, 235, 46, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 183, 57, 59, 58, 77, 216, 203, 0, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 16, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 185, 157, 160,
119, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 115, 192, 241, 99, 119, 228, 23, 129, 20, 114, 112, 99, 95, 99, 114, 101, 100,
105, 116, 115, 95, 112, 101, 114, 95, 104, 97, 115, 104, 6, 0, 0, 64, 0, 8, 114, 112,
99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 5, 129, 54, 131, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 180, 136, 168, 128, 35, 8,
229, 160, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114,
10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 185, 10, 68, 240, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 105, 127, 240, 231,
209, 2, 20, 136, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 32, 221, 180, 229, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 107, 14, 156, 75, 92, 180, 143, 152, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 218, 88, 21,
188, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 51, 234, 157, 182, 229, 103, 39, 88, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 5, 161, 73, 151, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 209, 162, 121, 222, 241,
125, 142, 156, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 162, 226, 61, 8, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 6, 64, 210, 168, 209, 134, 142, 135, 16, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 104, 238, 128, 24, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 252,
199, 163, 220, 223, 220, 85, 167, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101,
101, 100, 6, 132, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 12, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 89, 147,
109, 91, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 90, 73, 143, 67, 50, 148, 50, 51, 8, 114, 112, 99, 95, 112, 111, 114, 116,
7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 60, 225, 148, 220, 6, 109, 95, 112, 111, 114, 116, 7, 169, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 246, 236, 191, 85, 5, 65, 238, 143, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 84, 255, 235, 77, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 204,
171, 86, 75, 156, 202, 194, 17, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 172, 86, 75, 188, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 102, 18, 220, 13, 62, 231, 50, 12, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 128, 1, 0, 0, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 47, 154, 174,
191, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 142, 90, 23, 242, 187, 113, 45, 135, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 162, 253, 155, 86, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 115, 140, 18, 86, 209, 1,
136, 191, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 192, 42, 253, 215, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 172, 48, 83, 16, 150, 49, 179, 117, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 213, 239, 210, 144, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 191,
25, 48, 149, 62, 193, 163, 51, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 213, 251, 235, 252, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 30, 76, 94, 11, 19, 53, 222, 168, 8,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 82, 101,
246, 21, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 79, 214, 218, 34, 12, 184, 235, 67, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 88, 19, 42, 245, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 252, 148, 134, 97, 173,
59, 174, 174, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 94, 63, 121, 77, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 218, 250, 149, 133, 175, 197, 220, 9, 12, 112, 114, 117,
110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 134, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8,
4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 24, 49, 114, 184, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 71, 65,
104, 43, 162, 145, 13, 85, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 76, 185, 18, 32, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 7, 159, 72, 170, 11, 137, 53, 215, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 135, 1, 0, 0, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 68, 73, 201, 40, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
172, 77, 69, 220, 58, 59, 195, 192, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 84, 255,
235, 77, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2,
105, 100, 5, 204, 171, 86, 75, 156, 202, 194, 17, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 165, 227, 34, 105, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 154, 230, 103, 180,
139, 210, 99, 101, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 35, 236, 113, 185, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 49, 104, 224, 149, 51, 239, 29, 76, 16, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 165, 22, 12,
133, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 246, 50, 244, 226, 219, 198, 24, 105, 12, 112, 114, 117, 110, 105, 110, 103,
95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7,
169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 94, 156, 174, 100, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 183, 117, 141, 111, 95, 90, 58, 204, 12, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 5, 255, 100, 208, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 125,
25, 139, 251, 139, 116, 172, 24, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 161, 70,
16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 86,
94, 156, 130, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 181, 75, 11, 73, 214, 93, 48, 250, 12, 112, 114, 117, 110, 105, 110,
103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116,
7, 169, 70, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100,
114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 93, 95, 230, 245, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 84, 157, 243,
239, 183, 35, 54, 0, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6,
130, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 86, 14, 58, 98, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 131,
145, 164, 127, 110, 147, 147, 166, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 52, 15, 111, 162, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 196, 112, 61, 95, 122, 150, 241, 171,
8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 135, 181, 164, 52, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 52, 20, 78, 121, 22, 214,
174, 79, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114,
10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 159, 69, 121, 11, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 205, 44, 250, 54,
193, 111, 54, 245, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6,
128, 1, 0, 0, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 185, 241, 197, 18, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 161, 144, 210, 66, 72, 38, 182, 154, 8, 114, 112, 99,
95, 112, 111, 114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 24, 165, 109, 172, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 139, 244, 62, 181, 35, 101, 190,
130, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
95, 168, 216, 7, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 113, 28, 114, 184, 37, 154, 224, 95, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 138, 201, 120, 71, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 189, 245, 211,
35, 245, 56, 24, 89, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 91, 206, 14, 98, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 43, 84, 225, 240, 218, 249, 219, 193, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 61, 245, 153,
149, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 111, 172, 135, 36, 174, 143, 223, 112, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255,
31, 28, 25, 99, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
2, 2, 105, 100, 5, 129, 160, 102, 202, 168, 75, 67, 246, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 217, 168, 137, 14, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 198, 134, 120,
100, 110, 70, 149, 242, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 65, 21, 134, 181, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 192, 152, 237, 245, 35, 179, 133, 92, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 82, 29, 15, 7,
6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100,
5, 93, 238, 22, 254, 190, 156, 170, 53, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 100, 36, 215, 42, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 225, 217, 27, 239, 55, 77,
130, 218, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 221, 138, 212, 98, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 221, 115, 168, 71, 115, 113, 123, 226, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 95, 158, 72, 195, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 169,
55, 84, 253, 73, 235, 126, 1, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 47, 5, 146, 126, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 36, 7, 132, 185, 123, 44, 79, 9, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 255, 255, 199, 102, 237, 44, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 23, 191, 140, 80, 125, 5, 151, 123, 8,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 69, 163,
90, 6, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 2, 215, 175, 128, 152, 162, 90, 120, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 147, 135, 136, 35, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 151, 75, 230, 186, 160,
198, 175, 124, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100,
114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 83, 137, 41, 10, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 180, 29, 120,
16, 126, 72, 126, 35, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 45, 62, 207, 10, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 131, 144, 70, 177, 35, 135, 112, 205, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 134, 1, 0, 0, 8, 114, 112, 99,
95, 112, 111, 114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 98, 225, 91, 191, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 19, 165, 172, 148, 44, 52, 241,
105, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
88, 212, 32, 151, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 183, 237, 176, 171, 69, 241, 107, 86, 8, 114, 112, 99, 95, 112, 111,
114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97,
100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 51, 68, 214, 143, 6,
109, 95, 112, 111, 114, 116, 7, 48, 17, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5,
206, 248, 161, 56, 86, 125, 195, 243, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 152, 67, 175, 91, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 17, 83, 56, 240, 65, 163, 170,
158, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8,
114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 216, 232, 100, 178, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 167, 109, 78,
13, 99, 252, 5, 228, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97,
100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 146, 59, 0, 16, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 177,
145, 118, 10, 116, 105, 64, 90, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 185, 10, 68, 240, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 105, 127, 240, 231, 209, 2, 20, 136, 12,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 89, 2,
142, 180, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 1, 21, 198, 102, 36, 134, 38, 249, 8, 114, 112, 99, 95, 112, 111, 114,
116, 7, 161, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100,
100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 158, 140, 230, 233, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 0, 77,
233, 80, 62, 71, 201, 59, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101,
100, 6, 130, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 78, 106, 12, 87, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 119, 128, 137, 92, 63, 246, 81, 72, 12, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 145, 255, 252,
43, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 209, 7, 199, 121, 195, 30, 91, 168, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7,
169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 71, 219, 43, 129, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 165, 255, 243, 173, 120, 214, 100, 239, 12, 112, 114, 117, 110,
105, 110, 103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
255, 255, 65, 21, 137, 242, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 2, 2, 105, 100, 5, 56, 81, 223, 232, 243, 53, 150, 199, 8, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 76, 183, 153, 43, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
2, 169, 223, 201, 166, 219, 0, 108, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 146, 190, 226, 170, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 184, 126, 145, 86, 17, 171, 106,
193, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 99, 234,
74, 130, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 75, 225, 51, 71, 34, 147, 131, 93, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 34, 243, 233, 242, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 241, 169, 58, 47, 204,
254, 64, 194, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 37, 1, 201, 53, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 218, 61, 215, 233, 72, 244, 247, 85, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 100, 0, 31, 209, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 254, 223,
214, 77, 175, 11, 224, 27, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 38, 242, 201, 74, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 55, 246, 56, 7, 244, 33, 111, 185, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 178, 128, 226,
166, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 100, 250, 203, 11, 176, 225, 42, 24, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 45, 238, 67, 67, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 5, 230, 42, 159, 3, 83,
146, 72, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 178, 237, 47, 93, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 217, 28,
236, 27, 5, 79, 1, 193, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 104, 243, 43, 115, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 182, 193, 32, 251, 5, 189, 35, 25, 12, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 99, 190, 128, 178, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
147, 131, 114, 216, 68, 204, 57, 31, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115,
101, 101, 100, 6, 134, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 72, 49, 210, 248, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 87, 24, 255, 212, 205, 237, 120, 57, 8, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 85, 158, 27,
112, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 108, 53, 202, 66, 69, 64, 211, 203, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 144, 76, 58, 247, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 22, 231, 179, 203, 218,
156, 64, 157, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 128, 1,
0, 0, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
144, 24, 238, 60, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 196, 95, 184, 97, 94, 165, 98, 28, 12, 112, 114, 117, 110, 105, 110,
103, 95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 142, 112, 70, 86, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 2, 21, 138, 98, 236, 208,
52, 197, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 213, 29, 219, 208, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 101, 229, 157, 96, 78, 149, 79, 16, 8, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 64, 44, 139, 100, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 7, 203,
58, 177, 99, 127, 167, 27, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 104, 63, 131, 202, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 177, 178, 79, 176, 17, 114, 135, 128, 12, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 58, 211,
149, 42, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 254, 82, 29, 104, 17, 156, 165, 19, 12, 112, 114, 117, 110, 105, 110, 103,
95, 115, 101, 101, 100, 6, 131, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 66, 235, 43, 120, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 140, 48, 91, 174, 88, 66, 197,
27, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10,
64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 209, 182, 235, 183, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 20, 196, 15, 35,
230, 214, 44, 106, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 107, 213, 91, 83, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 43, 184, 117, 30, 44, 105, 95, 158, 12, 112, 114, 117,
110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 128, 1, 0, 0, 16, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 134, 122, 101, 227, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 34,
132, 26, 158, 228, 246, 37, 122, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101,
101, 100, 6, 135, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 5, 144, 96,
90, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 53, 151, 49, 57, 210, 176, 123, 26, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255,
45, 129, 183, 236, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 2, 2, 105, 100, 5, 114, 226, 183, 173, 149, 80, 29, 207, 12, 112, 114, 117, 110,
105, 110, 103, 95, 115, 101, 101, 100, 6, 132, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 96, 48, 253, 223, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 138, 193, 53,
95, 29, 24, 29, 67, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 209, 141, 137, 143, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 142, 18, 157, 53, 139, 6, 171, 205, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 132, 1, 0, 0, 12, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 195, 252, 42, 19, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
11, 193, 221, 161, 77, 26, 136, 247, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 161,
70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
136, 34, 220, 232, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 188, 71, 195, 46, 165, 220, 100, 247, 8, 3, 97, 100, 114, 12, 8,
4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 136, 50, 23, 101, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 142, 104,
228, 114, 184, 184, 155, 149, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 79, 23, 184, 106, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 9, 143, 211, 219, 201, 209, 78, 126, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 131, 1, 0, 0, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 70, 172, 62,
186, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 90, 74, 7, 27, 88, 104, 96, 63, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 83, 48, 63, 194, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 157, 41, 130, 225, 209, 57,
190, 227, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 192, 18, 141, 11, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 47, 149, 84, 128, 33, 31, 39, 27, 12, 112, 114, 117, 110,
105, 110, 103, 95, 115, 101, 101, 100, 6, 128, 1, 0, 0, 8, 114, 112, 99, 95, 112, 111,
114, 116, 7, 169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97,
100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 198, 251, 83, 134, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5,
240, 74, 2, 160, 144, 140, 44, 91, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70,
12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 173,
255, 205, 142, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 13, 193, 21, 155, 217, 217, 26, 41, 12, 112, 114, 117, 110, 105, 110,
103, 95, 115, 101, 101, 100, 6, 130, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 170, 39, 103, 46, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 110, 132, 154, 110, 72,
158, 220, 125, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 149, 202, 95, 149, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 227, 35, 58, 212, 189, 211, 121, 171, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 131, 1, 0, 0, 8, 114, 112, 99,
95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114,
12, 8, 4, 109, 95, 105, 112, 6, 20, 200, 83, 5, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 76, 28, 183, 239, 120, 19, 32, 226, 8,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 67, 191,
0, 235, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 46, 88, 119, 219, 137, 174, 116, 168, 12, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 178, 174, 135, 72, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 196, 76, 198, 87,
49, 143, 40, 85, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 128,
1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114,
10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255, 255, 139, 99, 124, 170, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 144, 217, 56,
205, 38, 203, 209, 20, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 46, 124, 180, 230, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 26, 138, 117, 131, 113, 28, 20, 89, 12, 112, 114,
117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 134, 1, 0, 0, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 185, 64, 105, 115, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 35,
92, 145, 24, 195, 195, 28, 241, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8,
4, 109, 95, 105, 112, 6, 136, 36, 57, 103, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 121, 160, 232, 102, 215, 85, 156, 52, 12,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 168, 235,
93, 184, 6, 109, 95, 112, 111, 114, 116, 7, 51, 212, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 158, 89, 86, 15, 219, 4, 128, 76, 12, 112, 114, 117, 110, 105, 110, 103,
95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 66, 205, 213, 214, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 169, 3, 234, 0, 122, 41, 147,
3, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
141, 98, 255, 143, 6, 109, 95, 112, 111, 114, 116, 7, 177, 214, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 73, 127, 80, 122, 34, 152, 103, 89, 12, 3, 97, 100, 114, 12, 8,
4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 255, 255, 172, 104, 177, 190, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 2, 2, 105, 100, 5, 55, 98, 90, 98, 247, 151, 65, 145, 8, 114, 112,
99, 95, 112, 111, 114, 116, 7, 161, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 77, 172, 230, 31, 6, 109, 95, 112, 111, 114, 116,
7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 37, 18, 78, 110, 19, 172, 149,
68, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
162, 218, 65, 156, 6, 109, 95, 112, 111, 114, 116, 7, 204, 71, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 34, 212, 28, 89, 219, 217, 198, 30, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 109, 145, 130, 189, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 210, 119, 203,
62, 62, 33, 221, 76, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 73, 97, 224, 138, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 233, 45, 113, 162, 132, 119, 238, 222, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 255, 255, 221, 234, 36, 39, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 231, 98, 155, 99, 120, 161, 139, 47,
8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 68,
109, 164, 26, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 143, 41, 243, 191, 26, 133, 133, 148, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 255,
255, 99, 234, 74, 130, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 2, 2, 105, 100, 5, 75, 225, 51, 71, 34, 147, 131, 93, 12, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 172, 104, 177, 190, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 55,
98, 90, 98, 247, 151, 65, 145, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 161, 70, 8,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 89, 39,
107, 63, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 253, 235, 42, 162, 145, 123, 118, 118, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 86, 19, 164, 20, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 254, 16, 203, 169,
195, 236, 197, 71, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 45, 132, 245, 124, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 203, 217, 203, 62, 235, 68, 79, 246, 8, 114, 112,
99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100,
114, 12, 8, 4, 109, 95, 105, 112, 6, 78, 47, 1, 42, 6, 109, 95, 112, 111, 114, 116, 7,
160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 62, 193, 188, 52, 253, 250, 177,
78, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
24, 9, 196, 18, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 181, 175, 149, 70, 255, 230, 120, 144, 12, 112, 114, 117, 110, 105,
110, 103, 95, 115, 101, 101, 100, 6, 131, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 81, 174, 147, 70, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 139, 117, 247, 251,
252, 120, 207, 122, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 24, 46, 131, 31, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 254, 139, 52, 145, 46, 192, 114, 218, 12, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 96, 41, 220, 52, 6,
109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5,
132, 117, 138, 75, 120, 0, 85, 225, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169,
70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
185, 104, 122, 37, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 134, 166, 173, 179, 232, 66, 0, 248, 8, 114, 112, 99, 95, 112,
111, 114, 116, 7, 161, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 71, 127, 156, 63, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 173, 130, 134, 129, 151, 73, 173, 117, 8,
114, 112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 132, 248, 211, 188, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 243, 244, 6,
201, 85, 211, 59, 11, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 221, 121, 132, 104, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 164, 8, 50, 88, 16, 202, 71, 19, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 98, 177, 226,
32, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 13, 149, 116, 215, 92, 35, 192, 137, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 107, 191, 99, 95, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 35, 130, 252, 209, 208,
87, 41, 16, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 82, 64, 20, 76, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 96, 72, 181, 187, 232, 78, 92, 185, 12, 3, 97, 100, 114, 12,
8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 188, 68, 50, 194, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 135, 253,
6, 30, 218, 46, 47, 159, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 161, 70, 8, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 159, 100, 254,
56, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 106, 11, 135, 247, 255, 124, 31, 27, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 23, 128, 248, 240, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 86, 157, 138, 8, 39, 38,
167, 237, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 95, 142, 45, 13, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 68, 53, 120, 88, 65, 106, 55, 62, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 139, 99, 124, 170, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 144, 217, 56,
205, 38, 203, 209, 20, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109,
95, 105, 112, 6, 149, 102, 137, 246, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 252, 192, 231, 60, 235, 176, 190, 192, 8, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 82, 66, 187,
193, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 167, 249, 42, 214, 200, 13, 147, 63, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 54, 36, 174, 4, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 1, 91, 183, 129, 113,
151, 228, 218, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 192, 9, 184, 123, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 237, 203, 144, 195, 93, 91, 36, 253, 12, 112, 114, 117,
110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 132, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8,
4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 146, 190, 105, 218, 6, 109, 95,
112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 106, 83,
142, 16, 72, 134, 65, 203, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 202, 112, 0, 98, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 254, 101, 149, 141, 81, 187, 225, 93, 12, 3,
97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 45, 67, 217,
103, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 0, 127, 172, 186, 143, 155, 201, 145, 8, 114, 112, 99, 95, 112, 111, 114, 116,
7, 169, 70, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105,
112, 6, 85, 247, 59, 77, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 226, 116, 172, 104, 27, 129, 198, 120, 12, 112, 114, 117,
110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 129, 1, 0, 0, 8, 114, 112, 99, 95, 112,
111, 114, 116, 7, 169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4,
109, 95, 105, 112, 6, 172, 93, 53, 9, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4,
116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 41, 95, 47, 152, 22, 90, 49, 102, 12, 112,
114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 132, 1, 0, 0, 16, 3, 97, 100,
114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 255, 255, 71, 237, 129, 0, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 2, 2, 105, 100, 5, 115, 63, 166, 229, 245, 229, 122, 239, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 135, 1, 0, 0, 8, 114,
112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 82, 121, 36, 202, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 216, 167, 229, 40, 235,
137, 72, 52, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 131, 1,
0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
118, 36, 228, 156, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101,
8, 1, 2, 105, 100, 5, 33, 1, 180, 80, 243, 161, 76, 54, 8, 3, 97, 100, 114, 12, 8, 4,
97, 100, 100, 114, 12, 8, 4, 97, 100, 100, 114, 10, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
255, 255, 73, 235, 168, 79, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 2, 2, 105, 100, 5, 120, 102, 85, 162, 254, 2, 48, 208, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 204, 12, 201, 36, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 239,
100, 244, 177, 67, 52, 196, 181, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 216, 249, 90, 57, 6, 109, 95, 112, 111, 114, 116, 7, 160,
70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 222, 93, 114, 147, 216, 204, 49, 151,
8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 83,
137, 41, 10, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1,
2, 105, 100, 5, 180, 29, 120, 16, 126, 72, 126, 35, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 63, 143, 48, 18, 6, 109, 95, 112, 111,
114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 193, 47, 218, 50,
81, 222, 85, 43, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 152, 89, 216, 153, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 31, 11, 87, 67, 7, 192, 1, 13, 8, 3, 97, 100, 114,
12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 77, 95, 229, 224, 6, 109,
95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 160,
119, 196, 130, 136, 75, 140, 54, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 82, 69, 12, 29, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 208, 247, 146, 51, 217, 73, 193, 226, 12,
3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 49, 12,
239, 156, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2,
105, 100, 5, 146, 129, 163, 49, 89, 133, 44, 64, 8, 114, 112, 99, 95, 112, 111, 114,
116, 7, 169, 70, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 185, 240, 242, 36, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116,
121, 112, 101, 8, 1, 2, 105, 100, 5, 197, 219, 86, 2, 247, 136, 229, 29, 12, 3, 97,
100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 134, 122, 61,
72, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105,
100, 5, 29, 223, 119, 202, 113, 165, 81, 7, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7,
169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112,
6, 195, 154, 242, 41, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112,
101, 8, 1, 2, 105, 100, 5, 177, 250, 184, 155, 105, 57, 83, 47, 8, 114, 112, 99, 95,
112, 111, 114, 116, 7, 169, 70, 16, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12,
8, 4, 109, 95, 105, 112, 6, 99, 248, 2, 29, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70,
4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 10, 209, 218, 249, 58, 74, 147, 253, 12,
112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 114,
112, 99, 95, 112, 111, 114, 116, 7, 169, 70, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100,
100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 51, 75, 64, 249, 6, 109, 95, 112, 111, 114,
116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 183, 101, 253, 145, 35,
156, 129, 45, 12, 112, 114, 117, 110, 105, 110, 103, 95, 115, 101, 101, 100, 6, 134, 1,
0, 0, 12, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6,
185, 203, 56, 7, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8,
1, 2, 105, 100, 5, 241, 202, 217, 195, 142, 200, 163, 55, 12, 112, 114, 117, 110, 105,
110, 103, 95, 115, 101, 101, 100, 6, 133, 1, 0, 0, 8, 3, 97, 100, 114, 12, 8, 4, 97,
100, 100, 114, 12, 8, 4, 109, 95, 105, 112, 6, 136, 243, 145, 198, 6, 109, 95, 112,
111, 114, 116, 7, 160, 70, 4, 116, 121, 112, 101, 8, 1, 2, 105, 100, 5, 53, 77, 124,
92, 86, 55, 9, 77, 8, 3, 97, 100, 114, 12, 8, 4, 97, 100, 100, 114, 12, 8, 4, 109, 95,
105, 112, 6, 46, 28, 204, 223, 6, 109, 95, 112, 111, 114, 116, 7, 160, 70, 4, 116, 121,
112, 101, 8, 1, 2, 105, 100, 5, 183, 162, 58, 143, 188, 33, 92, 33, 9, 110, 111, 100,
101, 95, 100, 97, 116, 97, 12, 20, 7, 109, 121, 95, 112, 111, 114, 116, 6, 160, 70, 0,
0, 10, 110, 101, 116, 119, 111, 114, 107, 95, 105, 100, 10, 64, 18, 48, 241, 113, 97,
4, 65, 97, 23, 49, 0, 130, 22, 161, 161, 16, 7, 112, 101, 101, 114, 95, 105, 100, 5,
172, 170, 135, 122, 19, 151, 202, 83, 8, 114, 112, 99, 95, 112, 111, 114, 116, 7, 169,
70, 13, 115, 117, 112, 112, 111, 114, 116, 95, 102, 108, 97, 103, 115, 6, 1, 0, 0, 0,
12, 112, 97, 121, 108, 111, 97, 100, 95, 100, 97, 116, 97, 12, 24, 21, 99, 117, 109,
117, 108, 97, 116, 105, 118, 101, 95, 100, 105, 102, 102, 105, 99, 117, 108, 116, 121,
5, 25, 78, 56, 125, 251, 152, 97, 3, 27, 99, 117, 109, 117, 108, 97, 116, 105, 118,
101, 95, 100, 105, 102, 102, 105, 99, 117, 108, 116, 121, 95, 116, 111, 112, 54, 52, 5,
0, 0, 0, 0, 0, 0, 0, 0, 14, 99, 117, 114, 114, 101, 110, 116, 95, 104, 101, 105, 103,
104, 116, 5, 127, 88, 42, 0, 0, 0, 0, 0, 12, 112, 114, 117, 110, 105, 110, 103, 95,
115, 101, 101, 100, 6, 130, 1, 0, 0, 6, 116, 111, 112, 95, 105, 100, 10, 128, 64, 120,
0, 114, 218, 233, 18, 49, 8, 89, 154, 159, 101, 133, 242, 71, 77, 3, 247, 182, 219,
181, 216, 193, 135, 23, 186, 168, 207, 119, 86, 235, 11, 116, 111, 112, 95, 118, 101,
114, 115, 105, 111, 110, 8, 16,
];
let handshake: HandshakeResponse = epee_serde::from_bytes(bytes).unwrap();
let basic_node_data = BasicNodeData {
my_port: 18080,
network_id: [
18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161, 161, 16,
],
peer_id: PeerID(6037804360359455404),
support_flags: 1,
rpc_port: 18089,
rpc_credits_per_hash: 0,
};
let core_sync_data = CoreSyncData {
cumulative_difficulty: 243644060759772697,
cumulative_difficulty_top64: 0,
current_height: 2775167,
pruning_seed: 386,
top_id: Hash::from_str(
"0x40780072dae9123108599a9f6585f2474d03f7b6dbb5d8c18717baa8cf7756eb",
)
.unwrap(),
top_version: 16,
};
assert_eq!(basic_node_data, handshake.node_data);
assert_eq!(core_sync_data, handshake.payload_data);
assert_eq!(250, handshake.local_peerlist_new.len());
let encoded_bytes = epee_serde::to_bytes(&handshake).unwrap();
let handshake_2: HandshakeResponse = epee_serde::from_bytes(encoded_bytes).unwrap();
assert_eq!(handshake, handshake_2);
}
}

View file

@ -0,0 +1,238 @@
//! Common types that are used across multiple messages.
//
use epee_serde::Value;
use monero::{Block, Hash, Transaction};
use serde::de;
use serde::ser::SerializeStruct;
use serde::{Deserialize, Serialize};
use serde_with::serde_as;
use serde_with::TryFromInto;
use super::zero_val;
use crate::NetworkAddress;
/// A PeerID, different from a `NetworkAddress`
#[derive(Debug, Clone, Copy, Deserialize, Serialize, PartialEq, Eq)]
#[serde(transparent)]
pub struct PeerID(pub u64);
/// Basic Node Data, information on the connected peer
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct BasicNodeData {
/// Port
pub my_port: u32,
/// The Network Id
pub network_id: [u8; 16],
/// Peer ID
pub peer_id: PeerID,
/// The Peers Support Flags
/// (If this is not in the message the default is 0)
#[serde(default = "zero_val")]
pub support_flags: u32,
/// RPC Port
/// (If this is not in the message the default is 0)
#[serde(default = "zero_val")]
pub rpc_port: u16,
/// RPC Credits Per Hash
/// (If this is not in the message the default is 0)
#[serde(default = "zero_val")]
pub rpc_credits_per_hash: u32,
}
/// Core Sync Data, information on the sync state of a peer
#[serde_as]
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq, Eq)]
pub struct CoreSyncData {
/// Cumulative Difficulty Low
/// The lower 64 bits of the 128 bit cumulative difficulty
pub cumulative_difficulty: u64,
/// Cumulative Difficulty High
/// The upper 64 bits of the 128 bit cumulative difficulty
#[serde(default = "zero_val")]
pub cumulative_difficulty_top64: u64,
/// Current Height of the peer
pub current_height: u64,
/// Pruning Seed of the peer
/// (If this is not in the message the default is 0)
#[serde(default = "zero_val")]
pub pruning_seed: u32,
/// Hash of the top block
#[serde_as(as = "TryFromInto<[u8; 32]>")]
pub top_id: Hash,
/// Version of the top block
#[serde(default = "zero_val")]
pub top_version: u8,
}
impl CoreSyncData {
/// Returns the 128 bit cumulative difficulty of the peers blockchain
pub fn cumulative_difficulty(&self) -> u128 {
let mut ret: u128 = self.cumulative_difficulty_top64 as u128;
ret <<= 64;
ret | self.cumulative_difficulty as u128
}
}
/// PeerListEntryBase, information kept on a peer which will be entered
/// in a peer list/store.
#[derive(Clone, Copy, Deserialize, Serialize, Debug, Eq, PartialEq)]
pub struct PeerListEntryBase {
/// The Peer Address
pub adr: NetworkAddress,
/// The Peer ID
pub id: PeerID,
/// The last Time The Peer Was Seen
#[serde(default = "zero_val")]
pub last_seen: i64,
/// The Pruning Seed
#[serde(default = "zero_val")]
pub pruning_seed: u32,
/// The RPC port
#[serde(default = "zero_val")]
pub rpc_port: u16,
/// The RPC credits per hash
#[serde(default = "zero_val")]
pub rpc_credits_per_hash: u32,
}
/// A pruned tx with the hash of the missing prunable data
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct TxBlobEntry {
/// The Tx
pub tx: Transaction, // ########### use pruned transaction when PR is merged ##############
/// The Prunable Tx Hash
pub prunable_hash: Hash,
}
impl TxBlobEntry {
fn from_epee_value<E: de::Error>(value: &Value) -> Result<Self, E> {
let tx_blob = get_val_from_map!(value, "blob", get_bytes, "Vec<u8>");
let tx = monero_decode_into_serde_err!(Transaction, tx_blob);
let prunable_hash_blob = get_val_from_map!(value, "prunable_hash", get_bytes, "Vec<u8>");
let prunable_hash = Hash::from_slice(prunable_hash_blob);
Ok(Self { tx, prunable_hash })
}
}
impl Serialize for TxBlobEntry {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
let mut state = serializer.serialize_struct("", 2)?;
let tx_blob = monero::consensus::serialize(&self.tx);
state.serialize_field("blob", &tx_blob)?;
let prunable_hash = self.prunable_hash.as_bytes();
state.serialize_field("prunable_hash", prunable_hash)?;
state.end()
}
}
/// A Block that can contain transactions
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct BlockCompleteEntry {
/// True if tx data is pruned
pub pruned: bool,
/// The Block
pub block: Block,
/// The Block Weight/Size
pub block_weight: u64,
/// If the Block is pruned the txs will be here
pub txs_pruned: Vec<TxBlobEntry>,
/// If the Block is not pruned the txs will be here
pub txs: Vec<Transaction>,
}
impl<'de> Deserialize<'de> for BlockCompleteEntry {
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
where
D: serde::Deserializer<'de>,
{
let value = Value::deserialize(deserializer)?;
let mut pruned = false;
if let Some(val) = value.get("pruned") {
pruned = *get_internal_val!(val, get_bool, "bool");
}
let block_bytes = get_val_from_map!(value, "block", get_bytes, "Vec<u8>");
let block = monero_decode_into_serde_err!(Block, block_bytes);
let mut block_weight = 0;
let mut txs_pruned = vec![];
let mut txs = vec![];
if pruned {
block_weight = *get_val_from_map!(value, "block_weight", get_u64, "u64");
if let Some(v) = value.get("txs") {
let v = get_internal_val!(v, get_seq, "a sequence");
txs_pruned.reserve(v.len());
for val in v {
txs_pruned.push(TxBlobEntry::from_epee_value(val)?);
}
}
} else if let Some(v) = value.get("txs") {
let v = get_internal_val!(v, get_seq, "a sequence");
txs.reserve(v.len());
for val in v {
let tx_buf = get_internal_val!(val, get_bytes, "Vec<u8>");
txs.push(monero_decode_into_serde_err!(Transaction, tx_buf));
}
}
Ok(BlockCompleteEntry {
pruned,
block,
block_weight,
txs_pruned,
txs,
})
}
}
impl Serialize for BlockCompleteEntry {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
let mut len = 1;
if !self.txs.is_empty() || !self.txs_pruned.is_empty() {
len += 1;
}
if self.pruned {
// one field to store the value of `pruned`
// another to sore the block weight
len += 2;
}
let mut state = serializer.serialize_struct("", len)?;
let block = monero::consensus::serialize(&self.block);
state.serialize_field("block", &block)?;
if self.pruned {
state.serialize_field("pruned", &true)?;
state.serialize_field("block_weight", &self.block_weight)?;
if !self.txs_pruned.is_empty() {
state.serialize_field("txs", &self.txs_pruned)?;
}
} else if !self.txs.is_empty() {
let mut tx_blobs = vec![];
for tx in self.txs.iter() {
tx_blobs.push(monero::consensus::serialize(tx));
}
state.serialize_field("txs", &tx_blobs)?;
}
state.end()
}
}

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,144 @@
//! This module defines the addresses that will get passed around the
//! Monero network. Core Monero has 4 main addresses: IPv4, IPv6, Tor,
//! I2p. Currently this module only has IPv(4/6).
//!
use std::{hash::Hash, net};
use epee_serde::Value;
use serde::{de, ser::SerializeStruct, Deserialize, Serialize};
/// An IPv4 address with a port
#[derive(Clone, Copy, Serialize, Debug, PartialEq, Eq, Hash)]
pub struct IPv4Address {
/// IP address
pub m_ip: u32,
/// Port
pub m_port: u16,
}
impl From<net::SocketAddrV4> for IPv4Address {
fn from(value: net::SocketAddrV4) -> Self {
IPv4Address {
m_ip: u32::from_le_bytes(value.ip().octets()),
m_port: value.port(),
}
}
}
impl IPv4Address {
fn from_value<E: de::Error>(value: &Value) -> Result<Self, E> {
let m_ip = get_val_from_map!(value, "m_ip", get_u32, "u32");
let m_port = get_val_from_map!(value, "m_port", get_u16, "u16");
Ok(IPv4Address {
m_ip: *m_ip,
m_port: *m_port,
})
}
}
/// An IPv6 address with a port
#[derive(Clone, Copy, Serialize, Debug, PartialEq, Eq, Hash)]
pub struct IPv6Address {
/// Address
pub addr: [u8; 16],
/// Port
pub m_port: u16,
}
impl From<net::SocketAddrV6> for IPv6Address {
fn from(value: net::SocketAddrV6) -> Self {
IPv6Address {
addr: value.ip().octets(),
m_port: value.port(),
}
}
}
impl IPv6Address {
fn from_value<E: de::Error>(value: &Value) -> Result<Self, E> {
let addr = get_val_from_map!(value, "addr", get_bytes, "Vec<u8>");
let m_port = get_val_from_map!(value, "m_port", get_u16, "u16");
Ok(IPv6Address {
addr: addr
.clone()
.try_into()
.map_err(|_| E::invalid_length(addr.len(), &"a 16-byte array"))?,
m_port: *m_port,
})
}
}
/// A network address which can be encoded into the format required
/// to send to other Monero peers.
#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash)]
pub enum NetworkAddress {
/// IPv4
IPv4(IPv4Address),
/// IPv6
IPv6(IPv6Address),
}
impl From<net::SocketAddrV4> for NetworkAddress {
fn from(value: net::SocketAddrV4) -> Self {
NetworkAddress::IPv4(value.into())
}
}
impl From<net::SocketAddrV6> for NetworkAddress {
fn from(value: net::SocketAddrV6) -> Self {
NetworkAddress::IPv6(value.into())
}
}
impl From<net::SocketAddr> for NetworkAddress {
fn from(value: net::SocketAddr) -> Self {
match value {
net::SocketAddr::V4(v4) => v4.into(),
net::SocketAddr::V6(v6) => v6.into(),
}
}
}
impl<'de> Deserialize<'de> for NetworkAddress {
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
where
D: serde::Deserializer<'de>,
{
let value = Value::deserialize(deserializer)?;
let addr_type = get_val_from_map!(value, "type", get_u8, "u8");
Ok(match addr_type {
1 => NetworkAddress::IPv4(IPv4Address::from_value(get_field_from_map!(value, "addr"))?),
2 => NetworkAddress::IPv6(IPv6Address::from_value(get_field_from_map!(value, "addr"))?),
_ => {
return Err(de::Error::custom(
"Network address type currently unsupported",
))
}
})
}
}
impl Serialize for NetworkAddress {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
let mut state = serializer.serialize_struct("", 2)?;
match self {
NetworkAddress::IPv4(v) => {
state.serialize_field("type", &1_u8)?;
state.serialize_field("addr", v)?;
}
NetworkAddress::IPv6(v) => {
state.serialize_field("type", &2_u8)?;
state.serialize_field("addr", v)?;
}
}
state.end()
}
}