From 80a4983c66d432d1e67b4a27c36fcd8af997dc5a Mon Sep 17 00:00:00 2001 From: Paul Bardea Date: Thu, 25 Jun 2020 01:03:29 -0400 Subject: [PATCH] sql: add metadata router This commit adds a DistSQL router that routes metadata to a given output stream based on the metadata's StreamIdx. This is used by flows which schedule DistSQL processors in order to coordinate work around the cluster. The motivation for this change is a refactoring to Restore which attempts to distribute the work of performing the restore across the cluster. RESTORE works by creating a pipeline of work with 2 stages. The first stage splits and scatters the ranges we are going to import. This shuffling means that a range could end up on a random node in the cluster. The second stage of the pipeline is to download the data from the backup file and restore the data, which is accomplished through an AddSSTable request. It is beneficial (as well as the motivation for this refactor) for the node which issues this request to also be the leaseholder of the range it is trying to import. This is to prevent a situation where many nodes are all waiting on one node that had the misfortune of being the recipient of many scatter ranges in a row. This router would allow restore to be implemented with 2 separate processors: one that splits and scatters the data, and one that imports the data. Using this router, the split and scatter processor could determine, on the fly, which processor is suitable to import the data. Release note: None --- pkg/sql/execinfra/testutils.go | 67 ++++- pkg/sql/execinfrapb/data.go | 11 +- pkg/sql/execinfrapb/data.pb.go | 320 ++++++++++++----------- pkg/sql/execinfrapb/data.proto | 3 + pkg/sql/execinfrapb/flow_diagram.go | 2 + pkg/sql/rowflow/routers.go | 85 ++++++ pkg/sql/rowflow/routers_test.go | 158 ++++++++++- pkg/testutils/distsqlutils/row_buffer.go | 16 ++ 8 files changed, 498 insertions(+), 164 deletions(-) diff --git a/pkg/sql/execinfra/testutils.go b/pkg/sql/execinfra/testutils.go index 4a5b863c83a1..f6c0b6d15924 100644 --- a/pkg/sql/execinfra/testutils.go +++ b/pkg/sql/execinfra/testutils.go @@ -25,6 +25,13 @@ import ( // StaticNodeID is the default Node ID to be used in tests. const StaticNodeID = roachpb.NodeID(3) +type RepeatableSource interface { + RowSource + // Reset resets the RepeatableSource such that a subsequent call to Next() + // returns the first row. + Reset() +} + // RepeatableRowSource is a RowSource used in benchmarks to avoid having to // reinitialize a new RowSource every time during multiple passes of the input. // It is intended to be initialized with all rows. @@ -66,8 +73,7 @@ func (r *RepeatableRowSource) Next() (sqlbase.EncDatumRow, *execinfrapb.Producer return nextRow, nil } -// Reset resets the RepeatableRowSource such that a subsequent call to Next() -// returns the first row. +// Reset is part of the RepeatableSource interface. func (r *RepeatableRowSource) Reset() { r.nextRowIdx = 0 } @@ -78,6 +84,63 @@ func (r *RepeatableRowSource) ConsumerDone() {} // ConsumerClosed is part of the RowSource interface. func (r *RepeatableRowSource) ConsumerClosed() {} +// TODO(pbardea): Document. +func MakeMetas(numMeta, numOutputs int) []*execinfrapb.ProducerMetadata { + metas := make([]*execinfrapb.ProducerMetadata, numMeta) + uniqueMetas := make([]*execinfrapb.ProducerMetadata, numOutputs) + for i := range metas { + metas[i] = &execinfrapb.ProducerMetadata{StreamIdx: i, SamplerProgress: &execinfrapb.RemoteProducerMetadata_SamplerProgress{ + RowsProcessed: 10, + }} + } + for i := range metas { + metas[i] = uniqueMetas[i%numOutputs] + } + return metas +} + +type RepeatableMetaSource struct { + metas []*execinfrapb.ProducerMetadata + nextMetaIdx int +} + +var _ RowSource = &RepeatableMetaSource{} + +// NewRepeatableMetaSource creates a RepeatableMetaSource with the given metas. +func NewRepeatableMetaSource(metas []*execinfrapb.ProducerMetadata) *RepeatableMetaSource { + return &RepeatableMetaSource{metas: metas} +} + +// OutputTypes is part of the RowSource interface. +func (r *RepeatableMetaSource) OutputTypes() []*types.T { + return []*types.T{} +} + +// Start is part of the RowSource interface. +func (r *RepeatableMetaSource) Start(ctx context.Context) context.Context { return ctx } + +// Next is part of the RowSource interface. +func (r *RepeatableMetaSource) Next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadata) { + // If we've emitted all metadata entries, signal that we have reached the end. + if r.nextMetaIdx >= len(r.metas) { + return nil, nil + } + nextMeta := r.metas[r.nextMetaIdx] + r.nextMetaIdx++ + return nil, nextMeta +} + +// Reset is part of the RepeatableSource interface. +func (r *RepeatableMetaSource) Reset() { + r.nextMetaIdx = 0 +} + +// ConsumerDone is part of the RowSource interface. +func (r *RepeatableMetaSource) ConsumerDone() {} + +// ConsumerClosed is part of the RowSource interface. +func (r *RepeatableMetaSource) ConsumerClosed() {} + // NewTestMemMonitor creates and starts a new memory monitor to be used in // tests. // TODO(yuzefovich): consider reusing this in tree.MakeTestingEvalContext diff --git a/pkg/sql/execinfrapb/data.go b/pkg/sql/execinfrapb/data.go index 2b5b11eebf51..6d3c679720f1 100644 --- a/pkg/sql/execinfrapb/data.go +++ b/pkg/sql/execinfrapb/data.go @@ -231,9 +231,14 @@ func (e *Error) ErrorDetail(ctx context.Context) (err error) { // ProducerMetadata represents a metadata record flowing through a DistSQL flow. type ProducerMetadata struct { - // Only one of these fields will be set. If this ever changes, note that - // there're consumers out there that extract the error and, if there is one, - // forward it in isolation and drop the rest of the record. + // StreamIdx informs DistSQL the index of the output stream that this metadata + // should be sent on when hooked up to a BY_META router. This is used by + // metadata-only streams (e.g. restore). This will be set in conjunction with + // another field. + StreamIdx int + // If StreamIdx is not set, only one of these fields will be set. If this ever + // changes, note that there're consumers out there that extract the error and, + // if there is one, forward it in isolation and drop the rest of the record. Ranges []roachpb.RangeInfo // TODO(vivek): change to type Error Err error diff --git a/pkg/sql/execinfrapb/data.pb.go b/pkg/sql/execinfrapb/data.pb.go index 45a76e1e6f8f..5fbf4ceaed7a 100644 --- a/pkg/sql/execinfrapb/data.pb.go +++ b/pkg/sql/execinfrapb/data.pb.go @@ -71,7 +71,7 @@ func (x *Ordering_Column_Direction) UnmarshalJSON(data []byte) error { return nil } func (Ordering_Column_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{2, 0, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{2, 0, 0} } type StreamEndpointSpec_Type int32 @@ -116,7 +116,7 @@ func (x *StreamEndpointSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (StreamEndpointSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{3, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{3, 0} } type InputSyncSpec_Type int32 @@ -156,7 +156,7 @@ func (x *InputSyncSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{4, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{4, 0} } type OutputRouterSpec_Type int32 @@ -172,6 +172,9 @@ const ( // Each row is sent to one stream, chosen according to preset boundaries // for the values of certain columns of the row. OutputRouterSpec_BY_RANGE OutputRouterSpec_Type = 3 + // Each metadata is sent to one stream, specified within the metadata + // itself. + OutputRouterSpec_BY_META OutputRouterSpec_Type = 4 ) var OutputRouterSpec_Type_name = map[int32]string{ @@ -179,12 +182,14 @@ var OutputRouterSpec_Type_name = map[int32]string{ 1: "MIRROR", 2: "BY_HASH", 3: "BY_RANGE", + 4: "BY_META", } var OutputRouterSpec_Type_value = map[string]int32{ "PASS_THROUGH": 0, "MIRROR": 1, "BY_HASH": 2, "BY_RANGE": 3, + "BY_META": 4, } func (x OutputRouterSpec_Type) Enum() *OutputRouterSpec_Type { @@ -204,7 +209,7 @@ func (x *OutputRouterSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{5, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{5, 0} } // Error is a generic representation including a string message. @@ -217,7 +222,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{0} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +250,7 @@ var xxx_messageInfo_Error proto.InternalMessageInfo func (m *Expression) Reset() { *m = Expression{} } func (*Expression) ProtoMessage() {} func (*Expression) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{1} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{1} } func (m *Expression) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -280,7 +285,7 @@ func (m *Ordering) Reset() { *m = Ordering{} } func (m *Ordering) String() string { return proto.CompactTextString(m) } func (*Ordering) ProtoMessage() {} func (*Ordering) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{2} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{2} } func (m *Ordering) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -314,7 +319,7 @@ func (m *Ordering_Column) Reset() { *m = Ordering_Column{} } func (m *Ordering_Column) String() string { return proto.CompactTextString(m) } func (*Ordering_Column) ProtoMessage() {} func (*Ordering_Column) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{2, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{2, 0} } func (m *Ordering_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -361,7 +366,7 @@ func (m *StreamEndpointSpec) Reset() { *m = StreamEndpointSpec{} } func (m *StreamEndpointSpec) String() string { return proto.CompactTextString(m) } func (*StreamEndpointSpec) ProtoMessage() {} func (*StreamEndpointSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{3} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{3} } func (m *StreamEndpointSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +405,7 @@ func (m *InputSyncSpec) Reset() { *m = InputSyncSpec{} } func (m *InputSyncSpec) String() string { return proto.CompactTextString(m) } func (*InputSyncSpec) ProtoMessage() {} func (*InputSyncSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{4} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{4} } func (m *InputSyncSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -444,7 +449,7 @@ func (m *OutputRouterSpec) Reset() { *m = OutputRouterSpec{} } func (m *OutputRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec) ProtoMessage() {} func (*OutputRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{5} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{5} } func (m *OutputRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -487,7 +492,7 @@ func (m *OutputRouterSpec_RangeRouterSpec) Reset() { *m = OutputRouterSp func (m *OutputRouterSpec_RangeRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{5, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{5, 0} } func (m *OutputRouterSpec_RangeRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -529,7 +534,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) String() string { } func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{5, 0, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{5, 0, 0} } func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -566,7 +571,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_Span) Reset() { *m = OutputRou func (m *OutputRouterSpec_RangeRouterSpec_Span) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec_Span) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{5, 0, 1} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{5, 0, 1} } func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -600,7 +605,7 @@ func (m *DatumInfo) Reset() { *m = DatumInfo{} } func (m *DatumInfo) String() string { return proto.CompactTextString(m) } func (*DatumInfo) ProtoMessage() {} func (*DatumInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{6} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{6} } func (m *DatumInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -635,7 +640,7 @@ func (m *ProducerHeader) Reset() { *m = ProducerHeader{} } func (m *ProducerHeader) String() string { return proto.CompactTextString(m) } func (*ProducerHeader) ProtoMessage() {} func (*ProducerHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{7} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{7} } func (m *ProducerHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -678,7 +683,7 @@ func (m *ProducerData) Reset() { *m = ProducerData{} } func (m *ProducerData) String() string { return proto.CompactTextString(m) } func (*ProducerData) ProtoMessage() {} func (*ProducerData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{8} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{8} } func (m *ProducerData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +725,7 @@ func (m *ProducerMessage) Reset() { *m = ProducerMessage{} } func (m *ProducerMessage) String() string { return proto.CompactTextString(m) } func (*ProducerMessage) ProtoMessage() {} func (*ProducerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{9} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{9} } func (m *ProducerMessage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +771,7 @@ func (m *RemoteProducerMetadata) Reset() { *m = RemoteProducerMetadata{} func (m *RemoteProducerMetadata) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata) ProtoMessage() {} func (*RemoteProducerMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10} } func (m *RemoteProducerMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1090,7 +1095,7 @@ func (m *RemoteProducerMetadata_RangeInfos) Reset() { *m = RemoteProduce func (m *RemoteProducerMetadata_RangeInfos) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RangeInfos) ProtoMessage() {} func (*RemoteProducerMetadata_RangeInfos) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 0} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 0} } func (m *RemoteProducerMetadata_RangeInfos) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1123,7 +1128,7 @@ func (m *RemoteProducerMetadata_TraceData) Reset() { *m = RemoteProducer func (m *RemoteProducerMetadata_TraceData) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_TraceData) ProtoMessage() {} func (*RemoteProducerMetadata_TraceData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 1} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 1} } func (m *RemoteProducerMetadata_TraceData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1169,7 @@ func (m *RemoteProducerMetadata_RowNum) Reset() { *m = RemoteProducerMet func (m *RemoteProducerMetadata_RowNum) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RowNum) ProtoMessage() {} func (*RemoteProducerMetadata_RowNum) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 2} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 2} } func (m *RemoteProducerMetadata_RowNum) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1204,7 +1209,7 @@ func (m *RemoteProducerMetadata_SamplerProgress) Reset() { func (m *RemoteProducerMetadata_SamplerProgress) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_SamplerProgress) ProtoMessage() {} func (*RemoteProducerMetadata_SamplerProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 3} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 3} } func (m *RemoteProducerMetadata_SamplerProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1245,7 +1250,7 @@ func (m *RemoteProducerMetadata_BulkProcessorProgress) String() string { } func (*RemoteProducerMetadata_BulkProcessorProgress) ProtoMessage() {} func (*RemoteProducerMetadata_BulkProcessorProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 4} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 4} } func (m *RemoteProducerMetadata_BulkProcessorProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1282,7 +1287,7 @@ func (m *RemoteProducerMetadata_Metrics) Reset() { *m = RemoteProducerMe func (m *RemoteProducerMetadata_Metrics) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_Metrics) ProtoMessage() {} func (*RemoteProducerMetadata_Metrics) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{10, 5} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{10, 5} } func (m *RemoteProducerMetadata_Metrics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1322,7 +1327,7 @@ func (m *DistSQLVersionGossipInfo) Reset() { *m = DistSQLVersionGossipIn func (m *DistSQLVersionGossipInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLVersionGossipInfo) ProtoMessage() {} func (*DistSQLVersionGossipInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{11} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{11} } func (m *DistSQLVersionGossipInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1363,7 @@ func (m *DistSQLDrainingInfo) Reset() { *m = DistSQLDrainingInfo{} } func (m *DistSQLDrainingInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLDrainingInfo) ProtoMessage() {} func (*DistSQLDrainingInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c66f806c4cbd67bf, []int{12} + return fileDescriptor_data_db3b7d282f9bd2e3, []int{12} } func (m *DistSQLDrainingInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5858,136 +5863,137 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_c66f806c4cbd67bf) } +func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_db3b7d282f9bd2e3) } -var fileDescriptor_data_c66f806c4cbd67bf = []byte{ - // 2046 bytes of a gzipped FileDescriptorProto +var fileDescriptor_data_db3b7d282f9bd2e3 = []byte{ + // 2053 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcd, 0x73, 0x1b, 0x49, - 0x15, 0xd7, 0xc8, 0xb2, 0x3e, 0x9e, 0x6d, 0x59, 0x6e, 0xb2, 0x59, 0x21, 0x82, 0xed, 0xd5, 0x52, - 0x54, 0x80, 0xdd, 0xd1, 0xae, 0x73, 0xc8, 0x62, 0xa0, 0x12, 0xcb, 0x92, 0x23, 0x65, 0x1d, 0xdb, - 0x3b, 0x72, 0xd8, 0xca, 0x2e, 0xd4, 0x30, 0x9a, 0x69, 0xc9, 0x43, 0xe6, 0x2b, 0xdd, 0x3d, 0xb1, - 0x7d, 0xa1, 0xa8, 0x82, 0x23, 0x07, 0x4e, 0x14, 0xc7, 0xe5, 0xc2, 0x85, 0x03, 0x37, 0xfe, 0x04, - 0xc8, 0x89, 0xda, 0x63, 0x8a, 0x43, 0x0a, 0x9c, 0x0b, 0x47, 0xae, 0xec, 0x89, 0xea, 0x2f, 0x7d, - 0xc5, 0x4e, 0xf0, 0x66, 0x2f, 0x76, 0xf7, 0x7b, 0xef, 0xf7, 0xeb, 0xf7, 0x5e, 0xbf, 0x7e, 0xdd, - 0x1a, 0xa8, 0xd1, 0x47, 0x41, 0x03, 0x9f, 0x60, 0xd7, 0x8f, 0x06, 0xc4, 0x49, 0xfa, 0x0d, 0xcf, - 0x61, 0x8e, 0x99, 0x90, 0x98, 0xc5, 0xa8, 0xea, 0xc6, 0xee, 0x43, 0x12, 0x3b, 0xee, 0x91, 0x49, - 0x1f, 0x05, 0xa6, 0xe7, 0x53, 0x46, 0x1f, 0x05, 0x24, 0x8d, 0x6a, 0x57, 0x7f, 0x1e, 0xf7, 0x69, - 0x83, 0xff, 0x49, 0xfa, 0xe2, 0x9f, 0x44, 0xd4, 0x56, 0x84, 0x75, 0xd2, 0x6f, 0x38, 0x89, 0xaf, - 0x44, 0x48, 0x8b, 0xc6, 0xc4, 0xb5, 0x2b, 0x5a, 0x86, 0x09, 0x89, 0x89, 0x06, 0xbf, 0x21, 0x67, - 0xb3, 0xe2, 0x35, 0xee, 0x61, 0x32, 0x3c, 0xf6, 0x09, 0x6e, 0x24, 0x43, 0xa1, 0x9a, 0x36, 0xb8, - 0xc6, 0x0d, 0xe8, 0xa3, 0xa0, 0xef, 0x50, 0xdc, 0xa0, 0x8c, 0xa4, 0x2e, 0x4b, 0x09, 0xf6, 0x26, - 0xe1, 0x5a, 0x8b, 0x23, 0x37, 0xf6, 0xb0, 0x67, 0x7b, 0x0e, 0x4b, 0x43, 0xbd, 0x2c, 0x37, 0x60, - 0xa7, 0x09, 0xa6, 0xf2, 0xaf, 0x12, 0xaf, 0xa7, 0xcc, 0x0f, 0x1a, 0x8c, 0x38, 0xae, 0x1f, 0x0d, - 0x1b, 0x04, 0xbb, 0x31, 0xe1, 0x48, 0x9a, 0x38, 0x91, 0x8e, 0x62, 0x18, 0x0f, 0x63, 0x31, 0x6c, - 0xf0, 0x91, 0x92, 0x56, 0x05, 0xee, 0x28, 0x70, 0x1b, 0xcc, 0x0f, 0x31, 0x65, 0x4e, 0x98, 0x28, - 0xcd, 0xd7, 0x87, 0x71, 0x3c, 0x0c, 0x70, 0x43, 0xcc, 0xfa, 0xe9, 0xa0, 0xe1, 0x44, 0xa7, 0x52, - 0x55, 0xff, 0x04, 0xe6, 0xdb, 0x3c, 0x24, 0x74, 0x0b, 0x60, 0x90, 0x06, 0x81, 0x2d, 0x02, 0xac, - 0xce, 0xad, 0x1b, 0xd7, 0x17, 0x36, 0xd6, 0xcd, 0xf1, 0x3e, 0xe8, 0x14, 0x99, 0x6d, 0x19, 0x89, - 0x40, 0x59, 0x25, 0x8e, 0x11, 0xc3, 0xcd, 0xdc, 0xef, 0x3f, 0x5b, 0xcb, 0xdc, 0xcd, 0x15, 0x8d, - 0x4a, 0xf6, 0x6e, 0xae, 0x98, 0xad, 0xcc, 0xd5, 0x0f, 0x00, 0xda, 0x27, 0x09, 0xc1, 0x94, 0xfa, - 0x71, 0x84, 0x56, 0xa1, 0xf0, 0x18, 0x13, 0x3e, 0xac, 0x1a, 0xeb, 0xc6, 0xf5, 0x52, 0x33, 0xf7, - 0xe4, 0xd9, 0x5a, 0xc6, 0xd2, 0x42, 0x54, 0x85, 0x1c, 0x3e, 0x49, 0x48, 0x35, 0x3b, 0xa1, 0x14, - 0x92, 0xcd, 0x22, 0x67, 0xfe, 0xcf, 0x1f, 0xd6, 0x32, 0xf5, 0x5f, 0x67, 0xa1, 0xb8, 0x4f, 0x3c, - 0x4c, 0xfc, 0x68, 0x88, 0xba, 0x50, 0x70, 0xe3, 0x20, 0x0d, 0x23, 0x5a, 0x35, 0xd6, 0xe7, 0xae, - 0x2f, 0x6c, 0x7c, 0xc7, 0xbc, 0xa8, 0x6c, 0x4c, 0x0d, 0x32, 0xb7, 0x05, 0x42, 0xaf, 0xad, 0xf0, - 0xb5, 0x3f, 0x1a, 0x90, 0x97, 0x1a, 0xf4, 0x4d, 0xc1, 0x6a, 0xfb, 0xde, 0x89, 0x70, 0x73, 0x49, - 0x99, 0xe6, 0xdd, 0x38, 0xe8, 0x7a, 0x27, 0xe8, 0x63, 0x28, 0x79, 0x3e, 0xc1, 0x2e, 0xe3, 0x71, - 0x70, 0x57, 0xcb, 0x1b, 0x37, 0xfe, 0xef, 0x65, 0xcd, 0x96, 0x86, 0x2a, 0xd6, 0x31, 0x57, 0x7d, - 0x15, 0x4a, 0x23, 0x2d, 0x2a, 0xc0, 0xdc, 0x56, 0x6f, 0xbb, 0x92, 0x41, 0x45, 0xc8, 0xb5, 0xda, - 0xbd, 0xed, 0x8a, 0xb1, 0x99, 0xfb, 0xf7, 0x67, 0x6b, 0xea, 0x6f, 0xfd, 0x6f, 0x59, 0x40, 0x3d, - 0x46, 0xb0, 0x13, 0xb6, 0x23, 0x2f, 0x89, 0xfd, 0x88, 0xf5, 0x12, 0xec, 0xa2, 0x0f, 0x21, 0xc7, - 0xeb, 0x48, 0xf8, 0x5d, 0xde, 0x78, 0xff, 0x62, 0xb7, 0x5e, 0xc4, 0x9a, 0x87, 0xa7, 0x09, 0xd6, - 0x49, 0xe7, 0x24, 0xe8, 0xfb, 0x50, 0xa2, 0xc2, 0xcc, 0xf6, 0x3d, 0x11, 0xe8, 0x7c, 0xf3, 0x1a, - 0x57, 0x9f, 0x3d, 0x5b, 0x2b, 0x4a, 0x7c, 0xb7, 0xf5, 0xc5, 0xc4, 0xd8, 0x2a, 0x4a, 0xf3, 0xae, - 0x87, 0x1e, 0x41, 0x99, 0x39, 0x64, 0x88, 0x99, 0x1d, 0xc5, 0x1e, 0xe6, 0xf8, 0x9c, 0xc0, 0x7f, - 0xa8, 0xf0, 0x8b, 0x87, 0x42, 0xbb, 0x17, 0x7b, 0x58, 0x70, 0xdc, 0x18, 0xfa, 0xec, 0x28, 0xed, - 0x9b, 0x6e, 0x1c, 0x36, 0x46, 0x3e, 0x7b, 0xfd, 0xf1, 0xb8, 0x91, 0x3c, 0x1c, 0x36, 0xd4, 0xc9, - 0x35, 0x25, 0xcc, 0x5a, 0x64, 0x63, 0x12, 0xaf, 0xfe, 0x1e, 0xe4, 0x78, 0x04, 0xa8, 0x04, 0xf3, - 0xbb, 0xfb, 0xdb, 0x5b, 0xbb, 0x95, 0x0c, 0x02, 0xc8, 0x5b, 0xed, 0x7b, 0xfb, 0x87, 0xed, 0x8a, - 0x81, 0x56, 0x60, 0xa9, 0xf7, 0x60, 0x6f, 0xdb, 0xb6, 0xda, 0xbd, 0x83, 0xfd, 0xbd, 0x5e, 0x5b, - 0x94, 0xe8, 0x5c, 0x25, 0x57, 0xff, 0x7b, 0x16, 0x96, 0xba, 0x51, 0x92, 0xb2, 0xde, 0x69, 0xe4, - 0x8a, 0x24, 0xee, 0x4c, 0x25, 0xf1, 0x9d, 0x8b, 0x93, 0x38, 0x05, 0x7b, 0x31, 0x7f, 0x2d, 0x28, - 0xc6, 0x6a, 0xf7, 0x45, 0xfa, 0x16, 0x36, 0xea, 0xaf, 0xae, 0x13, 0xc5, 0x30, 0x42, 0xa2, 0x5d, - 0x28, 0xc8, 0xb4, 0xd2, 0xea, 0x9c, 0xa8, 0xf1, 0x77, 0x2e, 0xb3, 0xab, 0xba, 0xcc, 0x15, 0x05, - 0xfa, 0x11, 0x2c, 0xca, 0x8a, 0xb7, 0x45, 0xbf, 0xa9, 0xe6, 0x04, 0x65, 0x6d, 0x86, 0x92, 0xe2, - 0xd0, 0x94, 0xfd, 0xe8, 0xd0, 0x5a, 0x90, 0xf6, 0x3c, 0x3a, 0x5a, 0xaf, 0xab, 0x24, 0x2f, 0x41, - 0xe9, 0xfe, 0xde, 0xbe, 0xd5, 0x6a, 0x5b, 0xed, 0x56, 0x25, 0x83, 0x16, 0xa0, 0xa0, 0x27, 0x46, - 0xfd, 0xbf, 0x79, 0xa8, 0xec, 0xa7, 0x2c, 0x49, 0x99, 0x15, 0xa7, 0x0c, 0x13, 0x91, 0xd3, 0xee, - 0x54, 0x4e, 0x1b, 0x2f, 0xc9, 0xc3, 0x0c, 0xf2, 0xc5, 0xb4, 0x4e, 0x24, 0x24, 0xfb, 0xfa, 0x09, - 0x79, 0x0b, 0x16, 0x8f, 0x1c, 0x7a, 0x64, 0xeb, 0x3e, 0xc2, 0x73, 0xbc, 0x64, 0x2d, 0x70, 0x99, - 0x3c, 0xb1, 0x14, 0x05, 0xb0, 0x42, 0x9c, 0x68, 0x88, 0x6d, 0x22, 0xbc, 0xb2, 0x69, 0x82, 0x5d, - 0x51, 0xcf, 0x0b, 0x1b, 0x9b, 0x97, 0x08, 0xc4, 0xe2, 0x1c, 0xe3, 0xb9, 0x72, 0x64, 0x99, 0x4c, - 0x8b, 0xd1, 0xfb, 0xb0, 0xe2, 0xf9, 0xd4, 0xe9, 0x07, 0xd8, 0xee, 0xa7, 0x83, 0x81, 0x2c, 0x9f, - 0xf9, 0x75, 0xe3, 0x7a, 0x51, 0x21, 0x2a, 0x4a, 0xdd, 0xd4, 0xda, 0xda, 0x5f, 0xe7, 0x60, 0x79, - 0x86, 0x1d, 0x7d, 0x0a, 0xf3, 0xfc, 0xba, 0xd0, 0x8d, 0xf1, 0xd6, 0x97, 0x77, 0xd4, 0xec, 0x25, - 0x8e, 0xee, 0x56, 0x92, 0x93, 0x27, 0xcd, 0xc3, 0x03, 0x27, 0x0d, 0x98, 0xed, 0x61, 0xca, 0x64, - 0x73, 0xb0, 0x16, 0x94, 0xac, 0x85, 0x29, 0x43, 0x21, 0x94, 0xc4, 0x85, 0xe7, 0x47, 0x43, 0x5d, - 0xb8, 0xdd, 0xd7, 0xf0, 0x41, 0xee, 0x45, 0x5b, 0x31, 0xea, 0xde, 0x39, 0x5a, 0xa1, 0xf6, 0x18, - 0xca, 0xd3, 0x26, 0xe8, 0x1a, 0xe4, 0xe5, 0x9e, 0xbe, 0xd0, 0xc4, 0x79, 0x8f, 0xdf, 0x81, 0xa2, - 0x06, 0xab, 0x1e, 0xfe, 0xad, 0xd9, 0x33, 0x20, 0xaf, 0x6d, 0xb3, 0xc5, 0xaf, 0xeb, 0x99, 0x85, - 0x47, 0xd8, 0xda, 0x2e, 0xe4, 0x78, 0x7a, 0xd0, 0x15, 0x98, 0xa7, 0xcc, 0x21, 0x4c, 0x2c, 0xb6, - 0x68, 0xc9, 0x09, 0xaa, 0xc0, 0x1c, 0x8e, 0x64, 0xef, 0x5c, 0xb4, 0xf8, 0x90, 0x7b, 0x25, 0x2b, - 0x4f, 0xdc, 0xaf, 0xf3, 0xda, 0x2b, 0x29, 0xab, 0xdf, 0x52, 0xc7, 0xab, 0x02, 0x8b, 0x07, 0x5b, - 0xbd, 0x9e, 0x7d, 0xd8, 0xb1, 0xf6, 0xef, 0xdf, 0xe9, 0xc8, 0x56, 0x76, 0xaf, 0x6b, 0x59, 0xfb, - 0x56, 0xc5, 0xe0, 0xa7, 0xad, 0xf9, 0xc0, 0xee, 0x6c, 0xf5, 0x3a, 0x95, 0x2c, 0x5a, 0x84, 0x62, - 0xf3, 0x81, 0x6d, 0x6d, 0xed, 0xdd, 0x69, 0x57, 0xe6, 0xea, 0xbf, 0x32, 0xa0, 0x24, 0x1c, 0xee, - 0x46, 0x83, 0x78, 0x2a, 0x48, 0xe3, 0xcb, 0x07, 0x89, 0x4c, 0x75, 0x78, 0x65, 0x13, 0x7b, 0x59, - 0xb3, 0x10, 0x76, 0xf5, 0x5f, 0x40, 0xf9, 0x80, 0xc4, 0x5e, 0xea, 0x62, 0xd2, 0xc1, 0x8e, 0x87, - 0x09, 0x7a, 0x1f, 0x0a, 0x83, 0x20, 0x3e, 0xe6, 0x17, 0x81, 0x48, 0x50, 0xb3, 0xca, 0x97, 0xf8, - 0xc7, 0xb3, 0xb5, 0xfc, 0x4e, 0x10, 0x1f, 0x77, 0x5b, 0x67, 0xa3, 0x91, 0x95, 0xe7, 0x86, 0x5d, - 0xef, 0x35, 0x6e, 0x9f, 0xfa, 0x9f, 0x0c, 0x58, 0xd4, 0x0e, 0xb4, 0x1c, 0xe6, 0xa0, 0x6f, 0x40, - 0x89, 0x38, 0xc7, 0x76, 0xff, 0x94, 0x61, 0xaa, 0x76, 0xa8, 0x48, 0x9c, 0xe3, 0x26, 0x9f, 0x23, - 0x0b, 0x8a, 0x21, 0x66, 0x0e, 0x7f, 0x22, 0xaa, 0x86, 0xf2, 0xde, 0xc5, 0x85, 0x6a, 0xe1, 0x30, - 0x66, 0x58, 0x93, 0xdf, 0x53, 0x38, 0x9d, 0x31, 0xcd, 0x83, 0xbe, 0x0b, 0xe5, 0x28, 0x0d, 0x6d, - 0x1c, 0x26, 0xec, 0xd4, 0x26, 0xf1, 0x31, 0x9d, 0xda, 0xee, 0xc5, 0x28, 0x0d, 0xdb, 0x5c, 0x65, - 0xc5, 0xc7, 0xb4, 0xfe, 0xd4, 0x80, 0xe5, 0x31, 0x21, 0xa5, 0xce, 0x10, 0xa3, 0xdb, 0x90, 0x3f, - 0x12, 0x99, 0x13, 0xde, 0x2e, 0x6c, 0x5c, 0xbf, 0xd8, 0xa3, 0xe9, 0x4c, 0x5b, 0x0a, 0x87, 0xb6, - 0x20, 0xcf, 0x4e, 0x13, 0x59, 0xde, 0x3c, 0xa6, 0xb7, 0x2f, 0x66, 0x18, 0x15, 0x8c, 0xae, 0x46, - 0x09, 0x44, 0xb7, 0x21, 0x27, 0x92, 0x22, 0x5f, 0x82, 0xdf, 0x7e, 0xb5, 0x0b, 0xad, 0x71, 0x2a, - 0x04, 0xb2, 0xfe, 0x97, 0x32, 0x5c, 0x3d, 0x3f, 0x63, 0xe8, 0x27, 0x00, 0xb2, 0xa9, 0xfa, 0xd1, - 0x20, 0x56, 0x51, 0xfe, 0xe0, 0xb2, 0x79, 0x97, 0x6d, 0x82, 0xbb, 0x4e, 0x3b, 0x19, 0xab, 0x44, - 0xf4, 0x0c, 0xdd, 0x84, 0x79, 0xf9, 0x8a, 0x95, 0x25, 0xbb, 0x76, 0x31, 0xb1, 0x78, 0xb9, 0x76, - 0x32, 0x96, 0xb4, 0x47, 0x9f, 0x02, 0xf0, 0x67, 0x37, 0xb6, 0x27, 0x22, 0xdf, 0xbc, 0xb4, 0x5b, - 0x87, 0x9c, 0x82, 0x67, 0x83, 0x7b, 0xc5, 0xf4, 0x04, 0x7d, 0x0c, 0x57, 0x02, 0xec, 0x0c, 0x6c, - 0x76, 0x12, 0xd9, 0x03, 0x3f, 0x72, 0x02, 0x9b, 0x32, 0x87, 0x61, 0x75, 0x97, 0x4c, 0x9e, 0x4d, - 0xfd, 0xd2, 0xd9, 0xc5, 0xce, 0xe0, 0xf0, 0x24, 0xda, 0xe1, 0xc6, 0x3d, 0x6e, 0xdb, 0xc9, 0x58, - 0x2b, 0xc1, 0xac, 0x10, 0x59, 0x50, 0x20, 0xf1, 0xb1, 0x1d, 0xa5, 0xa1, 0xb8, 0x29, 0x16, 0x36, - 0x6e, 0x5e, 0x3e, 0x93, 0xf1, 0xf1, 0x5e, 0x1a, 0x76, 0x32, 0x56, 0x9e, 0x88, 0x11, 0x0a, 0xa1, - 0x42, 0x9d, 0x30, 0x09, 0x30, 0xb1, 0x13, 0x12, 0x0f, 0xf9, 0x1b, 0xbe, 0x5a, 0x10, 0xe4, 0xb7, - 0x2f, 0x4d, 0xde, 0x93, 0x44, 0x07, 0x8a, 0xa7, 0x93, 0xb1, 0x96, 0xe9, 0xb4, 0x08, 0x1d, 0x42, - 0x21, 0xc4, 0x8c, 0xf8, 0x2e, 0xad, 0x16, 0xc5, 0x2a, 0x1f, 0x5c, 0x7a, 0x95, 0x7b, 0x12, 0xdf, - 0xc9, 0x58, 0x9a, 0x0a, 0xfd, 0xd2, 0x80, 0x37, 0xfb, 0x69, 0xf0, 0x90, 0x87, 0xe0, 0x62, 0x4a, - 0xe3, 0x89, 0x60, 0x4a, 0x62, 0x99, 0x9d, 0x4b, 0x2f, 0xd3, 0x4c, 0x83, 0x87, 0x07, 0x9a, 0x6e, - 0x22, 0xa4, 0x37, 0xfa, 0xe7, 0x29, 0x6a, 0xfb, 0x00, 0xe3, 0x2a, 0x45, 0x5b, 0x33, 0x65, 0xcf, - 0x8f, 0xe6, 0xb5, 0x73, 0x36, 0x7e, 0x04, 0xd1, 0x57, 0xdd, 0xa8, 0xb6, 0x6b, 0x3f, 0x83, 0xd2, - 0xa8, 0xbe, 0x50, 0x0f, 0x96, 0xdd, 0x38, 0x08, 0xb0, 0xcb, 0xd4, 0xef, 0x43, 0x7d, 0xe1, 0x4f, - 0x56, 0x13, 0xff, 0x55, 0x68, 0xaa, 0x5f, 0x93, 0xa6, 0xa5, 0x7e, 0x4d, 0x4e, 0xdc, 0xea, 0xe5, - 0x11, 0x05, 0x17, 0xd2, 0xda, 0x31, 0xe4, 0x65, 0x39, 0xa0, 0x77, 0xa1, 0x44, 0x71, 0xe4, 0x61, - 0xa2, 0x3b, 0x77, 0xa9, 0x59, 0x19, 0x35, 0x61, 0xa1, 0x10, 0x8d, 0x57, 0x8e, 0x3c, 0xfe, 0xcb, - 0x49, 0xd7, 0x61, 0x76, 0xf2, 0x7a, 0x53, 0x25, 0xb5, 0x06, 0xc5, 0xc0, 0xa1, 0xcc, 0x0e, 0xe9, - 0x50, 0x1c, 0x2d, 0xfd, 0xa2, 0x29, 0x70, 0xe9, 0x3d, 0x3a, 0xac, 0x51, 0x58, 0x9e, 0x29, 0x15, - 0xf4, 0x3d, 0x28, 0xf3, 0xfe, 0xa9, 0x37, 0x10, 0x4b, 0x37, 0x72, 0x0a, 0xb9, 0xc4, 0x75, 0x07, - 0x5a, 0x85, 0x6e, 0x00, 0x3a, 0xf2, 0x29, 0x8b, 0x87, 0xc4, 0x09, 0x6d, 0xf5, 0x4c, 0x92, 0x97, - 0x87, 0x5e, 0x6a, 0x65, 0xa4, 0x6f, 0x29, 0x75, 0xed, 0xcf, 0x39, 0x78, 0xe3, 0xdc, 0x3d, 0x45, - 0x3b, 0x3c, 0xb9, 0xdc, 0x9d, 0xd9, 0xe4, 0xbe, 0x79, 0xce, 0x8e, 0x4d, 0xe7, 0x53, 0xa1, 0x44, - 0x3e, 0xd1, 0x6f, 0x0c, 0x40, 0x63, 0xa2, 0x01, 0x71, 0xf4, 0x6f, 0x47, 0xce, 0xf5, 0xd3, 0xaf, - 0xa6, 0x00, 0xcd, 0x6d, 0xbd, 0xc0, 0x8e, 0xe2, 0x6f, 0x47, 0x8c, 0x9c, 0x5a, 0x2b, 0xee, 0xac, - 0x1c, 0x31, 0x00, 0x82, 0x69, 0x1a, 0x62, 0x3b, 0x89, 0xf5, 0xdb, 0xec, 0xfe, 0x57, 0xe4, 0x85, - 0x25, 0x88, 0x0f, 0x62, 0x2a, 0x57, 0x2f, 0x11, 0x3d, 0x47, 0x6d, 0xa8, 0xe8, 0xa3, 0x67, 0x7b, - 0x98, 0x39, 0x7e, 0x40, 0x55, 0xe3, 0xbb, 0x62, 0xca, 0x8f, 0x13, 0xa6, 0xfe, 0x38, 0x61, 0x6e, - 0x45, 0xa7, 0xfa, 0x79, 0xac, 0x31, 0x2d, 0x09, 0xa9, 0xb5, 0xe0, 0xea, 0xf9, 0x91, 0xf2, 0xc7, - 0xd6, 0x43, 0x7c, 0x2a, 0xca, 0x63, 0xde, 0xe2, 0x43, 0xfe, 0x28, 0x7b, 0xec, 0x04, 0xa9, 0x7c, - 0xb8, 0x64, 0x2d, 0x39, 0xd9, 0xcc, 0x7e, 0x60, 0xd4, 0x7e, 0x08, 0xe5, 0x69, 0x4f, 0x5f, 0x85, - 0x9e, 0x9b, 0x44, 0x7f, 0x04, 0x05, 0xd5, 0x6b, 0xd0, 0xdb, 0x00, 0xe2, 0x55, 0x61, 0x13, 0xec, - 0xc8, 0xd2, 0x9c, 0xd3, 0x27, 0x56, 0xc8, 0x2d, 0xec, 0x78, 0xe8, 0x2d, 0x28, 0x89, 0x1a, 0x16, - 0x36, 0xd9, 0x09, 0x9b, 0x22, 0x17, 0x73, 0x93, 0x66, 0x41, 0x2d, 0x76, 0x37, 0x57, 0xcc, 0x57, - 0x0a, 0xf5, 0xdf, 0x19, 0x50, 0x6d, 0xf9, 0x94, 0xf5, 0x3e, 0xda, 0xfd, 0xb1, 0xfc, 0x38, 0x72, - 0x27, 0xa6, 0xd4, 0x4f, 0xc4, 0xe5, 0xf6, 0xde, 0xf4, 0x67, 0x94, 0xa5, 0xe6, 0x55, 0x4e, 0xf6, - 0xc5, 0xb3, 0xb5, 0xf2, 0x34, 0x64, 0xfc, 0x61, 0xa5, 0x03, 0x57, 0x42, 0x3f, 0xb2, 0x1d, 0xd7, - 0xc5, 0x09, 0x2f, 0x41, 0x0d, 0xcf, 0xbe, 0x14, 0x8e, 0x42, 0x3f, 0xda, 0x52, 0x10, 0x25, 0xab, - 0xdf, 0x84, 0xaf, 0x29, 0xab, 0x16, 0x71, 0xfc, 0xc8, 0x8f, 0x86, 0xc2, 0xa5, 0x75, 0x28, 0x7a, - 0x6a, 0x2e, 0x7c, 0xd2, 0xc7, 0x6d, 0x24, 0x6d, 0xbe, 0xfb, 0xe4, 0x5f, 0xab, 0x99, 0x27, 0x67, - 0xab, 0xc6, 0xe7, 0x67, 0xab, 0xc6, 0xd3, 0xb3, 0x55, 0xe3, 0x9f, 0x67, 0xab, 0xc6, 0x6f, 0x9f, - 0xaf, 0x66, 0x3e, 0x7f, 0xbe, 0x9a, 0x79, 0xfa, 0x7c, 0x35, 0xf3, 0xc9, 0xc2, 0xc4, 0x97, 0xc0, - 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x98, 0xe2, 0x25, 0xa3, 0x1b, 0x14, 0x00, 0x00, + 0x15, 0xd7, 0xc8, 0xb2, 0x25, 0x3d, 0xdb, 0xb2, 0xdc, 0x24, 0x59, 0x21, 0x82, 0xed, 0xd5, 0x52, + 0x54, 0x80, 0xdd, 0x51, 0xe2, 0x1c, 0xb2, 0x18, 0xa8, 0x8d, 0x65, 0xc9, 0x91, 0xb2, 0xfe, 0xda, + 0x91, 0xc3, 0x56, 0x76, 0xa1, 0x86, 0xd1, 0x4c, 0x4b, 0x1e, 0x32, 0x5f, 0xe9, 0xee, 0x89, 0xed, + 0x0b, 0x45, 0x15, 0x14, 0x27, 0x0e, 0x9c, 0x28, 0x8e, 0xcb, 0x85, 0x0b, 0x07, 0x6e, 0xfc, 0x09, + 0x90, 0x13, 0xb5, 0xc7, 0x14, 0x87, 0x14, 0x38, 0x17, 0x8e, 0x9c, 0xf7, 0x44, 0xf5, 0x97, 0xbe, + 0x62, 0x27, 0x78, 0xb3, 0x17, 0xbb, 0xfb, 0xbd, 0xf7, 0xfb, 0xf5, 0x7b, 0xaf, 0x5f, 0xbf, 0x6e, + 0x0d, 0x54, 0xe9, 0xe3, 0xa0, 0x8e, 0x4f, 0xb0, 0xeb, 0x47, 0x7d, 0xe2, 0x24, 0xbd, 0xba, 0xe7, + 0x30, 0xc7, 0x4c, 0x48, 0xcc, 0x62, 0x54, 0x71, 0x63, 0xf7, 0x11, 0x89, 0x1d, 0xf7, 0xc8, 0xa4, + 0x8f, 0x03, 0xd3, 0xf3, 0x29, 0xa3, 0x8f, 0x03, 0x92, 0x46, 0xd5, 0x6b, 0x3f, 0x8f, 0x7b, 0xb4, + 0xce, 0xff, 0x24, 0x3d, 0xf1, 0x4f, 0x22, 0xaa, 0xcb, 0xc2, 0x3a, 0xe9, 0xd5, 0x9d, 0xc4, 0x57, + 0x22, 0xa4, 0x45, 0x23, 0xe2, 0xea, 0x15, 0x2d, 0xc3, 0x84, 0xc4, 0x44, 0x83, 0xaf, 0xca, 0xd9, + 0xb4, 0x78, 0x95, 0x7b, 0x98, 0x0c, 0x8e, 0x7d, 0x82, 0xeb, 0xc9, 0x40, 0xa8, 0x26, 0x0d, 0xae, + 0x73, 0x03, 0xfa, 0x38, 0xe8, 0x39, 0x14, 0xd7, 0x29, 0x23, 0xa9, 0xcb, 0x52, 0x82, 0xbd, 0x71, + 0xb8, 0xd6, 0xe2, 0xc8, 0x8d, 0x3d, 0xec, 0xd9, 0x9e, 0xc3, 0xd2, 0x50, 0x2f, 0xcb, 0x0d, 0xd8, + 0x69, 0x82, 0xa9, 0xfc, 0xab, 0xc4, 0x6b, 0x29, 0xf3, 0x83, 0x3a, 0x23, 0x8e, 0xeb, 0x47, 0x83, + 0x3a, 0xc1, 0x6e, 0x4c, 0x38, 0x92, 0x26, 0x4e, 0xa4, 0xa3, 0x18, 0xc4, 0x83, 0x58, 0x0c, 0xeb, + 0x7c, 0xa4, 0xa4, 0x15, 0x81, 0x3b, 0x0a, 0xdc, 0x3a, 0xf3, 0x43, 0x4c, 0x99, 0x13, 0x26, 0x4a, + 0xf3, 0xf5, 0x41, 0x1c, 0x0f, 0x02, 0x5c, 0x17, 0xb3, 0x5e, 0xda, 0xaf, 0x3b, 0xd1, 0xa9, 0x54, + 0xd5, 0x3e, 0x81, 0xd9, 0x16, 0x0f, 0x09, 0x7d, 0x00, 0xd0, 0x4f, 0x83, 0xc0, 0x16, 0x01, 0x56, + 0x66, 0xd6, 0x8c, 0x1b, 0xf3, 0xeb, 0x6b, 0xe6, 0x68, 0x1f, 0x74, 0x8a, 0xcc, 0x96, 0x8c, 0x44, + 0xa0, 0xac, 0x22, 0xc7, 0x88, 0xe1, 0x46, 0xee, 0x0f, 0x9f, 0xad, 0x66, 0xee, 0xe7, 0x0a, 0x46, + 0x39, 0x7b, 0x3f, 0x57, 0xc8, 0x96, 0x67, 0x6a, 0x07, 0x00, 0xad, 0x93, 0x84, 0x60, 0x4a, 0xfd, + 0x38, 0x42, 0x2b, 0x90, 0x7f, 0x82, 0x09, 0x1f, 0x56, 0x8c, 0x35, 0xe3, 0x46, 0xb1, 0x91, 0x7b, + 0xfa, 0x7c, 0x35, 0x63, 0x69, 0x21, 0xaa, 0x40, 0x0e, 0x9f, 0x24, 0xa4, 0x92, 0x1d, 0x53, 0x0a, + 0xc9, 0x46, 0x81, 0x33, 0xff, 0xf7, 0x8f, 0xab, 0x99, 0xda, 0xaf, 0xb3, 0x50, 0xd8, 0x27, 0x1e, + 0x26, 0x7e, 0x34, 0x40, 0x1d, 0xc8, 0xbb, 0x71, 0x90, 0x86, 0x11, 0xad, 0x18, 0x6b, 0x33, 0x37, + 0xe6, 0xd7, 0xbf, 0x63, 0x5e, 0x54, 0x36, 0xa6, 0x06, 0x99, 0x5b, 0x02, 0xa1, 0xd7, 0x56, 0xf8, + 0xea, 0x9f, 0x0c, 0x98, 0x93, 0x1a, 0xf4, 0x4d, 0xc1, 0x6a, 0xfb, 0xde, 0x89, 0x70, 0x73, 0x51, + 0x99, 0xce, 0xb9, 0x71, 0xd0, 0xf1, 0x4e, 0xd0, 0xc7, 0x50, 0xf4, 0x7c, 0x82, 0x5d, 0xc6, 0xe3, + 0xe0, 0xae, 0x96, 0xd6, 0x6f, 0xff, 0xdf, 0xcb, 0x9a, 0x4d, 0x0d, 0x55, 0xac, 0x23, 0xae, 0xda, + 0x0a, 0x14, 0x87, 0x5a, 0x94, 0x87, 0x99, 0xcd, 0xee, 0x56, 0x39, 0x83, 0x0a, 0x90, 0x6b, 0xb6, + 0xba, 0x5b, 0x65, 0x63, 0x23, 0xf7, 0x9f, 0xcf, 0x56, 0xd5, 0xdf, 0xda, 0xdf, 0xb3, 0x80, 0xba, + 0x8c, 0x60, 0x27, 0x6c, 0x45, 0x5e, 0x12, 0xfb, 0x11, 0xeb, 0x26, 0xd8, 0x45, 0x1f, 0x42, 0x8e, + 0xd7, 0x91, 0xf0, 0xbb, 0xb4, 0x7e, 0xeb, 0x62, 0xb7, 0x5e, 0xc6, 0x9a, 0x87, 0xa7, 0x09, 0xd6, + 0x49, 0xe7, 0x24, 0xe8, 0xfb, 0x50, 0xa4, 0xc2, 0xcc, 0xf6, 0x3d, 0x11, 0xe8, 0x6c, 0xe3, 0x3a, + 0x57, 0x9f, 0x3d, 0x5f, 0x2d, 0x48, 0x7c, 0xa7, 0xf9, 0xc5, 0xd8, 0xd8, 0x2a, 0x48, 0xf3, 0x8e, + 0x87, 0x1e, 0x43, 0x89, 0x39, 0x64, 0x80, 0x99, 0x1d, 0xc5, 0x1e, 0xe6, 0xf8, 0x9c, 0xc0, 0x7f, + 0xa8, 0xf0, 0x0b, 0x87, 0x42, 0xbb, 0x17, 0x7b, 0x58, 0x70, 0xdc, 0x1e, 0xf8, 0xec, 0x28, 0xed, + 0x99, 0x6e, 0x1c, 0xd6, 0x87, 0x3e, 0x7b, 0xbd, 0xd1, 0xb8, 0x9e, 0x3c, 0x1a, 0xd4, 0xd5, 0xc9, + 0x35, 0x25, 0xcc, 0x5a, 0x60, 0x23, 0x12, 0xaf, 0x76, 0x13, 0x72, 0x3c, 0x02, 0x54, 0x84, 0xd9, + 0x9d, 0xfd, 0xad, 0xcd, 0x9d, 0x72, 0x06, 0x01, 0xcc, 0x59, 0xad, 0xdd, 0xfd, 0xc3, 0x56, 0xd9, + 0x40, 0xcb, 0xb0, 0xd8, 0x7d, 0xb8, 0xb7, 0x65, 0x5b, 0xad, 0xee, 0xc1, 0xfe, 0x5e, 0xb7, 0x25, + 0x4a, 0x74, 0xa6, 0x9c, 0xab, 0xfd, 0x23, 0x0b, 0x8b, 0x9d, 0x28, 0x49, 0x59, 0xf7, 0x34, 0x72, + 0x45, 0x12, 0xb7, 0x27, 0x92, 0xf8, 0xee, 0xc5, 0x49, 0x9c, 0x80, 0xbd, 0x9c, 0xbf, 0x26, 0x14, + 0x62, 0xb5, 0xfb, 0x22, 0x7d, 0xf3, 0xeb, 0xb5, 0xd7, 0xd7, 0x89, 0x62, 0x18, 0x22, 0xd1, 0x0e, + 0xe4, 0x65, 0x5a, 0x69, 0x65, 0x46, 0xd4, 0xf8, 0xbb, 0x97, 0xd9, 0x55, 0x5d, 0xe6, 0x8a, 0x02, + 0xfd, 0x08, 0x16, 0x64, 0xc5, 0xdb, 0xa2, 0xdf, 0x54, 0x72, 0x82, 0xb2, 0x3a, 0x45, 0x49, 0x71, + 0x68, 0xca, 0x7e, 0x74, 0x68, 0xcd, 0x4b, 0x7b, 0x1e, 0x1d, 0xad, 0xd5, 0x54, 0x92, 0x17, 0xa1, + 0xf8, 0x60, 0x6f, 0xdf, 0x6a, 0xb6, 0xac, 0x56, 0xb3, 0x9c, 0x41, 0xf3, 0x90, 0xd7, 0x13, 0xa3, + 0xf6, 0x9b, 0x3c, 0x94, 0xf7, 0x53, 0x96, 0xa4, 0xcc, 0x8a, 0x53, 0x86, 0x89, 0xc8, 0x69, 0x67, + 0x22, 0xa7, 0xf5, 0x57, 0xe4, 0x61, 0x0a, 0xf9, 0x72, 0x5a, 0xc7, 0x12, 0x92, 0x7d, 0xf3, 0x84, + 0xbc, 0x0d, 0x0b, 0x47, 0x0e, 0x3d, 0xb2, 0x75, 0x1f, 0xe1, 0x39, 0x5e, 0xb4, 0xe6, 0xb9, 0x4c, + 0x9e, 0x58, 0x8a, 0x02, 0x58, 0x26, 0x4e, 0x34, 0xc0, 0x36, 0x11, 0x5e, 0xd9, 0x34, 0xc1, 0xae, + 0xa8, 0xe7, 0xf9, 0xf5, 0x8d, 0x4b, 0x04, 0x62, 0x71, 0x8e, 0xd1, 0x5c, 0x39, 0xb2, 0x44, 0x26, + 0xc5, 0xe8, 0x16, 0x2c, 0x7b, 0x3e, 0x75, 0x7a, 0x01, 0xb6, 0x7b, 0x69, 0xbf, 0x2f, 0xcb, 0x67, + 0x76, 0xcd, 0xb8, 0x51, 0x50, 0x88, 0xb2, 0x52, 0x37, 0xb4, 0xb6, 0xfa, 0xb7, 0x19, 0x58, 0x9a, + 0x62, 0x47, 0x9f, 0xc2, 0x2c, 0xbf, 0x2e, 0x74, 0x63, 0xfc, 0xe0, 0xcb, 0x3b, 0x6a, 0x76, 0x13, + 0x47, 0x77, 0x2b, 0xc9, 0xc9, 0x93, 0xe6, 0xe1, 0xbe, 0x93, 0x06, 0xcc, 0xf6, 0x30, 0x65, 0xb2, + 0x39, 0x58, 0xf3, 0x4a, 0xd6, 0xc4, 0x94, 0xa1, 0x10, 0x8a, 0xe2, 0xc2, 0xf3, 0xa3, 0x81, 0x2e, + 0xdc, 0xce, 0x1b, 0xf8, 0x20, 0xf7, 0xa2, 0xa5, 0x18, 0x75, 0xef, 0x1c, 0xae, 0x50, 0x7d, 0x02, + 0xa5, 0x49, 0x13, 0x74, 0x1d, 0xe6, 0xe4, 0x9e, 0xbe, 0xd4, 0xc4, 0x79, 0x8f, 0xdf, 0x86, 0x82, + 0x06, 0xab, 0x1e, 0xfe, 0xad, 0xe9, 0x33, 0x20, 0xaf, 0x6d, 0xb3, 0xc9, 0xaf, 0xeb, 0xa9, 0x85, + 0x87, 0xd8, 0xea, 0x0e, 0xe4, 0x78, 0x7a, 0xd0, 0x15, 0x98, 0xa5, 0xcc, 0x21, 0x4c, 0x2c, 0xb6, + 0x60, 0xc9, 0x09, 0x2a, 0xc3, 0x0c, 0x8e, 0x64, 0xef, 0x5c, 0xb0, 0xf8, 0x90, 0x7b, 0x25, 0x2b, + 0x4f, 0xdc, 0xaf, 0xb3, 0xda, 0x2b, 0x29, 0xab, 0xed, 0xa8, 0xe3, 0x55, 0x86, 0x85, 0x83, 0xcd, + 0x6e, 0xd7, 0x3e, 0x6c, 0x5b, 0xfb, 0x0f, 0xee, 0xb5, 0x65, 0x2b, 0xdb, 0xed, 0x58, 0xd6, 0xbe, + 0x55, 0x36, 0xf8, 0x69, 0x6b, 0x3c, 0xb4, 0xdb, 0x9b, 0xdd, 0x76, 0x39, 0x8b, 0x16, 0xa0, 0xd0, + 0x78, 0x68, 0x5b, 0x9b, 0x7b, 0xf7, 0x5a, 0xe5, 0x19, 0xa5, 0xda, 0x6d, 0x1d, 0x6e, 0x96, 0x73, + 0xb5, 0x5f, 0x19, 0x50, 0x14, 0xde, 0x77, 0xa2, 0x7e, 0x3c, 0x11, 0xb1, 0xf1, 0xe5, 0x23, 0x46, + 0xa6, 0x3a, 0xc9, 0xb2, 0xa3, 0xbd, 0xaa, 0x73, 0x08, 0xbb, 0xda, 0x2f, 0xa0, 0x74, 0x40, 0x62, + 0x2f, 0x75, 0x31, 0x69, 0x63, 0xc7, 0xc3, 0x04, 0xdd, 0x82, 0x7c, 0x3f, 0x88, 0x8f, 0xf9, 0xad, + 0x20, 0xb2, 0xd5, 0xa8, 0xf0, 0x25, 0xfe, 0xf9, 0x7c, 0x75, 0x6e, 0x3b, 0x88, 0x8f, 0x3b, 0xcd, + 0xb3, 0xe1, 0xc8, 0x9a, 0xe3, 0x86, 0x1d, 0xef, 0x0d, 0xae, 0xa2, 0xda, 0x9f, 0x0d, 0x58, 0xd0, + 0x0e, 0x34, 0x1d, 0xe6, 0xa0, 0x6f, 0x40, 0x91, 0x38, 0xc7, 0x76, 0xef, 0x94, 0x61, 0xaa, 0xb6, + 0xab, 0x40, 0x9c, 0xe3, 0x06, 0x9f, 0x23, 0x0b, 0x0a, 0x21, 0x66, 0x0e, 0x7f, 0x2f, 0xaa, 0xee, + 0x72, 0xf3, 0xe2, 0xaa, 0xb5, 0x70, 0x18, 0x33, 0xac, 0xc9, 0x77, 0x15, 0x4e, 0x67, 0x4c, 0xf3, + 0xa0, 0xef, 0x42, 0x29, 0x4a, 0x43, 0x1b, 0x87, 0x09, 0x3b, 0xb5, 0x49, 0x7c, 0x4c, 0x27, 0xf6, + 0x7e, 0x21, 0x4a, 0xc3, 0x16, 0x57, 0x59, 0xf1, 0x31, 0xad, 0x3d, 0x33, 0x60, 0x69, 0x44, 0x48, + 0xa9, 0x33, 0xc0, 0xe8, 0x2e, 0xcc, 0x1d, 0x89, 0xcc, 0x09, 0x6f, 0xe7, 0xd7, 0x6f, 0x5c, 0xec, + 0xd1, 0x64, 0xa6, 0x2d, 0x85, 0x43, 0x9b, 0x30, 0xc7, 0x4e, 0x13, 0x59, 0xeb, 0x3c, 0xa6, 0x77, + 0x2e, 0x66, 0x18, 0x16, 0x8c, 0x2e, 0x4d, 0x09, 0x44, 0x77, 0x21, 0x27, 0x92, 0x22, 0x9f, 0x85, + 0xdf, 0x7e, 0xbd, 0x0b, 0xcd, 0x51, 0x2a, 0x04, 0xb2, 0xf6, 0xd7, 0x12, 0x5c, 0x3b, 0x3f, 0x63, + 0xe8, 0x27, 0x00, 0xb2, 0xc3, 0xfa, 0x51, 0x3f, 0x56, 0x51, 0xfe, 0xe0, 0xb2, 0x79, 0x97, 0x3d, + 0x83, 0xbb, 0x4e, 0xdb, 0x19, 0xab, 0x48, 0xf4, 0x0c, 0xdd, 0x81, 0x59, 0xf9, 0xa4, 0x95, 0x25, + 0xbb, 0x7a, 0x31, 0xb1, 0x78, 0xc6, 0xb6, 0x33, 0x96, 0xb4, 0x47, 0x9f, 0x02, 0xf0, 0x37, 0x38, + 0xb6, 0xc7, 0x22, 0xdf, 0xb8, 0xb4, 0x5b, 0x87, 0x9c, 0x82, 0x67, 0x83, 0x7b, 0xc5, 0xf4, 0x04, + 0x7d, 0x0c, 0x57, 0x02, 0xec, 0xf4, 0x6d, 0x76, 0x12, 0xd9, 0x7d, 0x3f, 0x72, 0x02, 0x9b, 0x32, + 0x87, 0x61, 0x75, 0xb1, 0x8c, 0x9f, 0x4d, 0xfd, 0xec, 0xd9, 0xc1, 0x4e, 0xff, 0xf0, 0x24, 0xda, + 0xe6, 0xc6, 0x5d, 0x6e, 0xdb, 0xce, 0x58, 0xcb, 0xc1, 0xb4, 0x10, 0x59, 0x90, 0x27, 0xf1, 0xb1, + 0x1d, 0xa5, 0xa1, 0xb8, 0x36, 0xe6, 0xd7, 0xef, 0x5c, 0x3e, 0x93, 0xf1, 0xf1, 0x5e, 0x1a, 0xb6, + 0x33, 0xd6, 0x1c, 0x11, 0x23, 0x14, 0x42, 0x99, 0x3a, 0x61, 0x12, 0x60, 0x62, 0x27, 0x24, 0x1e, + 0xf0, 0x07, 0x7d, 0x25, 0x2f, 0xc8, 0xef, 0x5e, 0x9a, 0xbc, 0x2b, 0x89, 0x0e, 0x14, 0x4f, 0x3b, + 0x63, 0x2d, 0xd1, 0x49, 0x11, 0x3a, 0x84, 0x7c, 0x88, 0x19, 0xf1, 0x5d, 0x5a, 0x29, 0x88, 0x55, + 0xde, 0xbf, 0xf4, 0x2a, 0xbb, 0x12, 0xdf, 0xce, 0x58, 0x9a, 0x0a, 0xfd, 0xd2, 0x80, 0xb7, 0x7a, + 0x69, 0xf0, 0x88, 0x87, 0xe0, 0x62, 0x4a, 0xe3, 0xb1, 0x60, 0x8a, 0x62, 0x99, 0xed, 0x4b, 0x2f, + 0xd3, 0x48, 0x83, 0x47, 0x07, 0x9a, 0x6e, 0x2c, 0xa4, 0xab, 0xbd, 0xf3, 0x14, 0xd5, 0x7d, 0x80, + 0x51, 0x95, 0xa2, 0xcd, 0xa9, 0xb2, 0xe7, 0x47, 0xf3, 0xfa, 0x39, 0x1b, 0x3f, 0x84, 0xe8, 0x7b, + 0x6f, 0x58, 0xdb, 0xd5, 0x9f, 0x41, 0x71, 0x58, 0x5f, 0xa8, 0x0b, 0x4b, 0x6e, 0x1c, 0x04, 0xd8, + 0x65, 0xea, 0xc7, 0xa2, 0xbe, 0xfd, 0xc7, 0xab, 0x89, 0xff, 0x44, 0x34, 0xd5, 0x4f, 0x4b, 0xd3, + 0x52, 0x3f, 0x2d, 0xc7, 0xae, 0xf8, 0xd2, 0x90, 0x82, 0x0b, 0x69, 0xf5, 0x18, 0xe6, 0x64, 0x39, + 0xa0, 0xf7, 0xa0, 0x48, 0x71, 0xe4, 0x61, 0xa2, 0x3b, 0x77, 0xb1, 0x51, 0x1e, 0x36, 0x61, 0xa1, + 0x10, 0x8d, 0x57, 0x8e, 0x3c, 0xfe, 0x33, 0x4a, 0xd7, 0x61, 0x76, 0xfc, 0xae, 0x53, 0x25, 0xb5, + 0x0a, 0x85, 0xc0, 0xa1, 0xcc, 0x0e, 0xe9, 0x40, 0x1c, 0x2d, 0xfd, 0xbc, 0xc9, 0x73, 0xe9, 0x2e, + 0x1d, 0x54, 0x29, 0x2c, 0x4d, 0x95, 0x0a, 0xfa, 0x1e, 0x94, 0x78, 0xff, 0xd4, 0x1b, 0x88, 0xa5, + 0x1b, 0x39, 0x85, 0x5c, 0xe4, 0xba, 0x03, 0xad, 0x42, 0xb7, 0x01, 0x1d, 0xf9, 0x94, 0xc5, 0x03, + 0xe2, 0x84, 0xb6, 0x7a, 0x33, 0xc9, 0xcb, 0x43, 0x2f, 0xb5, 0x3c, 0xd4, 0x37, 0x95, 0xba, 0xfa, + 0x97, 0x1c, 0x5c, 0x3d, 0x77, 0x4f, 0xd1, 0x36, 0x4f, 0x2e, 0x77, 0x67, 0x3a, 0xb9, 0x6f, 0x9d, + 0xb3, 0x63, 0x93, 0xf9, 0x54, 0x28, 0x91, 0x4f, 0xf4, 0x5b, 0x03, 0xd0, 0x88, 0xa8, 0x4f, 0x1c, + 0xfd, 0x43, 0x92, 0x73, 0xfd, 0xf4, 0xab, 0x29, 0x40, 0x73, 0x4b, 0x2f, 0xb0, 0xad, 0xf8, 0x5b, + 0x11, 0x23, 0xa7, 0xd6, 0xb2, 0x3b, 0x2d, 0x47, 0x0c, 0x80, 0x60, 0x9a, 0x86, 0xd8, 0x4e, 0x62, + 0xfd, 0x50, 0x7b, 0xf0, 0x15, 0x79, 0x61, 0x09, 0xe2, 0x83, 0x98, 0xca, 0xd5, 0x8b, 0x44, 0xcf, + 0x51, 0x0b, 0xca, 0xfa, 0xe8, 0xd9, 0x1e, 0x66, 0x8e, 0x1f, 0x50, 0xd5, 0xf8, 0xae, 0x98, 0xf2, + 0x4b, 0x85, 0xa9, 0xbf, 0x54, 0x98, 0x9b, 0xd1, 0xa9, 0x7e, 0x2b, 0x6b, 0x4c, 0x53, 0x42, 0xaa, + 0x4d, 0xb8, 0x76, 0x7e, 0xa4, 0xfc, 0xe5, 0xf5, 0x08, 0x9f, 0x8a, 0xf2, 0x98, 0xb5, 0xf8, 0x90, + 0xbf, 0xd0, 0x9e, 0x38, 0x41, 0x2a, 0x1f, 0x2e, 0x59, 0x4b, 0x4e, 0x36, 0xb2, 0xef, 0x1b, 0xd5, + 0x1f, 0x42, 0x69, 0xd2, 0xd3, 0xd7, 0xa1, 0x67, 0xc6, 0xd1, 0x1f, 0x41, 0x5e, 0xf5, 0x1a, 0xf4, + 0x0e, 0x80, 0x78, 0x55, 0xd8, 0x04, 0x3b, 0xb2, 0x34, 0x67, 0xf4, 0x89, 0x15, 0x72, 0x0b, 0x3b, + 0x1e, 0x7a, 0x1b, 0x8a, 0xa2, 0x86, 0x85, 0x4d, 0x76, 0xcc, 0xa6, 0xc0, 0xc5, 0xdc, 0xa4, 0x91, + 0x57, 0x8b, 0xdd, 0xcf, 0x15, 0xe6, 0xca, 0xf9, 0xda, 0xef, 0x0d, 0xa8, 0x34, 0x7d, 0xca, 0xba, + 0x1f, 0xed, 0xfc, 0x58, 0x7e, 0x29, 0xb9, 0x17, 0x53, 0xea, 0x27, 0xe2, 0x72, 0xbb, 0x39, 0xf9, + 0x4d, 0x65, 0xb1, 0x71, 0x8d, 0x93, 0x7d, 0xf1, 0x7c, 0xb5, 0x34, 0x09, 0x19, 0x7d, 0x65, 0x69, + 0xc3, 0x95, 0xd0, 0x8f, 0x6c, 0xc7, 0x75, 0x71, 0xc2, 0x4b, 0x50, 0xc3, 0xb3, 0xaf, 0x84, 0xa3, + 0xd0, 0x8f, 0x36, 0x15, 0x44, 0xc9, 0x6a, 0x77, 0xe0, 0x6b, 0xca, 0xaa, 0x49, 0x1c, 0x3f, 0xf2, + 0xa3, 0x81, 0x70, 0x69, 0x0d, 0x0a, 0x9e, 0x9a, 0x0b, 0x9f, 0xf4, 0x71, 0x1b, 0x4a, 0x1b, 0xef, + 0x3d, 0xfd, 0xf7, 0x4a, 0xe6, 0xe9, 0xd9, 0x8a, 0xf1, 0xf9, 0xd9, 0x8a, 0xf1, 0xec, 0x6c, 0xc5, + 0xf8, 0xd7, 0xd9, 0x8a, 0xf1, 0xbb, 0x17, 0x2b, 0x99, 0xcf, 0x5f, 0xac, 0x64, 0x9e, 0xbd, 0x58, + 0xc9, 0x7c, 0x32, 0x3f, 0xf6, 0x59, 0xf0, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x12, 0xb3, 0x85, + 0x34, 0x28, 0x14, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index 3f087b1d858d..2c9ecf0a0bf8 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -146,6 +146,9 @@ message OutputRouterSpec { // Each row is sent to one stream, chosen according to preset boundaries // for the values of certain columns of the row. BY_RANGE = 3; + // Each metadata is sent to one stream, specified within the metadata + // itself. + BY_META = 4; } optional Type type = 1 [(gogoproto.nullable) = false]; repeated StreamEndpointSpec streams = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 96ca1e704ea6..d1d189b4c072 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -380,6 +380,8 @@ func (r *OutputRouterSpec) summary() (string, []string) { return "by hash", []string{colListStr(r.HashColumns)} case OutputRouterSpec_BY_RANGE: return "by range", []string{} + case OutputRouterSpec_BY_META: + return "by meta", []string{} default: return "unknown", []string{} } diff --git a/pkg/sql/rowflow/routers.go b/pkg/sql/rowflow/routers.go index 6eeff161f8ee..a877ff340243 100644 --- a/pkg/sql/rowflow/routers.go +++ b/pkg/sql/rowflow/routers.go @@ -69,6 +69,9 @@ func makeRouter( case execinfrapb.OutputRouterSpec_BY_RANGE: return makeRangeRouter(rb, spec.RangeRouterSpec) + case execinfrapb.OutputRouterSpec_BY_META: + return makeMetaRouter(rb, spec.RangeRouterSpec) + default: return nil, errors.Errorf("router type %s not supported", spec.Type) } @@ -473,9 +476,15 @@ type rangeRouter struct { defaultDest *int } +// TODO(pbardea): Document +type metaRouter struct { + routerBase +} + var _ execinfra.RowReceiver = &mirrorRouter{} var _ execinfra.RowReceiver = &hashRouter{} var _ execinfra.RowReceiver = &rangeRouter{} +var _ execinfra.RowReceiver = &metaRouter{} func makeMirrorRouter(rb routerBase) (router, error) { if len(rb.outputs) < 2 { @@ -697,6 +706,82 @@ func (rr *rangeRouter) spanForData(data []byte) int { return int(rr.spans[i].Stream) } +func makeMetaRouter( + rb routerBase, spec execinfrapb.OutputRouterSpec_RangeRouterSpec, +) (*metaRouter, error) { + // TODO: It may be a good idea to keep the default destination here. + //var defaultDest *int + //if spec.DefaultDest != nil { + // i := int(*spec.DefaultDest) + // defaultDest = &i + //} + + return &metaRouter{ + routerBase: rb, + //defaultDest: defaultDest, + }, nil +} + +// sendMetadataToStream attempts to send the given metadata to the output stream +// with the specified index. The boolean returns true iff the metadata was sent. +// If the requested streamIdx is out of range, an error will be returned. +func (mr *metaRouter) sendMetadataToStream( + meta *execinfrapb.ProducerMetadata, streamIdx int, +) (bool, error) { + if streamIdx < 0 || streamIdx >= len(mr.outputs) { + return false, errors.Newf( + "cannot send metadata to stream %d, only %d outputs", streamIdx, len(mr.outputs)) + } + ro := &mr.outputs[streamIdx] + ro.mu.Lock() + defer ro.mu.Unlock() + if ro.mu.streamStatus != execinfra.ConsumerClosed { + ro.addMetadataLocked(meta) + ro.mu.cond.Signal() + return true, nil + } + return false, nil +} + +func (mr *metaRouter) Push( + _ sqlbase.EncDatumRow, meta *execinfrapb.ProducerMetadata, +) execinfra.ConsumerStatus { + // TODO(pbardea): Verify that the row is empty. + if meta == nil { + log.Fatalf(context.TODO(), "sending empty metadata over a meta-data only stream") + } + + // TODO(pbardea): Should this semaphore usage be conditional on + // mr.shouldUseSemaphore()? We seem to always use the semaphore for metadata. + mr.semaphore <- struct{}{} + defer func() { <-mr.semaphore }() + + // TODO(pbardea): I can see an accidental ommission of this lead to everything + // being sent to the first stream. + targetStream := meta.StreamIdx + success, err := mr.sendMetadataToStream(meta, targetStream) + if err != nil { + log.Fatalf(context.TODO(), "%+v", err) + } + if success { + return mr.aggStatus() + } + + // Fallback to the first open stream. + for i := range mr.outputs { + if success, err := mr.sendMetadataToStream(meta, i); err != nil { + log.Fatalf(context.TODO(), "%+v", err) + } else if success { + return mr.aggStatus() + } + } + + // If we got here it means that we couldn't even forward metadata anywhere; + // all streams are closed. + atomic.StoreUint32(&mr.aggregatedStatus, uint32(execinfra.ConsumerClosed)) + return mr.aggStatus() +} + const routerOutputTagPrefix = "routeroutput." // Stats implements the SpanStats interface. diff --git a/pkg/sql/rowflow/routers_test.go b/pkg/sql/rowflow/routers_test.go index 16e3b82ddfed..9f7473015132 100644 --- a/pkg/sql/rowflow/routers_test.go +++ b/pkg/sql/rowflow/routers_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/opentracing/opentracing-go" + "github.com/stretchr/testify/require" ) // setupRouter creates and starts a router. Returns the router and a WaitGroup @@ -904,6 +905,147 @@ func TestRangeRouterInit(t *testing.T) { } } +// TestMetaRouter tests the metadata router. It behaves sufficiently differently +// from the other routers (namely, it doesn't accept rows) that it warrants its +// own test. +func TestMetaRouter(t *testing.T) { + defer leaktest.AfterTest(t) + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + evalCtx := tree.NewTestingEvalContext(st) + defer evalCtx.Stop(context.Background()) + diskMonitor := execinfra.NewTestDiskMonitor(ctx, st) + defer diskMonitor.Stop(ctx) + + const numMetas = 10 + const numBuckets = 4 + + spec := execinfrapb.OutputRouterSpec{Type: execinfrapb.OutputRouterSpec_BY_META} + + t.Run("routing", func(t *testing.T) { + bufs := make([]*distsqlutils.RowBuffer, numBuckets) + recvs := make([]execinfra.RowReceiver, numBuckets) + spec.Streams = make([]execinfrapb.StreamEndpointSpec, numBuckets) + + for i := 0; i < numBuckets; i++ { + bufs[i] = &distsqlutils.RowBuffer{} + recvs[i] = bufs[i] + spec.Streams[i] = execinfrapb.StreamEndpointSpec{StreamID: execinfrapb.StreamID(i)} + } + + r, wg := setupRouter(t, st, evalCtx, diskMonitor, spec, []*types.T{}, recvs) + + for i := 0; i < numMetas; i++ { + meta := &execinfrapb.ProducerMetadata{StreamIdx: i % numBuckets} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + } + r.ProducerDone() + wg.Wait() + + metaStreams := make([][]*execinfrapb.ProducerMetadata, len(bufs)) + for i, b := range bufs { + if !b.ProducerClosed() { + t.Fatalf("bucket not closed: %d", i) + } + metaStreams[i] = b.GetMetaNoRows(t) + } + + metaCount := 0 + for bIdx := range metaStreams { + for _, meta := range metaStreams[bIdx] { + metaCount++ + // Ensure that all of the metas were put in the right bucket. + require.Equal(t, bIdx, meta.StreamIdx) + } + } + require.Equal(t, numMetas, metaCount) + }) + + t.Run("consumer status", func(t *testing.T) { + const numStreams = 2 + bufs := make([]*distsqlutils.RowBuffer, numStreams) + recvs := make([]execinfra.RowReceiver, numStreams) + spec.Streams = make([]execinfrapb.StreamEndpointSpec, numStreams) + + for i := 0; i < numStreams; i++ { + bufs[i] = &distsqlutils.RowBuffer{} + recvs[i] = bufs[i] + spec.Streams[i] = execinfrapb.StreamEndpointSpec{StreamID: execinfrapb.StreamID(i)} + } + + r, wg := setupRouter(t, st, evalCtx, diskMonitor, spec, []*types.T{}, recvs) + + // Push a meta to stream 0. Expect NeedMoreRows. + meta := &execinfrapb.ProducerMetadata{StreamIdx: 0} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + + // Start draining stream 0. Keep expecting NeedMoreRows, regardless on + // which stream we send. + bufs[0].ConsumerDone() + meta = &execinfrapb.ProducerMetadata{StreamIdx: 0} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + meta = &execinfrapb.ProducerMetadata{StreamIdx: 1} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + + // Close stream 0. Continue to expect NeedMoreRows. + bufs[0].ConsumerClosed() + meta = &execinfrapb.ProducerMetadata{StreamIdx: 0} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + meta = &execinfrapb.ProducerMetadata{StreamIdx: 1} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + + // Start draining stream 1. Now that all streams are draining, expect + // DrainRequested. + bufs[1].ConsumerDone() + testutils.SucceedsSoon(t, func() error { + meta = &execinfrapb.ProducerMetadata{StreamIdx: 0} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + return nil + }) + + // Close stream 1. Everything's closed now, and the routers should detect + // this when trying to send metadata - so we expect everything to be closed. + bufs[1].ConsumerClosed() + testutils.SucceedsSoon(t, func() error { + meta = &execinfrapb.ProducerMetadata{StreamIdx: 0} + if status := r.Push(nil /* row */, meta); status != execinfra.NeedMoreRows { + t.Fatalf("unexpected status: %d", status) + } + return nil + }) + + r.ProducerDone() + wg.Wait() + }) + t.Run("blocks", func(t *testing.T) { + + }) + t.Run("disk spill", func(t *testing.T) { + + }) + t.Run("forward to close stream", func(t *testing.T) { + + }) + t.Run("streamIdx out of range", func(t *testing.T) { + + }) +} + func BenchmarkRouter(b *testing.B) { numCols := 1 numRows := 1 << 16 @@ -916,7 +1058,7 @@ func BenchmarkRouter(b *testing.B) { diskMonitor := execinfra.NewTestDiskMonitor(ctx, st) defer diskMonitor.Stop(ctx) - input := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, sqlbase.MakeIntRows(numRows, numCols)) + rowInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, sqlbase.MakeIntRows(numRows, numCols)) for _, spec := range []execinfrapb.OutputRouterSpec{ { @@ -933,14 +1075,26 @@ func BenchmarkRouter(b *testing.B) { { Type: execinfrapb.OutputRouterSpec_MIRROR, }, + { + Type: execinfrapb.OutputRouterSpec_BY_META, + }, } { b.Run(spec.Type.String(), func(b *testing.B) { + var input execinfra.RepeatableSource = rowInput for _, nOutputs := range []int{2, 4, 8} { + if spec.Type == execinfrapb.OutputRouterSpec_BY_META { + input = execinfra.NewRepeatableMetaSource(execinfra.MakeMetas(numRows, nOutputs)) + } chans := make([]execinfra.RowChannel, nOutputs) recvs := make([]execinfra.RowReceiver, nOutputs) spec.Streams = make([]execinfrapb.StreamEndpointSpec, nOutputs) b.Run(fmt.Sprintf("outputs=%d", nOutputs), func(b *testing.B) { - b.SetBytes(int64(nOutputs * numCols * numRows * 8)) + if spec.Type == execinfrapb.OutputRouterSpec_BY_META { + // TODO(pbardea): This seems... wrong? + b.SetBytes(int64(numRows)) + } else { + b.SetBytes(int64(nOutputs * numCols * numRows * 8)) + } for i := 0; i < b.N; i++ { input.Reset() for i := 0; i < nOutputs; i++ { diff --git a/pkg/testutils/distsqlutils/row_buffer.go b/pkg/testutils/distsqlutils/row_buffer.go index 8d59dfb55fa3..86c797493517 100644 --- a/pkg/testutils/distsqlutils/row_buffer.go +++ b/pkg/testutils/distsqlutils/row_buffer.go @@ -217,6 +217,22 @@ func (rb *RowBuffer) NextNoMeta(tb testing.TB) sqlbase.EncDatumRow { return row } +// GetMetaNoRows returns the metadata in the buffer; it fails the test if it +// encounters any rows. +func (rb *RowBuffer) GetMetaNoRows(t *testing.T) []*execinfrapb.ProducerMetadata { + var res []*execinfrapb.ProducerMetadata + for { + // TODO(pbardea): Fix this - we should be checking that we don't get any rows. + _, meta := rb.Next() + if meta == nil { + break + //t.Fatalf("unexpected empty meta with row: %+v, res %+v", row, res) + } + res = append(res, meta) + } + return res +} + // GetRowsNoMeta returns the rows in the buffer; it fails the test if it // encounters any metadata. func (rb *RowBuffer) GetRowsNoMeta(t *testing.T) sqlbase.EncDatumRows {