From 7261e5d5dfd1e5695d4d47c81458fc7e1788afcd Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 27 Jan 2020 19:40:56 -0500 Subject: [PATCH 1/4] sql: propagate row-level locking modes through execbuilder to row.Fetcher Relates to #40205. This commit is a follow-up to #44015. It propagates the row-level locking modes from the optbuilder all the way down to the row.Fetcher. This requires passing the locking information through the execbuilder and then through DistSQL. The commit leaves off at the point where `roachpb.ScanRequest`s are constructed. I couldn't find a good way to test this. It's possible I missed something and would love any tips, but I think we may need to wait until the plumbing into the KV API is complete before performing end-to-end tests. The next step will be to hook the locking modes into the KV API. Release note: None --- pkg/ccl/changefeedccl/rowfetcher_cache.go | 6 +- pkg/sql/backfill/backfill.go | 14 +- pkg/sql/colexec/cfetcher.go | 13 +- pkg/sql/colexec/colbatch_scan.go | 5 +- pkg/sql/delete_range.go | 12 +- pkg/sql/distsql_physical_planner.go | 26 +- pkg/sql/distsql_plan_join.go | 10 +- pkg/sql/execinfrapb/processors_sql.pb.go | 590 +++++++++++++------ pkg/sql/execinfrapb/processors_sql.proto | 50 ++ pkg/sql/opt/bench/stub_factory.go | 1 + pkg/sql/opt/exec/execbuilder/relational.go | 1 + pkg/sql/opt/exec/factory.go | 3 + pkg/sql/opt_exec_factory.go | 5 + pkg/sql/row/cascader.go | 9 + pkg/sql/row/errors.go | 7 +- pkg/sql/row/fetcher.go | 17 +- pkg/sql/row/fetcher_mvcc_test.go | 6 +- pkg/sql/row/fetcher_test.go | 15 +- pkg/sql/row/fk_existence_base.go | 8 +- pkg/sql/row/kv_batch_fetcher.go | 9 +- pkg/sql/row/kv_fetcher.go | 6 +- pkg/sql/rowexec/index_skip_table_reader.go | 10 +- pkg/sql/rowexec/indexjoiner.go | 1 + pkg/sql/rowexec/interleaved_reader_joiner.go | 13 +- pkg/sql/rowexec/joinreader.go | 2 +- pkg/sql/rowexec/rowfetcher.go | 3 +- pkg/sql/rowexec/scrub_tablereader.go | 2 +- pkg/sql/rowexec/tablereader.go | 2 +- pkg/sql/rowexec/zigzagjoiner.go | 3 + pkg/sql/scan.go | 5 + pkg/sql/sqlbase/locking.go | 51 ++ pkg/sql/sqlbase/locking.pb.go | 232 ++++++++ pkg/sql/sqlbase/locking.proto | 136 +++++ pkg/sql/tablewriter_delete.go | 20 +- 34 files changed, 1069 insertions(+), 224 deletions(-) create mode 100644 pkg/sql/sqlbase/locking.go create mode 100644 pkg/sql/sqlbase/locking.pb.go create mode 100644 pkg/sql/sqlbase/locking.proto diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index ec15fa836d2d..821df5fa3e34 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -98,7 +98,11 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( var rf row.Fetcher if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, &c.a, + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + &c.a, row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(), Desc: tableDesc, diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 848261924f30..df109803e03a 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -121,7 +121,12 @@ func (cb *ColumnBackfiller) Init( ValNeededForCol: valNeededForCol, } return cb.fetcher.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, &cb.alloc, tableArgs, + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + &cb.alloc, + tableArgs, ) } @@ -358,7 +363,12 @@ func (ib *IndexBackfiller) Init(desc *sqlbase.ImmutableTableDescriptor) error { ValNeededForCol: valNeededForCol, } return ib.fetcher.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, &ib.alloc, tableArgs, + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + &ib.alloc, + tableArgs, ) } diff --git a/pkg/sql/colexec/cfetcher.go b/pkg/sql/colexec/cfetcher.go index 179f1b8ab612..8c27930e362d 100644 --- a/pkg/sql/colexec/cfetcher.go +++ b/pkg/sql/colexec/cfetcher.go @@ -179,6 +179,9 @@ type cFetcher struct { // table has no interleave children. mustDecodeIndexKey bool + // lockStr represents the row-level locking mode to use when fetching rows. + lockStr sqlbase.ScanLockingStrength + // returnRangeInfo, if set, causes the underlying kvBatchFetcher to return // information about the ranges descriptors/leases uses in servicing the // requests. This has some cost, so it's only enabled by DistSQL when this @@ -241,7 +244,12 @@ type cFetcher struct { // non-primary index, tables.ValNeededForCol can only refer to columns in the // index. func (rf *cFetcher) Init( - allocator *Allocator, reverse, returnRangeInfo bool, isCheck bool, tables ...row.FetcherTableArgs, + allocator *Allocator, + reverse bool, + lockStr sqlbase.ScanLockingStrength, + returnRangeInfo bool, + isCheck bool, + tables ...row.FetcherTableArgs, ) error { rf.adapter.allocator = allocator if len(tables) == 0 { @@ -249,6 +257,7 @@ func (rf *cFetcher) Init( } rf.reverse = reverse + rf.lockStr = lockStr rf.returnRangeInfo = returnRangeInfo if len(tables) > 1 { @@ -464,7 +473,7 @@ func (rf *cFetcher) StartScan( } f, err := row.NewKVFetcher( - txn, spans, rf.reverse, limitBatches, firstBatchLimit, rf.returnRangeInfo, + txn, spans, rf.reverse, limitBatches, firstBatchLimit, rf.lockStr, rf.returnRangeInfo, ) if err != nil { return err diff --git a/pkg/sql/colexec/colbatch_scan.go b/pkg/sql/colexec/colbatch_scan.go index b874e40edea7..ace9fba46ace 100644 --- a/pkg/sql/colexec/colbatch_scan.go +++ b/pkg/sql/colexec/colbatch_scan.go @@ -127,7 +127,7 @@ func newColBatchScan( fetcher := cFetcher{} if _, _, err := initCRowFetcher( allocator, &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, spec.Reverse, - neededColumns, spec.IsCheck, spec.Visibility, + neededColumns, spec.IsCheck, spec.Visibility, spec.LockingStrength, ); err != nil { return nil, err } @@ -157,6 +157,7 @@ func initCRowFetcher( valNeededForCol util.FastIntSet, isCheck bool, scanVisibility execinfrapb.ScanVisibility, + lockStr sqlbase.ScanLockingStrength, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -177,7 +178,7 @@ func initCRowFetcher( ValNeededForCol: valNeededForCol, } if err := fetcher.Init( - allocator, reverseScan, true /* returnRangeInfo */, isCheck, tableArgs, + allocator, reverseScan, lockStr, true /* returnRangeInfo */, isCheck, tableArgs, ); err != nil { return nil, false, err } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index e8c750fbb70d..a9d668d3086d 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -168,7 +168,6 @@ func (d *deleteRangeNode) startExec(params runParams) error { if err := params.p.maybeSetSystemConfig(d.desc.GetID()); err != nil { return err } - if d.interleavedFastPath { for i := range d.spans { d.spans[i].EndKey = d.spans[i].EndKey.PrefixEnd() @@ -189,7 +188,16 @@ func (d *deleteRangeNode) startExec(params runParams) error { } } if err := d.fetcher.Init( - false, false, false, ¶ms.p.alloc, allTables...); err != nil { + false, /* reverse */ + // TODO(nvanbenschoten): it might make sense to use a FOR_UPDATE locking + // strength here. Consider hooking this in to the same knob that will + // control whether we perform locking implicitly during DELETEs. + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + ¶ms.p.alloc, + allTables..., + ); err != nil { return err } ctx := params.ctx diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index e2e27b3da9bd..adad6712b545 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -862,10 +862,12 @@ func initTableReaderSpec( ) (*execinfrapb.TableReaderSpec, execinfrapb.PostProcessSpec, error) { s := physicalplan.NewTableReaderSpec() *s = execinfrapb.TableReaderSpec{ - Table: *n.desc.TableDesc(), - Reverse: n.reverse, - IsCheck: n.isCheck, - Visibility: n.colCfg.visibility.toDistSQLScanVisibility(), + Table: *n.desc.TableDesc(), + Reverse: n.reverse, + IsCheck: n.isCheck, + Visibility: n.colCfg.visibility.toDistSQLScanVisibility(), + LockingStrength: n.lockingStrength, + LockingWaitPolicy: n.lockingWaitPolicy, // Retain the capacity of the spans slice. Spans: s.Spans[:0], @@ -1806,9 +1808,11 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( plan.AddProjection(pkCols) joinReaderSpec := execinfrapb.JoinReaderSpec{ - Table: *n.table.desc.TableDesc(), - IndexIdx: 0, - Visibility: n.table.colCfg.visibility.toDistSQLScanVisibility(), + Table: *n.table.desc.TableDesc(), + IndexIdx: 0, + Visibility: n.table.colCfg.visibility.toDistSQLScanVisibility(), + LockingStrength: n.table.lockingStrength, + LockingWaitPolicy: n.table.lockingWaitPolicy, } filter, err := physicalplan.MakeExpression( @@ -1871,9 +1875,11 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( } joinReaderSpec := execinfrapb.JoinReaderSpec{ - Table: *n.table.desc.TableDesc(), - Type: n.joinType, - Visibility: n.table.colCfg.visibility.toDistSQLScanVisibility(), + Table: *n.table.desc.TableDesc(), + Type: n.joinType, + Visibility: n.table.colCfg.visibility.toDistSQLScanVisibility(), + LockingStrength: n.table.lockingStrength, + LockingWaitPolicy: n.table.lockingWaitPolicy, } joinReaderSpec.IndexIdx, err = getIndexIdx(n.table) if err != nil { diff --git a/pkg/sql/distsql_plan_join.go b/pkg/sql/distsql_plan_join.go index a6490c3875e6..84bcc9c57665 100644 --- a/pkg/sql/distsql_plan_join.go +++ b/pkg/sql/distsql_plan_join.go @@ -199,10 +199,12 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( Tables: processorTables, // We previously checked that both scans are in the // same direction (useInterleavedJoin). - Reverse: ancestor.reverse, - LimitHint: totalLimitHint, - OnExpr: onExpr, - Type: joinType, + Reverse: ancestor.reverse, + LimitHint: totalLimitHint, + LockingStrength: ancestor.lockingStrength, + LockingWaitPolicy: ancestor.lockingWaitPolicy, + OnExpr: onExpr, + Type: joinType, } proc := physicalplan.Processor{ diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 8fe7975ecea0..7053b2067ee8 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -64,7 +64,7 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { return nil } func (ScanVisibility) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -166,7 +166,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{10, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{10, 0} } type AggregatorSpec_Type int32 @@ -212,7 +212,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{10, 1} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{10, 1} } type WindowerSpec_WindowFunc int32 @@ -276,7 +276,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 0} } // Mode indicates which mode of framing is used. @@ -320,7 +320,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1, 0} } // BoundType indicates which type of boundary is used. @@ -367,7 +367,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1, 1} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -410,7 +410,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1, 2} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -430,7 +430,7 @@ func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } func (*ValuesCoreSpec) ProtoMessage() {} func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -519,13 +519,23 @@ type TableReaderSpec struct { // older than this value. // MaxTimestampAgeNanos uint64 `protobuf:"varint,9,opt,name=max_timestamp_age_nanos,json=maxTimestampAgeNanos" json:"max_timestamp_age_nanos"` + // Indicates the row-level locking strength to be used by the scan. If set to + // FOR_NONE, no row-level locking should be performed. + LockingStrength sqlbase.ScanLockingStrength `protobuf:"varint,10,opt,name=locking_strength,json=lockingStrength,enum=cockroach.sql.sqlbase.ScanLockingStrength" json:"locking_strength"` + // Indicates the policy to be used by the scan when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,11,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` } func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } func (*TableReaderSpec) ProtoMessage() {} func (*TableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{1} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -567,13 +577,23 @@ type IndexSkipTableReaderSpec struct { // changes. Visibility ScanVisibility `protobuf:"varint,4,opt,name=visibility,enum=cockroach.sql.distsqlrun.ScanVisibility" json:"visibility"` Reverse bool `protobuf:"varint,5,opt,name=reverse" json:"reverse"` + // Indicates the row-level locking strength to be used by the scan. If set to + // FOR_NONE, no row-level locking should be performed. + LockingStrength sqlbase.ScanLockingStrength `protobuf:"varint,6,opt,name=locking_strength,json=lockingStrength,enum=cockroach.sql.sqlbase.ScanLockingStrength" json:"locking_strength"` + // Indicates the policy to be used by the scan when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,7,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` } func (m *IndexSkipTableReaderSpec) Reset() { *m = IndexSkipTableReaderSpec{} } func (m *IndexSkipTableReaderSpec) String() string { return proto.CompactTextString(m) } func (*IndexSkipTableReaderSpec) ProtoMessage() {} func (*IndexSkipTableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{2} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{2} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -655,13 +675,23 @@ type JoinReaderSpec struct { // default PUBLIC state. Causes the index join to return these schema change // columns. Visibility ScanVisibility `protobuf:"varint,7,opt,name=visibility,enum=cockroach.sql.distsqlrun.ScanVisibility" json:"visibility"` + // Indicates the row-level locking strength to be used by the join. If set to + // FOR_NONE, no row-level locking should be performed. + LockingStrength sqlbase.ScanLockingStrength `protobuf:"varint,9,opt,name=locking_strength,json=lockingStrength,enum=cockroach.sql.sqlbase.ScanLockingStrength" json:"locking_strength"` + // Indicates the policy to be used by the join when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,10,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` } func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } func (*JoinReaderSpec) ProtoMessage() {} func (*JoinReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{3} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{3} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -705,7 +735,7 @@ func (m *SorterSpec) Reset() { *m = SorterSpec{} } func (m *SorterSpec) String() string { return proto.CompactTextString(m) } func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{4} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{4} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +779,7 @@ func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } func (*DistinctSpec) ProtoMessage() {} func (*DistinctSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{5} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{5} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -784,7 +814,7 @@ func (m *OrdinalitySpec) Reset() { *m = OrdinalitySpec{} } func (m *OrdinalitySpec) String() string { return proto.CompactTextString(m) } func (*OrdinalitySpec) ProtoMessage() {} func (*OrdinalitySpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{6} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{6} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -842,7 +872,7 @@ func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{7} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{7} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -918,7 +948,7 @@ func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{8} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{8} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1005,7 +1035,7 @@ func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} func (*HashJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{9} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{9} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1051,7 +1081,7 @@ func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} func (*AggregatorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{10} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{10} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1102,7 +1132,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{10, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{10, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1153,6 +1183,17 @@ type InterleavedReaderJoinerSpec struct { // Not used if there is a limit set in the PostProcessSpec of this processor // (that value will be used for sizing batches instead). LimitHint int64 `protobuf:"varint,3,opt,name=limit_hint,json=limitHint" json:"limit_hint"` + // Indicates the row-level locking strength to be used by the scan over the + // tables. If set to FOR_NONE, no row-level locking should be performed. + LockingStrength sqlbase.ScanLockingStrength `protobuf:"varint,6,opt,name=locking_strength,json=lockingStrength,enum=cockroach.sql.sqlbase.ScanLockingStrength" json:"locking_strength"` + // Indicates the policy to be used by the scan over the tables when dealing + // with rows being locked. Always set to BLOCK when locking_stength is + // FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,7,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` // "ON" expression (in addition to the equality constraints captured by the // orderings). Assuming that the left table has N columns and the second // table stream has M columns, in this expression ordinal references @1 to @N @@ -1166,7 +1207,7 @@ func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJo func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{11} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{11} } func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1222,7 +1263,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedRe func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{11, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{11, 0} } func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1262,7 +1303,7 @@ func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{12} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{12} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1304,7 +1345,7 @@ func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} func (*WindowerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1340,7 +1381,7 @@ func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1376,7 +1417,7 @@ func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1417,7 +1458,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1, 0} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1453,7 +1494,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 1, 1} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1503,7 +1544,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_3df83dee2c766e91, []int{13, 2} + return fileDescriptor_processors_sql_3c1f9b327a58ba1d, []int{13, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1665,6 +1706,12 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x48 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.MaxTimestampAgeNanos)) + dAtA[i] = 0x50 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingStrength)) + dAtA[i] = 0x58 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) return i, nil } @@ -1717,6 +1764,12 @@ func (m *IndexSkipTableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x30 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingStrength)) + dAtA[i] = 0x38 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) return i, nil } @@ -1785,6 +1838,12 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x48 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingStrength)) + dAtA[i] = 0x50 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) return i, nil } @@ -2274,6 +2333,12 @@ func (m *InterleavedReaderJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x28 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) + dAtA[i] = 0x30 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingStrength)) + dAtA[i] = 0x38 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) return i, nil } @@ -2663,6 +2728,8 @@ func (m *TableReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.Visibility)) n += 1 + sovProcessorsSql(uint64(m.MaxResults)) n += 1 + sovProcessorsSql(uint64(m.MaxTimestampAgeNanos)) + n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) + n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) return n } @@ -2683,6 +2750,8 @@ func (m *IndexSkipTableReaderSpec) Size() (n int) { } n += 1 + sovProcessorsSql(uint64(m.Visibility)) n += 2 + n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) + n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) return n } @@ -2707,6 +2776,8 @@ func (m *JoinReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.Type)) n += 1 + sovProcessorsSql(uint64(m.Visibility)) n += 2 + n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) + n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) return n } @@ -2907,6 +2978,8 @@ func (m *InterleavedReaderJoinerSpec) Size() (n int) { l = m.OnExpr.Size() n += 1 + l + sovProcessorsSql(uint64(l)) n += 1 + sovProcessorsSql(uint64(m.Type)) + n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) + n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) return n } @@ -3450,6 +3523,44 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { break } } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) + } + m.LockingStrength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) + } + m.LockingWaitPolicy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -3619,6 +3730,44 @@ func (m *IndexSkipTableReaderSpec) Unmarshal(dAtA []byte) error { } } m.Reverse = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) + } + m.LockingStrength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) + } + m.LockingWaitPolicy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -3879,6 +4028,44 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { } } m.LookupColumnsAreKey = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) + } + m.LockingStrength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) + } + m.LockingWaitPolicy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -5641,6 +5828,44 @@ func (m *InterleavedReaderJoinerSpec) Unmarshal(dAtA []byte) error { break } } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) + } + m.LockingStrength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) + } + m.LockingWaitPolicy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -7022,155 +7247,162 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_3df83dee2c766e91) -} - -var fileDescriptor_processors_sql_3df83dee2c766e91 = []byte{ - // 2330 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xcf, 0x73, 0xdb, 0xc6, - 0xf5, 0x17, 0x48, 0x50, 0x22, 0x1f, 0x7f, 0x68, 0xb3, 0x71, 0x62, 0x7e, 0xe9, 0xef, 0xc8, 0x32, - 0xe3, 0xd4, 0x72, 0xe2, 0x48, 0xa9, 0xda, 0x49, 0x9b, 0xa4, 0x87, 0x80, 0x24, 0x48, 0x53, 0xa6, - 0x00, 0x19, 0x24, 0xe5, 0x1f, 0x9d, 0x29, 0x06, 0x22, 0x56, 0x14, 0x22, 0x12, 0xa0, 0xf0, 0xc3, - 0xa2, 0xf2, 0x0f, 0xe4, 0xda, 0x4e, 0x2f, 0x3d, 0x75, 0x72, 0xe9, 0x5f, 0xd0, 0x1e, 0x7a, 0xca, - 0xd9, 0xc7, 0x9c, 0xda, 0xb4, 0x87, 0x4c, 0x63, 0xff, 0x11, 0xbd, 0x75, 0x3a, 0xbb, 0x58, 0x40, - 0xa0, 0xc6, 0xf4, 0x98, 0xb6, 0xa7, 0xbe, 0x71, 0xdf, 0xdb, 0xcf, 0x67, 0xdf, 0xbe, 0xf7, 0xf6, - 0xed, 0x5b, 0x10, 0xae, 0x7b, 0x27, 0xa3, 0x2d, 0x32, 0x25, 0x03, 0xcb, 0x3e, 0x74, 0x8d, 0xc9, - 0xc1, 0xd6, 0xc4, 0x75, 0x06, 0xc4, 0xf3, 0x1c, 0xd7, 0xd3, 0xbd, 0x93, 0xd1, 0xe6, 0xc4, 0x75, - 0x7c, 0x07, 0x97, 0x07, 0xce, 0xe0, 0xd8, 0x75, 0x8c, 0xc1, 0xd1, 0x26, 0x15, 0x9a, 0x96, 0xe7, - 0x7b, 0x27, 0x23, 0x37, 0xb0, 0x2b, 0xff, 0x4f, 0xf1, 0xde, 0xc9, 0xe8, 0xc0, 0xf0, 0xc8, 0x96, - 0xe7, 0xbb, 0xc1, 0xc0, 0x0f, 0x5c, 0x62, 0x86, 0xb8, 0xca, 0x95, 0xa4, 0xf6, 0x4b, 0xc7, 0xb2, - 0x75, 0xff, 0x6c, 0x42, 0xb8, 0xb2, 0x72, 0x71, 0x69, 0xd3, 0xf0, 0x0d, 0xae, 0x7b, 0xff, 0x39, - 0x66, 0x51, 0x32, 0x3e, 0xed, 0xd2, 0xd0, 0x19, 0x3a, 0xec, 0xe7, 0x16, 0xfd, 0x15, 0x4a, 0xab, - 0xbf, 0x17, 0xa0, 0xb4, 0x6f, 0x8c, 0x02, 0xe2, 0xd5, 0x1d, 0x97, 0x74, 0x27, 0x64, 0x80, 0xeb, - 0xb0, 0x32, 0x70, 0x46, 0xc1, 0xd8, 0xf6, 0xca, 0xc2, 0x7a, 0x7a, 0x23, 0xbf, 0xfd, 0xde, 0xe6, - 0xbc, 0x2d, 0x6d, 0x36, 0x0c, 0x3f, 0x18, 0xb7, 0xed, 0x43, 0xa7, 0x26, 0x3e, 0xfe, 0xe1, 0xea, - 0x92, 0x16, 0x21, 0xf1, 0x15, 0xc8, 0xb9, 0xc6, 0xa9, 0x7e, 0x70, 0xe6, 0x13, 0xaf, 0x9c, 0x5a, - 0x4f, 0x6f, 0x14, 0xb4, 0xac, 0x6b, 0x9c, 0xd6, 0xe8, 0x18, 0x5f, 0x85, 0xac, 0x1d, 0x8c, 0x75, - 0xd7, 0x39, 0xf5, 0xca, 0xe9, 0x75, 0x61, 0x43, 0x8c, 0xd0, 0x76, 0x30, 0xd6, 0x9c, 0x53, 0xaf, - 0xfa, 0x8f, 0x34, 0xac, 0xf6, 0x8c, 0x83, 0x11, 0xd1, 0x88, 0x61, 0x12, 0x97, 0x99, 0x55, 0x83, - 0x8c, 0x4f, 0x45, 0x65, 0x61, 0x5d, 0xd8, 0xc8, 0x6f, 0xff, 0xe4, 0x82, 0x51, 0xdc, 0x73, 0x9b, - 0x0c, 0xd6, 0x20, 0xde, 0xc0, 0xb5, 0x26, 0xbe, 0xe3, 0x72, 0xe6, 0x10, 0x8a, 0xaf, 0x41, 0xce, - 0xb2, 0x4d, 0x32, 0xd5, 0x2d, 0x73, 0x5a, 0x4e, 0xad, 0x0b, 0x1b, 0x45, 0xae, 0xcf, 0x32, 0x71, - 0xdb, 0x9c, 0xe2, 0x35, 0x58, 0x71, 0xc9, 0x23, 0xe2, 0x7a, 0x84, 0x99, 0x96, 0x8d, 0x4c, 0xe3, - 0x42, 0x2c, 0x43, 0xc6, 0x9b, 0x18, 0xb6, 0x57, 0x16, 0x99, 0x6f, 0x6e, 0xce, 0xf7, 0xcd, 0xcc, - 0x06, 0x0c, 0x3b, 0xb2, 0x84, 0xa1, 0xf1, 0x7b, 0x00, 0x23, 0x6b, 0x6c, 0xf9, 0xfa, 0x91, 0x65, - 0xfb, 0xe5, 0xcc, 0xba, 0xb0, 0x91, 0xe6, 0x13, 0x72, 0x4c, 0x7e, 0xdb, 0xb2, 0x7d, 0xea, 0x27, - 0xcb, 0xd3, 0x07, 0x47, 0x64, 0x70, 0x5c, 0x5e, 0x4e, 0x1a, 0x63, 0x79, 0x75, 0x2a, 0xc4, 0x0a, - 0xc0, 0x23, 0xcb, 0xb3, 0x0e, 0xac, 0x91, 0xe5, 0x9f, 0x95, 0x57, 0xd6, 0x85, 0x8d, 0xd2, 0xf6, - 0xc6, 0x7c, 0x8b, 0xba, 0x03, 0xc3, 0xde, 0x8f, 0xe7, 0x73, 0xb2, 0x04, 0x03, 0x7e, 0x1f, 0xf2, - 0x63, 0x63, 0xaa, 0xbb, 0xc4, 0x0b, 0x46, 0xbe, 0x57, 0xce, 0x26, 0x62, 0x03, 0x63, 0x63, 0xaa, - 0x85, 0x72, 0xfc, 0x39, 0x5c, 0xa6, 0xd3, 0x7c, 0x6b, 0x4c, 0x3c, 0xdf, 0x18, 0x4f, 0x74, 0x63, - 0x48, 0x74, 0xdb, 0xb0, 0x1d, 0xaf, 0x9c, 0x4b, 0x40, 0x2e, 0x8d, 0x8d, 0x69, 0x2f, 0x9a, 0x23, - 0x0d, 0x89, 0x42, 0x67, 0x54, 0xff, 0x9a, 0x82, 0x72, 0x9b, 0x7a, 0xbb, 0x7b, 0x6c, 0x4d, 0xde, - 0x50, 0x90, 0xe3, 0x20, 0xa6, 0x5f, 0x29, 0x88, 0xb3, 0xee, 0x17, 0x5f, 0xd9, 0xfd, 0x89, 0xdc, - 0xcb, 0x3c, 0x23, 0xf7, 0xaa, 0x7f, 0x4e, 0x43, 0x69, 0xc7, 0xb1, 0xec, 0xff, 0xbd, 0xc3, 0x6e, - 0x42, 0x69, 0xe4, 0x38, 0xc7, 0xc1, 0x44, 0x8f, 0x4a, 0x03, 0xf5, 0x5c, 0xb1, 0x96, 0x42, 0x82, - 0x56, 0x0c, 0x35, 0x75, 0x7e, 0xf2, 0xeb, 0xb0, 0xe2, 0xd8, 0x3a, 0x99, 0x4e, 0x5c, 0xe6, 0x91, - 0xfc, 0xf6, 0xf5, 0xf9, 0x1e, 0x91, 0xa7, 0x13, 0x97, 0x78, 0x9e, 0xe5, 0x44, 0x8e, 0x5d, 0x76, - 0x6c, 0x2a, 0xc3, 0x9f, 0x82, 0x48, 0xab, 0x1f, 0xcb, 0xfa, 0xd2, 0xf6, 0xd5, 0x39, 0xbb, 0xa2, - 0xbe, 0xe8, 0x9d, 0x4d, 0x08, 0x07, 0x33, 0xc8, 0x6b, 0x3f, 0x13, 0x9f, 0xc2, 0xbb, 0xb3, 0x5b, - 0xd7, 0x0d, 0x97, 0xe8, 0xc7, 0xe4, 0x8c, 0x1d, 0x8f, 0x28, 0x46, 0x6f, 0xcf, 0x38, 0x41, 0x72, - 0xc9, 0x1d, 0x72, 0xb6, 0x23, 0x66, 0x33, 0x68, 0x99, 0x96, 0x58, 0xe8, 0x3a, 0xae, 0xcf, 0x23, - 0x76, 0x17, 0x56, 0x9d, 0xc0, 0x9f, 0x04, 0xbe, 0xee, 0xb8, 0x26, 0x71, 0x2d, 0x7b, 0xc8, 0x63, - 0x57, 0x9d, 0x6f, 0xa4, 0xca, 0x67, 0xf2, 0xc5, 0x4a, 0x21, 0x41, 0x24, 0xc5, 0xdb, 0x80, 0x23, - 0x2e, 0x7d, 0x6c, 0xf8, 0x83, 0x23, 0x7d, 0x44, 0xec, 0x99, 0x48, 0xa2, 0x48, 0xbf, 0x4b, 0xd5, - 0x1d, 0x62, 0x57, 0x0f, 0xa0, 0xd0, 0xb0, 0x3c, 0xdf, 0xb2, 0x07, 0x3e, 0x33, 0xeb, 0x06, 0xac, - 0xb2, 0x39, 0xc4, 0xd4, 0x93, 0xd5, 0xbf, 0xa8, 0x95, 0xb8, 0x38, 0x8a, 0xef, 0x4d, 0x40, 0x26, - 0x07, 0xc6, 0x33, 0x53, 0x6c, 0xe6, 0x6a, 0x24, 0xe7, 0x53, 0xab, 0x08, 0x4a, 0xaa, 0x6b, 0x5a, - 0xb6, 0x41, 0x1d, 0x49, 0x57, 0xa9, 0xfe, 0x21, 0x0d, 0xe8, 0xa1, 0x35, 0xfc, 0xca, 0x18, 0xd2, - 0xd8, 0x71, 0x8f, 0x34, 0x60, 0x99, 0x25, 0x62, 0x74, 0xdf, 0x2c, 0x96, 0xc4, 0x1c, 0x8b, 0x9b, - 0x00, 0xe4, 0x64, 0xc6, 0xa2, 0xfc, 0xf6, 0xb5, 0xf9, 0x2e, 0xe5, 0x36, 0x46, 0x45, 0x97, 0x9c, - 0x9c, 0xef, 0xaf, 0x14, 0x9e, 0x06, 0x27, 0x34, 0x7d, 0x26, 0xd5, 0x99, 0x86, 0xef, 0xe9, 0x35, - 0xa5, 0xfa, 0x1d, 0x28, 0x1c, 0x5a, 0x53, 0x62, 0xea, 0x8f, 0xd8, 0x35, 0x5c, 0xce, 0x30, 0xcb, - 0x9f, 0x93, 0xb1, 0xb3, 0xd7, 0xb5, 0x96, 0x67, 0xe8, 0x50, 0xf8, 0x0a, 0xe7, 0xa6, 0xfa, 0xb7, - 0x34, 0xac, 0xee, 0x12, 0x77, 0x48, 0x12, 0x91, 0xd9, 0x85, 0xe2, 0x88, 0x1c, 0xbe, 0x42, 0xa6, - 0x16, 0x28, 0x3c, 0xce, 0x53, 0x15, 0x4a, 0xae, 0x35, 0x3c, 0x4a, 0xf0, 0xa5, 0x16, 0xe4, 0x2b, - 0x32, 0x7c, 0x4c, 0x98, 0x08, 0x40, 0xe6, 0x4d, 0xd4, 0x9a, 0x9b, 0x50, 0xb4, 0x83, 0xd1, 0x48, - 0x27, 0x27, 0x81, 0x11, 0x97, 0x9b, 0xa8, 0x24, 0x14, 0xa8, 0x4a, 0xe6, 0x1a, 0xfc, 0x19, 0x5c, - 0x66, 0xae, 0x3c, 0xcf, 0xd1, 0x39, 0x75, 0x84, 0x1c, 0xfa, 0xf2, 0xc9, 0x4c, 0x1d, 0xc1, 0xbf, - 0x82, 0x72, 0xe8, 0xb7, 0x67, 0x80, 0x73, 0x09, 0xf0, 0x25, 0x36, 0xeb, 0x02, 0xba, 0xfa, 0xbb, - 0x34, 0x94, 0x6e, 0x1b, 0xde, 0x51, 0x22, 0xae, 0x1f, 0xc0, 0xea, 0x05, 0x63, 0xc2, 0xc3, 0xce, - 0xeb, 0x79, 0xd2, 0x04, 0x7c, 0x0b, 0xd0, 0xc5, 0xc5, 0xc3, 0xf3, 0xce, 0x26, 0x97, 0x66, 0x97, - 0x7c, 0xe3, 0x11, 0xf9, 0x10, 0x4a, 0x63, 0x9a, 0xc4, 0xe7, 0x55, 0x2c, 0x19, 0x92, 0x62, 0xa8, - 0x8b, 0x8c, 0x7d, 0x73, 0x31, 0xf9, 0x7a, 0x05, 0x4a, 0xd2, 0x70, 0xe8, 0x92, 0xa1, 0xe1, 0x3b, - 0x61, 0x4c, 0xae, 0x01, 0x0c, 0x5d, 0x27, 0xbc, 0x66, 0x92, 0x1e, 0xce, 0x31, 0x69, 0xdd, 0x19, - 0x79, 0xf8, 0x37, 0x50, 0x30, 0x38, 0xc8, 0x72, 0xe2, 0xee, 0xe5, 0xe7, 0xf3, 0x3d, 0x3c, 0xbb, - 0x44, 0x3c, 0x3c, 0xf7, 0xf8, 0x0c, 0x1f, 0xfe, 0x98, 0xdf, 0x23, 0xc4, 0xd4, 0x13, 0xa6, 0x88, - 0xb1, 0x29, 0x88, 0x6b, 0x5b, 0xb1, 0x45, 0x2d, 0x1e, 0xa9, 0x0c, 0x8b, 0xd4, 0x47, 0x2f, 0x6c, - 0xc9, 0xc5, 0xb8, 0x55, 0xbe, 0x4e, 0x41, 0x3e, 0x61, 0x1e, 0x25, 0x3e, 0x0c, 0xec, 0x01, 0x2b, - 0x38, 0x8b, 0x10, 0x37, 0x03, 0x7b, 0x10, 0x11, 0x53, 0x02, 0xbc, 0x0e, 0xd9, 0xe8, 0x5a, 0x62, - 0xd5, 0x26, 0x8a, 0x4b, 0x2c, 0xc5, 0xd7, 0xa1, 0x74, 0x68, 0x8d, 0x7c, 0xe2, 0xd2, 0xed, 0xb2, - 0x1e, 0x88, 0x16, 0xf3, 0xa2, 0x56, 0x08, 0xa5, 0x75, 0x67, 0x44, 0x3b, 0xa0, 0xcb, 0xec, 0x55, - 0xc4, 0xd4, 0x19, 0x76, 0xdb, 0x2d, 0x0f, 0x42, 0xc5, 0x6d, 0xc8, 0x19, 0xee, 0x30, 0x18, 0x13, - 0xdb, 0xf7, 0xca, 0xcb, 0x2c, 0x22, 0x8b, 0xe4, 0xfc, 0x39, 0x78, 0x47, 0xcc, 0xa6, 0x91, 0x58, - 0xfd, 0x36, 0x05, 0x22, 0xdd, 0x05, 0x46, 0x50, 0x90, 0x94, 0x07, 0xba, 0xa2, 0xf6, 0x74, 0xa5, - 0xdf, 0xe9, 0xa0, 0x25, 0xbc, 0x02, 0x69, 0x69, 0xbf, 0x85, 0x04, 0x5c, 0x80, 0x6c, 0x4d, 0x55, - 0x3b, 0xba, 0xa4, 0x34, 0x50, 0x0a, 0xe7, 0x61, 0x85, 0x8d, 0x54, 0x0d, 0xa5, 0x71, 0x09, 0xa0, - 0xae, 0x2a, 0x75, 0xa9, 0xa7, 0x4b, 0xad, 0x16, 0x12, 0x71, 0x0e, 0x32, 0x75, 0xb5, 0xaf, 0xf4, - 0x50, 0x86, 0xc2, 0x77, 0xa5, 0xfb, 0x68, 0x85, 0xfd, 0x68, 0x2b, 0x28, 0x8b, 0x01, 0x96, 0xbb, - 0xbd, 0x46, 0x43, 0xde, 0x47, 0x39, 0x2a, 0xec, 0xf6, 0x77, 0x11, 0x50, 0xba, 0x6e, 0x7f, 0x57, - 0x6f, 0x2b, 0x3d, 0x94, 0xa7, 0x2b, 0xed, 0x4b, 0x5a, 0x5b, 0x52, 0xea, 0x32, 0x2a, 0x50, 0xd5, - 0x7d, 0x55, 0x63, 0xcc, 0xc5, 0x70, 0xa5, 0xbe, 0xd2, 0xd3, 0x35, 0xf5, 0x5e, 0x17, 0x95, 0x18, - 0xee, 0xae, 0xd6, 0x68, 0x37, 0x9b, 0x68, 0x15, 0x63, 0x28, 0x35, 0xdb, 0x8a, 0xd4, 0xd1, 0x63, - 0x34, 0xa2, 0x1b, 0x0a, 0x65, 0x7c, 0xcd, 0xb7, 0x70, 0x11, 0x72, 0x92, 0xa6, 0x49, 0x0f, 0x18, - 0x23, 0xa6, 0x8b, 0xed, 0x74, 0x55, 0x85, 0x8d, 0xde, 0xa6, 0x4a, 0x3a, 0xaa, 0xb1, 0xe1, 0x25, - 0xba, 0x5c, 0xb7, 0xa7, 0xb5, 0x95, 0x16, 0x1b, 0xbf, 0xc3, 0x76, 0xdd, 0xee, 0x31, 0x17, 0xbc, - 0x4b, 0x37, 0x42, 0x07, 0xaa, 0x86, 0x2e, 0x57, 0x6f, 0x81, 0x48, 0xd3, 0x0b, 0x67, 0x41, 0x94, - 0xfa, 0x3d, 0x15, 0x2d, 0xb1, 0x6d, 0xd6, 0xa5, 0x8e, 0xa4, 0x21, 0x81, 0xd2, 0x28, 0xaa, 0xa2, - 0xf3, 0x71, 0xaa, 0xfa, 0x1f, 0x11, 0xae, 0xb4, 0x6d, 0x9f, 0xb8, 0x23, 0x62, 0x3c, 0x22, 0x66, - 0xd8, 0x5a, 0x27, 0x4a, 0xe5, 0x83, 0x0b, 0xcd, 0xc9, 0xe7, 0xf3, 0x63, 0xfb, 0x1c, 0x9a, 0xb0, - 0x77, 0xb9, 0xd0, 0xb1, 0x24, 0xda, 0xfd, 0xd4, 0xb3, 0x9e, 0x9a, 0xb3, 0x6f, 0xc4, 0xf4, 0xb3, - 0xdf, 0x88, 0xaf, 0xb5, 0xdd, 0xce, 0x2c, 0x5c, 0x70, 0x2b, 0xdf, 0xa6, 0x20, 0xc3, 0x36, 0x87, - 0xbf, 0x00, 0xd1, 0x24, 0xde, 0xe0, 0xa5, 0x5e, 0x22, 0x0c, 0xf9, 0x22, 0x0f, 0x91, 0x3a, 0x88, - 0x13, 0xc7, 0x0b, 0xbd, 0xf1, 0xdc, 0x87, 0xdb, 0x9e, 0xe3, 0xf9, 0x7b, 0xe1, 0x87, 0x10, 0x1a, - 0x80, 0x68, 0x1d, 0x0a, 0xc6, 0x0d, 0xc8, 0xc6, 0x1d, 0x88, 0xb8, 0x60, 0x07, 0x12, 0x23, 0xcf, - 0x1f, 0x91, 0x99, 0x57, 0x79, 0x44, 0x56, 0x9f, 0x08, 0x50, 0xda, 0x73, 0x9d, 0x2f, 0xc9, 0xc0, - 0xef, 0x92, 0xb0, 0x17, 0xff, 0x02, 0x32, 0x34, 0xa0, 0x51, 0xca, 0x2d, 0x12, 0xd1, 0x10, 0x88, - 0x09, 0xbc, 0x35, 0x24, 0x36, 0x71, 0x0d, 0x3f, 0x71, 0x13, 0xb2, 0xcf, 0x30, 0xb5, 0x5f, 0xd2, - 0x79, 0xff, 0xfc, 0xe1, 0xea, 0xc7, 0x43, 0xcb, 0x3f, 0x0a, 0x0e, 0x36, 0x07, 0xce, 0x78, 0x2b, - 0xe6, 0x37, 0x0f, 0xce, 0x7f, 0x6f, 0x4d, 0x8e, 0x87, 0x5b, 0xde, 0xc9, 0x68, 0x8b, 0xc6, 0xd9, - 0xdb, 0xec, 0x69, 0x28, 0xa6, 0x8c, 0x2e, 0xd0, 0x1b, 0x80, 0xec, 0x60, 0xcc, 0xae, 0x09, 0x7d, - 0x42, 0x5c, 0x7d, 0x48, 0xec, 0xb0, 0x5b, 0xd6, 0x8a, 0x76, 0x30, 0xa6, 0x37, 0xc4, 0x1e, 0x71, - 0x5b, 0xc4, 0xae, 0xfe, 0x58, 0x84, 0xc2, 0x3d, 0xcb, 0x36, 0x9d, 0x53, 0x7e, 0xac, 0xd6, 0x21, - 0x3f, 0x31, 0x5c, 0xdf, 0x62, 0x77, 0xd1, 0x19, 0x7f, 0x6a, 0x24, 0x45, 0xb8, 0x0b, 0xb9, 0x53, - 0x86, 0x68, 0xc6, 0xed, 0xfc, 0xd6, 0x7c, 0x47, 0x24, 0xc9, 0xf9, 0xa0, 0x19, 0x97, 0xd8, 0x98, - 0xa7, 0xf2, 0x17, 0x81, 0x17, 0xd7, 0x2e, 0x14, 0xa3, 0xab, 0x8f, 0x34, 0x5f, 0xf6, 0xa2, 0xd1, - 0x66, 0x39, 0xf0, 0x5d, 0x00, 0xbe, 0x14, 0x65, 0x4c, 0x31, 0xc6, 0x9f, 0x2e, 0x66, 0x33, 0x65, - 0x4d, 0x90, 0x7c, 0x26, 0x3e, 0xfe, 0xe6, 0xaa, 0x50, 0xf9, 0x66, 0x05, 0x32, 0x4d, 0xd7, 0x18, - 0x13, 0x7c, 0x07, 0xc4, 0xb1, 0x63, 0x12, 0x6e, 0xee, 0x8b, 0x92, 0x33, 0xec, 0xe6, 0xae, 0x63, - 0xc6, 0x67, 0x97, 0x92, 0xe0, 0xbb, 0xb0, 0x7c, 0xe0, 0x04, 0xb6, 0xe9, 0xf1, 0x3e, 0xfc, 0x67, - 0x0b, 0xd1, 0xd5, 0x18, 0x34, 0xaa, 0x24, 0x21, 0x11, 0x7e, 0x08, 0x39, 0x32, 0x1d, 0x8c, 0x02, - 0x9a, 0x92, 0xec, 0x90, 0x96, 0xb6, 0x3f, 0x59, 0x88, 0x55, 0x8e, 0xd0, 0xf1, 0xcb, 0x2c, 0x12, - 0x54, 0xfe, 0x2d, 0x40, 0x86, 0x2d, 0x4a, 0x57, 0x61, 0xeb, 0xd1, 0x6a, 0xc4, 0x5d, 0xf1, 0xc9, - 0xe2, 0xb6, 0x27, 0x6a, 0xd9, 0x39, 0x1d, 0xad, 0xba, 0x96, 0xed, 0xeb, 0xce, 0xe1, 0xa1, 0x47, - 0xc2, 0x96, 0x21, 0xfa, 0x9e, 0x95, 0xb3, 0x6c, 0x5f, 0x65, 0x62, 0x7c, 0x0d, 0x0a, 0xf4, 0x54, - 0x98, 0xd1, 0x34, 0xba, 0xd3, 0x82, 0x96, 0x67, 0x32, 0x3e, 0x65, 0x07, 0xf2, 0xa1, 0x92, 0x7d, - 0xc7, 0xe5, 0x75, 0x66, 0x81, 0x4f, 0xa9, 0x10, 0xa2, 0xa9, 0x4d, 0x95, 0x3f, 0x0a, 0xb0, 0x1c, - 0xba, 0x1b, 0x2b, 0x90, 0xf1, 0x7c, 0xc3, 0xf5, 0x79, 0x99, 0xdd, 0x5e, 0x7c, 0xdb, 0x71, 0xf9, - 0xa1, 0x34, 0xb8, 0x01, 0x69, 0x62, 0x9b, 0x3c, 0x01, 0x5e, 0x82, 0x4d, 0xa3, 0xf0, 0xea, 0x0d, - 0x10, 0x69, 0x76, 0xd1, 0x6e, 0x43, 0x93, 0x94, 0x96, 0x8c, 0x96, 0xe8, 0xf5, 0xcb, 0x1a, 0x03, - 0x81, 0x5e, 0xbf, 0x2d, 0x4d, 0xed, 0xef, 0x75, 0x51, 0xaa, 0xfa, 0x15, 0xe4, 0x62, 0xdf, 0xe3, - 0xcb, 0xf0, 0x76, 0x5f, 0xa9, 0xa9, 0x7d, 0xa5, 0x21, 0x37, 0xf4, 0x3d, 0x4d, 0xae, 0xcb, 0x8d, - 0xb6, 0xd2, 0x42, 0x4b, 0xb3, 0x8a, 0xa6, 0xda, 0xe9, 0xa8, 0xf7, 0xa8, 0x42, 0xc0, 0x97, 0x00, - 0xa9, 0xcd, 0x66, 0x57, 0xee, 0x25, 0xa6, 0xa7, 0x12, 0xd2, 0xf3, 0xb9, 0x69, 0xbc, 0x0a, 0xf9, - 0x7a, 0x5f, 0xd3, 0xe4, 0xb0, 0x43, 0x41, 0x62, 0xf5, 0xd7, 0x90, 0x8b, 0xb3, 0x8b, 0x36, 0x23, - 0x8a, 0xaa, 0xcb, 0xf7, 0xeb, 0x9d, 0x7e, 0xb7, 0xad, 0x2a, 0xe1, 0xa2, 0x6c, 0xd8, 0x90, 0xf5, - 0x24, 0x4e, 0xc0, 0x6f, 0x41, 0x31, 0x52, 0xb0, 0x7d, 0xa0, 0x14, 0x45, 0x47, 0xa2, 0x5e, 0x5b, - 0xee, 0xa2, 0x74, 0xe5, 0xef, 0x29, 0xc8, 0x46, 0x75, 0x07, 0xcb, 0x89, 0xee, 0x35, 0xbf, 0xfd, - 0xe1, 0x8b, 0x7a, 0xf5, 0x62, 0xef, 0xfa, 0x7a, 0xee, 0xa9, 0x1a, 0x64, 0x0e, 0x69, 0xbc, 0xf8, - 0x83, 0xec, 0xd6, 0x22, 0x31, 0xd6, 0x42, 0x28, 0xde, 0x80, 0x99, 0x6e, 0x98, 0xbd, 0xcc, 0x32, - 0xd1, 0x1b, 0x62, 0xa6, 0x4f, 0xae, 0x40, 0xd6, 0x70, 0x87, 0x5e, 0xdb, 0x9c, 0xd2, 0xa7, 0x17, - 0xad, 0xea, 0xf1, 0x98, 0xb2, 0x84, 0x5f, 0xae, 0x38, 0x4b, 0x36, 0x71, 0xc5, 0xcf, 0x68, 0x76, - 0xc4, 0x6c, 0x0a, 0xa5, 0x79, 0x43, 0xfc, 0x27, 0x01, 0xe0, 0xbc, 0x3a, 0xd2, 0x06, 0x4e, 0x53, - 0xef, 0xe9, 0x4a, 0x7f, 0xb7, 0x26, 0x6b, 0x3c, 0xcf, 0x24, 0xe5, 0x4e, 0xd8, 0xda, 0x35, 0x64, - 0xa5, 0x2b, 0xeb, 0x6c, 0xcc, 0x82, 0xb4, 0x27, 0x6b, 0x75, 0x16, 0x48, 0x2a, 0x49, 0xd3, 0x96, - 0xb2, 0xde, 0xdf, 0x95, 0xf5, 0x46, 0xbb, 0xdb, 0x0b, 0x7b, 0x63, 0xa5, 0xd7, 0xee, 0xc8, 0x61, - 0x6f, 0xdc, 0x91, 0x5a, 0x68, 0x99, 0xd2, 0x75, 0x64, 0xa9, 0x81, 0x56, 0x68, 0xfe, 0x34, 0xdb, - 0x5a, 0xb7, 0xa7, 0xef, 0x4b, 0x9d, 0xbe, 0x8c, 0xb2, 0x94, 0xbf, 0x23, 0xc5, 0xe3, 0x1c, 0x65, - 0x53, 0x7a, 0xb7, 0xf9, 0x10, 0x3e, 0xf8, 0x05, 0x94, 0x66, 0x3f, 0x26, 0xd2, 0xc4, 0xdf, 0xeb, - 0xd7, 0x3a, 0xed, 0x3a, 0x5a, 0xc2, 0xff, 0x07, 0xef, 0x84, 0xbf, 0x69, 0xc7, 0xca, 0x9a, 0x7a, - 0xae, 0x12, 0x6a, 0x1f, 0x3d, 0xfe, 0x71, 0x6d, 0xe9, 0xf1, 0x93, 0x35, 0xe1, 0xbb, 0x27, 0x6b, - 0xc2, 0xf7, 0x4f, 0xd6, 0x84, 0x7f, 0x3d, 0x59, 0x13, 0x7e, 0xfb, 0x74, 0x6d, 0xe9, 0xbb, 0xa7, - 0x6b, 0x4b, 0xdf, 0x3f, 0x5d, 0x5b, 0x7a, 0x98, 0x4f, 0xfc, 0xbd, 0xf3, 0xdf, 0x00, 0x00, 0x00, - 0xff, 0xff, 0x1a, 0x5a, 0x53, 0x4b, 0x87, 0x1a, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_3c1f9b327a58ba1d) +} + +var fileDescriptor_processors_sql_3c1f9b327a58ba1d = []byte{ + // 2434 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x59, 0xcf, 0x73, 0xdb, 0xc6, + 0xf5, 0x17, 0xf8, 0x4b, 0xe4, 0xe3, 0x0f, 0xc1, 0x6b, 0x27, 0x66, 0x94, 0xef, 0xc8, 0x32, 0x93, + 0x7c, 0x2d, 0x3b, 0x8e, 0x94, 0xaa, 0x9d, 0xb4, 0x49, 0x7a, 0x08, 0x48, 0x82, 0x34, 0x65, 0x0a, + 0x90, 0x41, 0x52, 0xfe, 0x91, 0x99, 0x62, 0x20, 0x60, 0x45, 0x21, 0x02, 0x01, 0x0a, 0x3f, 0x2c, + 0x2a, 0xff, 0x40, 0xae, 0xed, 0xf4, 0xd2, 0xe9, 0xa1, 0x93, 0x4b, 0xff, 0x83, 0x9e, 0xd3, 0xab, + 0x8f, 0x39, 0xb5, 0x99, 0x1e, 0x32, 0x8d, 0xfd, 0x2f, 0x74, 0xa6, 0xd3, 0x5b, 0x67, 0x17, 0x0b, + 0x08, 0xd4, 0x88, 0xae, 0x69, 0x2b, 0xf5, 0xf4, 0xc6, 0x7d, 0x6f, 0x3f, 0x9f, 0x7d, 0xfb, 0xde, + 0xdb, 0xb7, 0x6f, 0x41, 0x78, 0xd7, 0x3b, 0xb2, 0x36, 0xf0, 0x04, 0xeb, 0xa6, 0xbd, 0xef, 0x6a, + 0xe3, 0xbd, 0x8d, 0xb1, 0xeb, 0xe8, 0xd8, 0xf3, 0x1c, 0xd7, 0x53, 0xbd, 0x23, 0x6b, 0x7d, 0xec, + 0x3a, 0xbe, 0x83, 0xaa, 0xba, 0xa3, 0x1f, 0xba, 0x8e, 0xa6, 0x1f, 0xac, 0x13, 0xa1, 0x61, 0x7a, + 0xbe, 0x77, 0x64, 0xb9, 0x81, 0xbd, 0xfc, 0x7f, 0x04, 0xef, 0x1d, 0x59, 0x7b, 0x9a, 0x87, 0x37, + 0x3c, 0xdf, 0x0d, 0x74, 0x3f, 0x70, 0xb1, 0x11, 0xe2, 0x96, 0xdf, 0x4e, 0x6a, 0xbf, 0x70, 0x4c, + 0x5b, 0xf5, 0x4f, 0xc6, 0x98, 0x29, 0xdf, 0x4a, 0x2a, 0x2d, 0x47, 0x3f, 0x34, 0xed, 0x21, 0x53, + 0x2d, 0x9f, 0xb5, 0xca, 0xd0, 0x7c, 0x8d, 0xe9, 0xde, 0x7b, 0x8e, 0xc5, 0x84, 0x8a, 0x4d, 0xbb, + 0x32, 0x74, 0x86, 0x0e, 0xfd, 0xb9, 0x41, 0x7e, 0x85, 0xd2, 0xda, 0x6f, 0x39, 0xa8, 0xec, 0x6a, + 0x56, 0x80, 0xbd, 0x86, 0xe3, 0xe2, 0xde, 0x18, 0xeb, 0xa8, 0x01, 0x8b, 0xba, 0x63, 0x05, 0x23, + 0xdb, 0xab, 0x72, 0xab, 0xe9, 0xb5, 0xe2, 0xe6, 0x3b, 0xeb, 0xb3, 0x76, 0xbb, 0xde, 0xd4, 0xfc, + 0x60, 0xd4, 0xb1, 0xf7, 0x9d, 0x7a, 0xe6, 0xc9, 0xf7, 0xd7, 0x16, 0x94, 0x08, 0x89, 0xde, 0x86, + 0x82, 0xab, 0x1d, 0xab, 0x7b, 0x27, 0x3e, 0xf6, 0xaa, 0xa9, 0xd5, 0xf4, 0x5a, 0x49, 0xc9, 0xbb, + 0xda, 0x71, 0x9d, 0x8c, 0xd1, 0x35, 0xc8, 0xdb, 0xc1, 0x48, 0x75, 0x9d, 0x63, 0xaf, 0x9a, 0x5e, + 0xe5, 0xd6, 0x32, 0x11, 0xda, 0x0e, 0x46, 0x8a, 0x73, 0xec, 0xd5, 0x7e, 0x9f, 0x85, 0xa5, 0xbe, + 0xb6, 0x67, 0x61, 0x05, 0x6b, 0x06, 0x76, 0xa9, 0x59, 0x75, 0xc8, 0xfa, 0x44, 0x54, 0xe5, 0x56, + 0xb9, 0xb5, 0xe2, 0xe6, 0xff, 0x9f, 0x31, 0x8a, 0xf9, 0x6d, 0x9d, 0xc2, 0x9a, 0xd8, 0xd3, 0x5d, + 0x73, 0xec, 0x3b, 0x2e, 0x63, 0x0e, 0xa1, 0xe8, 0x3a, 0x14, 0x4c, 0xdb, 0xc0, 0x13, 0xd5, 0x34, + 0x26, 0xd5, 0xd4, 0x2a, 0xb7, 0x56, 0x66, 0xfa, 0x3c, 0x15, 0x77, 0x8c, 0x09, 0x5a, 0x81, 0x45, + 0x17, 0x3f, 0xc6, 0xae, 0x87, 0xa9, 0x69, 0xf9, 0xc8, 0x34, 0x26, 0x44, 0x22, 0x64, 0xbd, 0xb1, + 0x66, 0x7b, 0xd5, 0x0c, 0xf5, 0xcd, 0xcd, 0xd9, 0xbe, 0x99, 0xda, 0x80, 0x66, 0x47, 0x96, 0x50, + 0x34, 0x7a, 0x07, 0xc0, 0x32, 0x47, 0xa6, 0xaf, 0x1e, 0x98, 0xb6, 0x5f, 0xcd, 0xae, 0x72, 0x6b, + 0x69, 0x36, 0xa1, 0x40, 0xe5, 0x77, 0x4c, 0xdb, 0x27, 0x7e, 0x32, 0x3d, 0x55, 0x3f, 0xc0, 0xfa, + 0x61, 0x35, 0x97, 0x34, 0xc6, 0xf4, 0x1a, 0x44, 0x88, 0x24, 0x80, 0xc7, 0xa6, 0x67, 0xee, 0x99, + 0x96, 0xe9, 0x9f, 0x54, 0x17, 0x57, 0xb9, 0xb5, 0xca, 0xe6, 0xda, 0x6c, 0x8b, 0x7a, 0xba, 0x66, + 0xef, 0xc6, 0xf3, 0x19, 0x59, 0x82, 0x01, 0xbd, 0x07, 0xc5, 0x91, 0x36, 0x51, 0x5d, 0xec, 0x05, + 0x96, 0xef, 0x55, 0xf3, 0x89, 0xd8, 0xc0, 0x48, 0x9b, 0x28, 0xa1, 0x1c, 0x7d, 0x0a, 0x57, 0xc9, + 0x34, 0xdf, 0x1c, 0x61, 0xcf, 0xd7, 0x46, 0x63, 0x55, 0x1b, 0x62, 0xd5, 0xd6, 0x6c, 0xc7, 0xab, + 0x16, 0x12, 0x90, 0x2b, 0x23, 0x6d, 0xd2, 0x8f, 0xe6, 0x08, 0x43, 0x2c, 0x91, 0x19, 0xe8, 0x73, + 0xe0, 0x59, 0x6e, 0xab, 0x9e, 0xef, 0x62, 0x7b, 0xe8, 0x1f, 0x54, 0x81, 0x5a, 0x7e, 0x6b, 0x46, + 0x48, 0x89, 0xd9, 0xdd, 0x10, 0xd2, 0x63, 0x08, 0xb6, 0xc2, 0x92, 0x35, 0x2d, 0x46, 0x7b, 0x70, + 0x39, 0x22, 0x3f, 0xd6, 0x4c, 0x5f, 0x1d, 0x3b, 0x96, 0xa9, 0x9f, 0x54, 0x8b, 0x94, 0xff, 0xf6, + 0x7f, 0xe6, 0xbf, 0xaf, 0x99, 0xfe, 0x0e, 0xc5, 0xb0, 0x15, 0x2e, 0x59, 0x67, 0x15, 0xb5, 0x7f, + 0xa5, 0xa1, 0xda, 0x21, 0xe9, 0xd2, 0x3b, 0x34, 0xc7, 0xaf, 0x29, 0x4b, 0xe3, 0x2c, 0x4c, 0xbf, + 0x52, 0x16, 0x4e, 0xe7, 0x4f, 0xe6, 0x95, 0xf3, 0x27, 0x71, 0x78, 0xb2, 0xe7, 0x1d, 0x9e, 0xf3, + 0x62, 0x9f, 0xfb, 0x91, 0x63, 0xbf, 0x78, 0x91, 0xb1, 0xff, 0x47, 0x06, 0x2a, 0x5b, 0x8e, 0x69, + 0xff, 0xf7, 0x23, 0x7e, 0x13, 0x2a, 0x96, 0xe3, 0x1c, 0x06, 0x63, 0x35, 0x2a, 0xce, 0x24, 0xf4, + 0xe5, 0x7a, 0x8a, 0xe7, 0x94, 0x72, 0xa8, 0x69, 0xb0, 0xda, 0xdb, 0x80, 0x45, 0xc7, 0x56, 0xf1, + 0x64, 0xec, 0xd2, 0x90, 0x16, 0x37, 0xdf, 0x9d, 0x1d, 0x52, 0x71, 0x32, 0x76, 0xb1, 0xe7, 0x99, + 0x4e, 0x94, 0x19, 0x39, 0xc7, 0x26, 0x32, 0xf4, 0x31, 0x64, 0xc8, 0xd5, 0xc4, 0xc2, 0x73, 0x6d, + 0xc6, 0xae, 0x88, 0x2f, 0xfa, 0x27, 0x63, 0xcc, 0xc0, 0x14, 0x72, 0xe1, 0x55, 0xe9, 0x63, 0x78, + 0x73, 0x7a, 0xeb, 0xaa, 0xe6, 0x62, 0xf5, 0x10, 0x9f, 0xd0, 0x02, 0x15, 0x25, 0xd9, 0xe5, 0x29, + 0x27, 0x08, 0x2e, 0xbe, 0x8b, 0x4f, 0xce, 0x4d, 0xb8, 0xc2, 0x8f, 0x9c, 0x70, 0x70, 0x81, 0x09, + 0xb7, 0x95, 0xc9, 0x67, 0xf9, 0x1c, 0xb9, 0xa5, 0xa1, 0xe7, 0xb8, 0x3e, 0x4b, 0xb9, 0x7b, 0xb0, + 0xe4, 0x04, 0xfe, 0x38, 0xf0, 0x55, 0xc7, 0x35, 0xb0, 0x6b, 0xda, 0x43, 0x96, 0x7c, 0xb5, 0xd9, + 0x5e, 0x96, 0xd9, 0x4c, 0xb6, 0x54, 0x25, 0x24, 0x88, 0xa4, 0x68, 0x13, 0x50, 0xc4, 0xa5, 0x8e, + 0x34, 0x5f, 0x3f, 0x50, 0x2d, 0x6c, 0x4f, 0xa5, 0x22, 0x1f, 0xe9, 0xb7, 0x89, 0xba, 0x8b, 0xed, + 0xda, 0x1e, 0x94, 0x9a, 0xa6, 0xe7, 0x9b, 0xb6, 0xee, 0x53, 0xb3, 0x6e, 0xc0, 0x12, 0x9d, 0x83, + 0x0d, 0x35, 0xd9, 0x40, 0x94, 0x95, 0x0a, 0x13, 0x47, 0x09, 0x7a, 0x13, 0x78, 0x83, 0x01, 0xe3, + 0x99, 0x29, 0x3a, 0x73, 0x29, 0x92, 0xb3, 0xa9, 0x35, 0x1e, 0x2a, 0xb2, 0x6b, 0x98, 0xb6, 0x46, + 0x32, 0x81, 0xac, 0x52, 0xfb, 0x5d, 0x1a, 0xf8, 0x47, 0xe6, 0xf0, 0x4b, 0x6d, 0x48, 0x92, 0x8f, + 0x79, 0xa4, 0x09, 0x39, 0x7a, 0x92, 0xa2, 0x96, 0x65, 0xbe, 0x53, 0xc8, 0xb0, 0xa8, 0x05, 0x80, + 0x8f, 0xa6, 0x2c, 0x2a, 0x6e, 0x5e, 0x9f, 0xed, 0x52, 0x66, 0x63, 0x74, 0x6f, 0xe3, 0xa3, 0xd3, + 0xfd, 0x55, 0xc2, 0xe3, 0xec, 0x84, 0xa6, 0x4f, 0x9d, 0x55, 0xaa, 0x61, 0x7b, 0xba, 0xa0, 0xb3, + 0x7a, 0x17, 0x4a, 0xfb, 0xe6, 0x04, 0x1b, 0xea, 0x63, 0xda, 0xc9, 0x55, 0xb3, 0xd4, 0xf2, 0xe7, + 0x1c, 0xb9, 0xe9, 0x8e, 0x4f, 0x29, 0x52, 0x74, 0x28, 0x7c, 0x85, 0x83, 0x5f, 0xfb, 0x4b, 0x1a, + 0x96, 0xb6, 0xb1, 0x3b, 0xc4, 0x89, 0xc8, 0x6c, 0x43, 0xd9, 0xc2, 0xfb, 0xaf, 0x90, 0xa9, 0x25, + 0x02, 0x8f, 0xf3, 0x54, 0x86, 0x8a, 0x6b, 0x0e, 0x0f, 0x12, 0x7c, 0xa9, 0x39, 0xf9, 0xca, 0x14, + 0x1f, 0x13, 0x26, 0x02, 0x90, 0x7d, 0x1d, 0xc5, 0xf2, 0x26, 0x94, 0xed, 0xc0, 0xb2, 0x54, 0x7c, + 0x14, 0x68, 0x71, 0xbd, 0x8c, 0x6a, 0x5a, 0x89, 0xa8, 0x44, 0xa6, 0x41, 0x9f, 0xc0, 0x55, 0xea, + 0xca, 0xd3, 0x1c, 0x9d, 0x51, 0x08, 0xf1, 0xbe, 0x2f, 0x1e, 0x4d, 0x17, 0xc2, 0x5f, 0x42, 0x35, + 0xf4, 0xdb, 0x39, 0xe0, 0x42, 0x02, 0x7c, 0x85, 0xce, 0x3a, 0x83, 0xae, 0xfd, 0x26, 0x0d, 0x95, + 0x3b, 0x9a, 0x77, 0x90, 0x88, 0xeb, 0x2d, 0x58, 0x3a, 0x63, 0x4c, 0x78, 0xd8, 0xd9, 0x85, 0x94, + 0x34, 0x01, 0xdd, 0x06, 0xfe, 0xec, 0xe2, 0xe1, 0x79, 0xa7, 0x93, 0x2b, 0xd3, 0x4b, 0xbe, 0xf6, + 0x88, 0xbc, 0x0f, 0x95, 0x11, 0x49, 0xe2, 0xd3, 0x2a, 0x96, 0x0c, 0x49, 0x39, 0xd4, 0x45, 0xc6, + 0xbe, 0xbe, 0x98, 0x7c, 0xb5, 0x08, 0x15, 0x61, 0x38, 0x74, 0xf1, 0x50, 0xf3, 0x9d, 0x30, 0x26, + 0xd7, 0x01, 0x86, 0xae, 0x13, 0xde, 0x93, 0x49, 0x0f, 0x17, 0xa8, 0xb4, 0xe1, 0x58, 0x1e, 0xfa, + 0x15, 0x94, 0x34, 0x06, 0x32, 0x9d, 0xb8, 0x7f, 0xfc, 0xd9, 0x6c, 0x0f, 0x4f, 0x2f, 0x11, 0x0f, + 0x4f, 0x3d, 0x3e, 0xc5, 0x87, 0x3e, 0x64, 0xf7, 0x08, 0x36, 0xd4, 0x84, 0x29, 0x99, 0xd8, 0x14, + 0x9e, 0x69, 0xdb, 0xb1, 0x45, 0x6d, 0x16, 0xa9, 0x2c, 0x8d, 0xd4, 0x07, 0x2f, 0x6c, 0xc9, 0xd9, + 0xb8, 0x2d, 0x7f, 0x95, 0x82, 0x62, 0xc2, 0x3c, 0x42, 0xbc, 0x1f, 0xd8, 0x3a, 0x2d, 0x38, 0xf3, + 0x10, 0xb7, 0x02, 0x5b, 0x8f, 0x88, 0x09, 0x01, 0x5a, 0x85, 0x7c, 0x74, 0x2d, 0xd1, 0x6a, 0x13, + 0xc5, 0x25, 0x96, 0xa2, 0x77, 0xa1, 0xb2, 0x6f, 0x5a, 0x3e, 0x76, 0xc9, 0x76, 0x69, 0x13, 0x47, + 0x8a, 0x79, 0x59, 0x29, 0x85, 0xd2, 0x86, 0x63, 0x91, 0x16, 0xee, 0x2a, 0x7d, 0x58, 0x53, 0x75, + 0x96, 0xde, 0x76, 0x39, 0x3d, 0x54, 0xdc, 0x81, 0x82, 0xe6, 0x0e, 0x83, 0x11, 0xb6, 0x7d, 0xaf, + 0x9a, 0xa3, 0x11, 0x99, 0x27, 0xe7, 0x4f, 0xc1, 0x5b, 0x99, 0x7c, 0x9a, 0xcf, 0xd4, 0xbe, 0x49, + 0x41, 0x86, 0xec, 0x02, 0xf1, 0x50, 0x12, 0xa4, 0x87, 0xaa, 0x24, 0xf7, 0x55, 0x69, 0xd0, 0xed, + 0xf2, 0x0b, 0x68, 0x11, 0xd2, 0xc2, 0x6e, 0x9b, 0xe7, 0x50, 0x09, 0xf2, 0x75, 0x59, 0xee, 0xaa, + 0x82, 0xd4, 0xe4, 0x53, 0xa8, 0x08, 0x8b, 0x74, 0x24, 0x2b, 0x7c, 0x1a, 0x55, 0x00, 0x1a, 0xb2, + 0xd4, 0x10, 0xfa, 0xaa, 0xd0, 0x6e, 0xf3, 0x19, 0x54, 0x80, 0x6c, 0x43, 0x1e, 0x48, 0x7d, 0x3e, + 0x4b, 0xe0, 0xdb, 0xc2, 0x03, 0x7e, 0x91, 0xfe, 0xe8, 0x48, 0x7c, 0x1e, 0x01, 0xe4, 0x7a, 0xfd, + 0x66, 0x53, 0xdc, 0xe5, 0x0b, 0x44, 0xd8, 0x1b, 0x6c, 0xf3, 0x40, 0xe8, 0x7a, 0x83, 0x6d, 0xb5, + 0x23, 0xf5, 0xf9, 0x22, 0x59, 0x69, 0x57, 0x50, 0x3a, 0x82, 0xd4, 0x10, 0xf9, 0x12, 0x51, 0x3d, + 0x90, 0x15, 0xca, 0x5c, 0x0e, 0x57, 0x1a, 0x48, 0x7d, 0x55, 0x91, 0xef, 0xf7, 0xf8, 0x0a, 0xc5, + 0xdd, 0x53, 0x9a, 0x9d, 0x56, 0x8b, 0x5f, 0x42, 0x08, 0x2a, 0xad, 0x8e, 0x24, 0x74, 0xd5, 0x18, + 0xcd, 0x93, 0x0d, 0x85, 0x32, 0xb6, 0xe6, 0x25, 0x54, 0x86, 0x82, 0xa0, 0x28, 0xc2, 0x43, 0xca, + 0x88, 0xc8, 0x62, 0x5b, 0x3d, 0x59, 0xa2, 0xa3, 0xcb, 0x44, 0x49, 0x46, 0x75, 0x3a, 0xbc, 0x42, + 0x96, 0xeb, 0xf5, 0x95, 0x8e, 0xd4, 0xa6, 0xe3, 0x37, 0xe8, 0xae, 0x3b, 0x7d, 0xea, 0x82, 0x37, + 0xc9, 0x46, 0xc8, 0x40, 0x56, 0xf8, 0xab, 0xb5, 0xdb, 0x90, 0x21, 0xe9, 0x85, 0xf2, 0x90, 0x11, + 0x06, 0x7d, 0x99, 0x5f, 0xa0, 0xdb, 0x6c, 0x08, 0x5d, 0x41, 0xe1, 0x39, 0x42, 0x23, 0xc9, 0x92, + 0xca, 0xc6, 0xa9, 0xda, 0x9f, 0x73, 0xf0, 0x76, 0xc7, 0xf6, 0xb1, 0x6b, 0x61, 0xed, 0x31, 0x36, + 0xc2, 0xb7, 0x41, 0xa2, 0x54, 0x3e, 0x3c, 0xd3, 0x9c, 0x7c, 0x3a, 0x3b, 0xb6, 0xcf, 0xa1, 0x09, + 0x7b, 0x97, 0x33, 0x1d, 0x4b, 0xe2, 0xc1, 0x95, 0x3a, 0xef, 0xc1, 0x35, 0xfd, 0x99, 0x21, 0x7d, + 0xfe, 0x67, 0x86, 0x0b, 0x7d, 0x2f, 0x64, 0xe7, 0x2f, 0xb8, 0xff, 0xeb, 0xaf, 0xc2, 0xe5, 0x6f, + 0x52, 0x90, 0xa5, 0xd1, 0x41, 0x9f, 0x41, 0xc6, 0xc0, 0x9e, 0xfe, 0x52, 0x6f, 0x41, 0x8a, 0x7c, + 0x91, 0xa7, 0x60, 0x03, 0x32, 0x63, 0xc7, 0x0b, 0xc3, 0xf9, 0xdc, 0xb7, 0xff, 0x8e, 0xe3, 0xf9, + 0x3b, 0xe1, 0xc7, 0x40, 0x92, 0x41, 0xd1, 0x3a, 0x04, 0x8c, 0x9a, 0x90, 0x8f, 0x5b, 0xa8, 0xcc, + 0x9c, 0x2d, 0x54, 0x8c, 0x3c, 0xfd, 0x0e, 0x91, 0x7d, 0x95, 0xef, 0x10, 0xb5, 0xa7, 0x1c, 0x54, + 0x76, 0x5c, 0xe7, 0x0b, 0xac, 0xfb, 0x3d, 0x1c, 0x3e, 0x26, 0x3e, 0x83, 0x2c, 0xc9, 0xc8, 0xe8, + 0xcc, 0xcc, 0x93, 0x92, 0x21, 0x10, 0x61, 0xb8, 0x34, 0xc4, 0x36, 0x76, 0x35, 0x3f, 0x71, 0x95, + 0xd3, 0x4f, 0x91, 0xf5, 0x5f, 0x90, 0x79, 0x7f, 0xfb, 0xfe, 0xda, 0x87, 0x43, 0xd3, 0x3f, 0x08, + 0xf6, 0xd6, 0x75, 0x67, 0xb4, 0x11, 0xf3, 0x1b, 0x7b, 0xa7, 0xbf, 0x37, 0xc6, 0x87, 0xc3, 0x0d, + 0xef, 0xc8, 0xda, 0x20, 0x89, 0xea, 0xad, 0xf7, 0x15, 0x3e, 0xa6, 0x8c, 0x3a, 0x80, 0x1b, 0xc0, + 0xdb, 0xc1, 0x88, 0xde, 0x73, 0xea, 0x18, 0xbb, 0xea, 0x10, 0xdb, 0x61, 0xbb, 0xaf, 0x94, 0xed, + 0x60, 0x44, 0xae, 0xb8, 0x1d, 0xec, 0xb6, 0xb1, 0x5d, 0xfb, 0xa1, 0x0c, 0xa5, 0xfb, 0xa6, 0x6d, + 0x38, 0xc7, 0xac, 0x2e, 0xac, 0x42, 0x71, 0xac, 0xb9, 0xbe, 0x49, 0x2f, 0xd3, 0x13, 0xf6, 0x56, + 0x4a, 0x8a, 0x50, 0x0f, 0x0a, 0xc7, 0x14, 0xd1, 0x8a, 0xdf, 0x23, 0x1b, 0xb3, 0x1d, 0x91, 0x24, + 0x67, 0x83, 0x56, 0x7c, 0x47, 0xc4, 0x3c, 0xcb, 0x7f, 0xe2, 0xd8, 0xed, 0xd0, 0x83, 0x72, 0x74, + 0x77, 0xe3, 0xd6, 0xcb, 0xde, 0x94, 0xca, 0x34, 0x07, 0xba, 0x07, 0xc0, 0x96, 0x22, 0x8c, 0x29, + 0xca, 0xf8, 0x93, 0xf9, 0x6c, 0x26, 0xac, 0x09, 0x92, 0x4f, 0x32, 0x4f, 0xbe, 0xbe, 0xc6, 0x2d, + 0x7f, 0xbd, 0x08, 0xd9, 0x96, 0xab, 0x8d, 0x30, 0xba, 0x0b, 0x99, 0x91, 0x63, 0x60, 0x66, 0xee, + 0x8b, 0x92, 0x53, 0xec, 0xfa, 0xb6, 0x63, 0xc4, 0xc5, 0x87, 0x90, 0xa0, 0x7b, 0x90, 0xdb, 0x73, + 0x02, 0xdb, 0xf0, 0xd8, 0x43, 0xe2, 0xa7, 0x73, 0xd1, 0xd5, 0x29, 0x34, 0x2a, 0x85, 0x21, 0x11, + 0x7a, 0x04, 0x05, 0x3c, 0xd1, 0xad, 0x80, 0xa4, 0x24, 0x3d, 0xa4, 0x95, 0xcd, 0x8f, 0xe6, 0x62, + 0x15, 0x23, 0x74, 0xfc, 0xb4, 0x8c, 0x04, 0xcb, 0xff, 0xe4, 0x20, 0x4b, 0x17, 0x25, 0xab, 0xd0, + 0xf5, 0x48, 0x39, 0x65, 0xae, 0xf8, 0x68, 0x7e, 0xdb, 0x13, 0xc5, 0xf8, 0x94, 0x8e, 0x5c, 0x1b, + 0xa6, 0xed, 0xab, 0xce, 0xfe, 0xbe, 0x87, 0xc3, 0x9e, 0x27, 0xfa, 0xa6, 0x5b, 0x30, 0x6d, 0x5f, + 0xa6, 0x62, 0x74, 0x1d, 0x4a, 0xe4, 0x54, 0x18, 0xd1, 0x34, 0xb2, 0xd3, 0x92, 0x52, 0xa4, 0x32, + 0x36, 0x65, 0x0b, 0x8a, 0xa1, 0x92, 0xfe, 0xcd, 0xc1, 0xea, 0xcc, 0x1c, 0x7f, 0x27, 0x40, 0x88, + 0x26, 0x36, 0x2d, 0xff, 0x81, 0x83, 0x5c, 0xe8, 0x6e, 0x24, 0x41, 0xd6, 0xf3, 0x35, 0xd7, 0x67, + 0x65, 0x76, 0x73, 0xfe, 0x6d, 0xc7, 0xe5, 0x87, 0xd0, 0xa0, 0x26, 0xa4, 0xb1, 0x6d, 0xb0, 0x04, + 0x78, 0x09, 0x36, 0x85, 0xc0, 0x6b, 0x37, 0x20, 0x43, 0xb2, 0x8b, 0xb4, 0x4b, 0x8a, 0x20, 0xb5, + 0x45, 0x7e, 0x81, 0xf4, 0x0f, 0xb4, 0xb3, 0xe1, 0x48, 0xff, 0xd0, 0x56, 0xe4, 0xc1, 0x4e, 0x8f, + 0x4f, 0xd5, 0xbe, 0x84, 0x42, 0xec, 0x7b, 0x74, 0x15, 0x2e, 0x0f, 0xa4, 0xba, 0x3c, 0x90, 0x9a, + 0x62, 0x53, 0xdd, 0x51, 0xc4, 0x86, 0xd8, 0xec, 0x48, 0x6d, 0x7e, 0x61, 0x5a, 0xd1, 0x92, 0xbb, + 0x5d, 0xf9, 0x3e, 0x51, 0x70, 0xe8, 0x0a, 0xf0, 0x72, 0xab, 0xd5, 0x13, 0xfb, 0x89, 0xe9, 0xa9, + 0x84, 0xf4, 0x74, 0x6e, 0x1a, 0x2d, 0x41, 0xb1, 0x31, 0x50, 0x14, 0x31, 0x6c, 0xb1, 0xf8, 0x4c, + 0xed, 0x73, 0x28, 0xc4, 0xd9, 0x45, 0xba, 0x29, 0x49, 0x56, 0xc5, 0x07, 0x8d, 0xee, 0xa0, 0xd7, + 0x91, 0xa5, 0x70, 0x51, 0x3a, 0x6c, 0x8a, 0x6a, 0x12, 0xc7, 0xa1, 0x4b, 0x50, 0x8e, 0x14, 0x74, + 0x1f, 0x7c, 0x8a, 0xa0, 0x23, 0x51, 0xbf, 0x23, 0xf6, 0xf8, 0xf4, 0xf2, 0x5f, 0x53, 0x90, 0x8f, + 0xea, 0x0e, 0x12, 0x13, 0xed, 0x77, 0x71, 0xf3, 0xfd, 0x17, 0xf5, 0xea, 0xd9, 0xe6, 0xfb, 0x62, + 0xee, 0xa9, 0x3a, 0x64, 0xf7, 0x49, 0xbc, 0xd8, 0x8b, 0xf2, 0xf6, 0x3c, 0x31, 0x56, 0x42, 0x28, + 0x5a, 0x83, 0xa9, 0x76, 0x9e, 0xb6, 0x28, 0xd9, 0xe8, 0x11, 0x34, 0xd5, 0xe8, 0x2f, 0x43, 0x5e, + 0x73, 0x87, 0x5e, 0xc7, 0x98, 0x90, 0xb7, 0x23, 0xa9, 0xea, 0xf1, 0x98, 0xb0, 0x84, 0x9f, 0xde, + 0x18, 0x4b, 0x3e, 0x71, 0xc5, 0x4f, 0x69, 0xb6, 0x32, 0xf9, 0x14, 0x9f, 0x66, 0x1d, 0xfd, 0x1f, + 0x39, 0x80, 0xd3, 0xea, 0x48, 0x3a, 0x50, 0x45, 0xbe, 0xaf, 0x4a, 0x83, 0xed, 0xba, 0xa8, 0xb0, + 0x3c, 0x13, 0xa4, 0xbb, 0x61, 0x6f, 0xda, 0x14, 0xa5, 0x9e, 0xa8, 0xd2, 0x31, 0x0d, 0xd2, 0x8e, + 0xa8, 0x34, 0x68, 0x20, 0x89, 0x24, 0x4d, 0x7a, 0xe2, 0xc6, 0x60, 0x5b, 0x54, 0x9b, 0x9d, 0x5e, + 0x3f, 0x6c, 0xee, 0xa5, 0x7e, 0xa7, 0x2b, 0x86, 0xcd, 0x7d, 0x57, 0x68, 0xf3, 0x39, 0x42, 0xd7, + 0x15, 0x85, 0x26, 0xbf, 0x48, 0xf2, 0xa7, 0xd5, 0x51, 0x7a, 0x7d, 0x75, 0x57, 0xe8, 0x0e, 0x44, + 0x3e, 0x4f, 0xf8, 0xbb, 0x42, 0x3c, 0x2e, 0x10, 0x36, 0xa9, 0x7f, 0x87, 0x0d, 0xe1, 0xd6, 0xcf, + 0xa1, 0x32, 0xfd, 0x39, 0x97, 0x24, 0xfe, 0xce, 0xa0, 0xde, 0xed, 0x34, 0xf8, 0x05, 0xf4, 0x16, + 0xbc, 0x11, 0xfe, 0x26, 0x2d, 0x37, 0x7d, 0x95, 0x30, 0x15, 0x57, 0xff, 0xe0, 0xc9, 0x0f, 0x2b, + 0x0b, 0x4f, 0x9e, 0xae, 0x70, 0xdf, 0x3e, 0x5d, 0xe1, 0xbe, 0x7b, 0xba, 0xc2, 0xfd, 0xfd, 0xe9, + 0x0a, 0xf7, 0xeb, 0x67, 0x2b, 0x0b, 0xdf, 0x3e, 0x5b, 0x59, 0xf8, 0xee, 0xd9, 0xca, 0xc2, 0xa3, + 0x62, 0xe2, 0x2f, 0xce, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0xb9, 0x03, 0x84, 0x7e, 0xa6, 0x1d, + 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 4ab50d13ee14..9fcbbbdb4241 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -21,6 +21,7 @@ option go_package = "execinfrapb"; import "sql/sqlbase/structured.proto"; import "sql/sqlbase/join_type.proto"; +import "sql/sqlbase/locking.proto"; import "sql/execinfrapb/data.proto"; import "sql/execinfrapb/processors_base.proto"; import "gogoproto/gogo.proto"; @@ -116,6 +117,18 @@ message TableReaderSpec { // older than this value. // optional uint64 max_timestamp_age_nanos = 9 [(gogoproto.nullable) = false]; + + // Indicates the row-level locking strength to be used by the scan. If set to + // FOR_NONE, no row-level locking should be performed. + optional sqlbase.ScanLockingStrength locking_strength = 10 [(gogoproto.nullable) = false]; + + // Indicates the policy to be used by the scan when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 11 [(gogoproto.nullable) = false]; } // IndexSkipTableReaderSpec is the specification for a table reader that @@ -138,6 +151,18 @@ message IndexSkipTableReaderSpec { optional ScanVisibility visibility = 4 [(gogoproto.nullable) = false]; optional bool reverse = 5 [(gogoproto.nullable) = false]; + + // Indicates the row-level locking strength to be used by the scan. If set to + // FOR_NONE, no row-level locking should be performed. + optional sqlbase.ScanLockingStrength locking_strength = 6 [(gogoproto.nullable) = false]; + + // Indicates the policy to be used by the scan when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 7 [(gogoproto.nullable) = false]; } // JoinReaderSpec is the specification for a "join reader". A join reader @@ -207,6 +232,18 @@ message JoinReaderSpec { // default PUBLIC state. Causes the index join to return these schema change // columns. optional ScanVisibility visibility = 7 [(gogoproto.nullable) = false]; + + // Indicates the row-level locking strength to be used by the join. If set to + // FOR_NONE, no row-level locking should be performed. + optional sqlbase.ScanLockingStrength locking_strength = 9 [(gogoproto.nullable) = false]; + + // Indicates the policy to be used by the join when dealing with rows being + // locked. Always set to BLOCK when locking_stength is FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 10 [(gogoproto.nullable) = false]; } // SorterSpec is the specification for a "sorting aggregator". A sorting @@ -545,6 +582,19 @@ message InterleavedReaderJoinerSpec { // Not used if there is a limit set in the PostProcessSpec of this processor // (that value will be used for sizing batches instead). optional int64 limit_hint = 3 [(gogoproto.nullable) = false]; + + // Indicates the row-level locking strength to be used by the scan over the + // tables. If set to FOR_NONE, no row-level locking should be performed. + optional sqlbase.ScanLockingStrength locking_strength = 6 [(gogoproto.nullable) = false]; + + // Indicates the policy to be used by the scan over the tables when dealing + // with rows being locked. Always set to BLOCK when locking_stength is + // FOR_NONE. + // + // NOTE: this is currently set but unused because only the BLOCK wait policy + // makes it out of the SQL optimizer without throwing an error. If/when other + // wait policies are supported, this field will be plumbed further. + optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 7 [(gogoproto.nullable) = false]; // Joiner component diff --git a/pkg/sql/opt/bench/stub_factory.go b/pkg/sql/opt/bench/stub_factory.go index e5fffa7290e0..9f5fad2dbb43 100644 --- a/pkg/sql/opt/bench/stub_factory.go +++ b/pkg/sql/opt/bench/stub_factory.go @@ -43,6 +43,7 @@ func (f *stubFactory) ConstructScan( maxResults uint64, reqOrdering exec.OutputOrdering, rowCount float64, + locking *tree.LockingItem, ) (exec.Node, error) { return struct{}{}, nil } diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 6f5f6a5a7261..f5e3c88f8269 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -501,6 +501,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { b.indexConstraintMaxResults(scan), res.reqOrdering(scan), rowCount, + scan.Locking, ) if err != nil { return execPlan{}, err diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index 82462b7804dd..a8550223f926 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -58,6 +58,8 @@ type Factory interface { // be 0. // - If maxResults > 0, the scan is guaranteed to return at most maxResults // rows. + // - If locking is provided, the scan should use the specified row-level + // locking mode. ConstructScan( table cat.Table, index cat.Index, @@ -69,6 +71,7 @@ type Factory interface { maxResults uint64, reqOrdering OutputOrdering, rowCount float64, + locking *tree.LockingItem, ) (Node, error) // ConstructVirtualScan returns a node that represents the scan of a virtual diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 73567d772131..18fb51840346 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -78,6 +78,7 @@ func (ef *execFactory) ConstructScan( maxResults uint64, reqOrdering exec.OutputOrdering, rowCount float64, + locking *tree.LockingItem, ) (exec.Node, error) { tabDesc := table.(*optTable).desc indexDesc := index.(*optIndex).desc @@ -122,6 +123,10 @@ func (ef *execFactory) ConstructScan( } scan.reqOrdering = ReqOrdering(reqOrdering) scan.estimatedRowCount = uint64(rowCount) + if locking != nil { + scan.lockingStrength = sqlbase.ToScanLockingStrength(locking.Strength) + scan.lockingWaitPolicy = sqlbase.ToScanLockingWaitPolicy(locking.WaitPolicy) + } return scan, nil } diff --git a/pkg/sql/row/cascader.go b/pkg/sql/row/cascader.go index 0de1b9a2a578..33aee8c33f57 100644 --- a/pkg/sql/row/cascader.go +++ b/pkg/sql/row/cascader.go @@ -414,6 +414,7 @@ func (c *cascader) addIndexPKRowFetcher( var rowFetcher Fetcher if err := rowFetcher.Init( false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, false, /* returnRangeInfo */ false, /* isCheck */ c.alloc, @@ -478,6 +479,10 @@ func (c *cascader) addRowDeleter( var rowFetcher Fetcher if err := rowFetcher.Init( false, /* reverse */ + // TODO(nvanbenschoten): it might make sense to use a FOR_UPDATE locking + // strength here. Consider hooking this in to the same knob that will + // control whether we perform locking implicitly during DELETEs. + sqlbase.ScanLockingStrength_FOR_NONE, false, /* returnRangeInfo */ false, /* isCheck */ c.alloc, @@ -540,6 +545,10 @@ func (c *cascader) addRowUpdater( var rowFetcher Fetcher if err := rowFetcher.Init( false, /* reverse */ + // TODO(nvanbenschoten): it might make sense to use a FOR_UPDATE locking + // strength here. Consider hooking this in to the same knob that will + // control whether we perform locking implicitly during UPDATEs. + sqlbase.ScanLockingStrength_FOR_NONE, false, /* returnRangeInfo */ false, /* isCheck */ c.alloc, diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 5f8e1593e7d7..3bdd028202f4 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -109,7 +109,12 @@ func NewUniquenessConstraintViolationError( ValNeededForCol: valNeededForCol, } if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, &sqlbase.DatumAlloc{}, tableArgs, + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + &sqlbase.DatumAlloc{}, + tableArgs, ); err != nil { return err } diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index b916f826804d..caf6422c5073 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -190,6 +190,9 @@ type Fetcher struct { // table has no interleave children. mustDecodeIndexKey bool + // lockStr represents the row-level locking mode to use when fetching rows. + lockStr sqlbase.ScanLockingStrength + // returnRangeInfo, if set, causes the underlying kvBatchFetcher to return // information about the ranges descriptors/leases uses in servicing the // requests. This has some cost, so it's only enabled by DistSQL when this @@ -242,7 +245,9 @@ func (rf *Fetcher) Reset() { // non-primary index, tables.ValNeededForCol can only refer to columns in the // index. func (rf *Fetcher) Init( - reverse, returnRangeInfo bool, + reverse bool, + lockStr sqlbase.ScanLockingStrength, + returnRangeInfo bool, isCheck bool, alloc *sqlbase.DatumAlloc, tables ...FetcherTableArgs, @@ -252,6 +257,7 @@ func (rf *Fetcher) Init( } rf.reverse = reverse + rf.lockStr = lockStr rf.returnRangeInfo = returnRangeInfo rf.alloc = alloc rf.isCheck = isCheck @@ -464,7 +470,13 @@ func (rf *Fetcher) StartScan( rf.traceKV = traceKV f, err := makeKVBatchFetcher( - txn, spans, rf.reverse, limitBatches, rf.firstBatchLimit(limitHint), rf.returnRangeInfo, + txn, + spans, + rf.reverse, + limitBatches, + rf.firstBatchLimit(limitHint), + rf.lockStr, + rf.returnRangeInfo, ) if err != nil { return err @@ -543,6 +555,7 @@ func (rf *Fetcher) StartInconsistentScan( rf.reverse, limitBatches, rf.firstBatchLimit(limitHint), + rf.lockStr, rf.returnRangeInfo, ) if err != nil { diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 421ae3d9456d..dc6d681d3fa6 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -106,7 +106,11 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { } var rf row.Fetcher if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, true /* isCheck */, &sqlbase.DatumAlloc{}, + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + true, /* isCheck */ + &sqlbase.DatumAlloc{}, args..., ); err != nil { t.Fatal(err) diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index cb5121994e25..ba4feb530711 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -71,8 +71,14 @@ func initFetcher( fetcherArgs := makeFetcherArgs(entries) - if err := fetcher.Init(reverseScan, false /*reverse*/, false, /* isCheck */ - alloc, fetcherArgs...); err != nil { + if err := fetcher.Init( + reverseScan, + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + alloc, + fetcherArgs..., + ); err != nil { return nil, err } @@ -1051,8 +1057,9 @@ func TestRowFetcherReset(t *testing.T) { // didn't reset. fetcherArgs := makeFetcherArgs(args) - if err := resetFetcher.Init(false, false /*reverse*/, false, /* isCheck */ - &da, fetcherArgs...); err != nil { + if err := resetFetcher.Init( + false /*reverse*/, 0 /* todo */, false /* returnRangeInfo */, false /* isCheck */, &da, fetcherArgs..., + ); err != nil { t.Fatal(err) } diff --git a/pkg/sql/row/fk_existence_base.go b/pkg/sql/row/fk_existence_base.go index d601243aea72..8716035e31cc 100644 --- a/pkg/sql/row/fk_existence_base.go +++ b/pkg/sql/row/fk_existence_base.go @@ -147,7 +147,13 @@ func makeFkExistenceCheckBaseHelper( } rf := &Fetcher{} if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, alloc, tableArgs); err != nil { + false, /* reverse */ + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + alloc, + tableArgs, + ); err != nil { return ret, err } diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index ff75b011506f..42a9e1de7bfd 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -53,6 +53,8 @@ type txnKVFetcher struct { firstBatchLimit int64 useBatchLimit bool reverse bool + // lockStr represents the locking mode to use when fetching KVs. + lockStr sqlbase.ScanLockingStrength // returnRangeInfo, if set, causes the kvBatchFetcher to populate rangeInfos. // See also rowFetcher.returnRangeInfo. returnRangeInfo bool @@ -145,6 +147,7 @@ func makeKVBatchFetcher( reverse bool, useBatchLimit bool, firstBatchLimit int64, + lockStr sqlbase.ScanLockingStrength, returnRangeInfo bool, ) (txnKVFetcher, error) { sendFn := func(ctx context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, error) { @@ -155,7 +158,7 @@ func makeKVBatchFetcher( return res, nil } return makeKVBatchFetcherWithSendFunc( - sendFn, spans, reverse, useBatchLimit, firstBatchLimit, returnRangeInfo, + sendFn, spans, reverse, useBatchLimit, firstBatchLimit, lockStr, returnRangeInfo, ) } @@ -167,6 +170,7 @@ func makeKVBatchFetcherWithSendFunc( reverse bool, useBatchLimit bool, firstBatchLimit int64, + lockStr sqlbase.ScanLockingStrength, returnRangeInfo bool, ) (txnKVFetcher, error) { if firstBatchLimit < 0 || (!useBatchLimit && firstBatchLimit != 0) { @@ -218,6 +222,7 @@ func makeKVBatchFetcherWithSendFunc( reverse: reverse, useBatchLimit: useBatchLimit, firstBatchLimit: firstBatchLimit, + lockStr: lockStr, returnRangeInfo: returnRangeInfo, }, nil } @@ -233,6 +238,7 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { for i := range f.spans { scans[i].ScanFormat = roachpb.BATCH_RESPONSE scans[i].SetSpan(f.spans[i]) + // TODO(nvanbenschoten): use f.lockStr here. ba.Requests[i].MustSetInner(&scans[i]) } } else { @@ -240,6 +246,7 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { for i := range f.spans { scans[i].ScanFormat = roachpb.BATCH_RESPONSE scans[i].SetSpan(f.spans[i]) + // TODO(nvanbenschoten): use f.lockStr here. ba.Requests[i].MustSetInner(&scans[i]) } } diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 2280a124dcc2..5536b60e078e 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" ) @@ -38,9 +39,12 @@ func NewKVFetcher( reverse bool, useBatchLimit bool, firstBatchLimit int64, + lockStr sqlbase.ScanLockingStrength, returnRangeInfo bool, ) (*KVFetcher, error) { - kvBatchFetcher, err := makeKVBatchFetcher(txn, spans, reverse, useBatchLimit, firstBatchLimit, returnRangeInfo) + kvBatchFetcher, err := makeKVBatchFetcher( + txn, spans, reverse, useBatchLimit, firstBatchLimit, lockStr, returnRangeInfo, + ) return newKVFetcher(&kvBatchFetcher), err } diff --git a/pkg/sql/rowexec/index_skip_table_reader.go b/pkg/sql/rowexec/index_skip_table_reader.go index c61656c86678..8034ac35e462 100644 --- a/pkg/sql/rowexec/index_skip_table_reader.go +++ b/pkg/sql/rowexec/index_skip_table_reader.go @@ -124,8 +124,14 @@ func newIndexSkipTableReader( ValNeededForCol: neededColumns, } - if err := t.fetcher.Init(t.reverse, true, /* returnRangeInfo */ - false /* isCheck */, &t.alloc, tableArgs); err != nil { + if err := t.fetcher.Init( + t.reverse, + spec.LockingStrength, + true, /* returnRangeInfo */ + false, /* isCheck */ + &t.alloc, + tableArgs, + ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/indexjoiner.go b/pkg/sql/rowexec/indexjoiner.go index 0db0eb915a05..682051b4f5a5 100644 --- a/pkg/sql/rowexec/indexjoiner.go +++ b/pkg/sql/rowexec/indexjoiner.go @@ -108,6 +108,7 @@ func newIndexJoiner( false, /* isCheck */ &ij.alloc, spec.Visibility, + spec.LockingStrength, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/interleaved_reader_joiner.go b/pkg/sql/rowexec/interleaved_reader_joiner.go index 9bcadbb26e3b..84d7fecb0937 100644 --- a/pkg/sql/rowexec/interleaved_reader_joiner.go +++ b/pkg/sql/rowexec/interleaved_reader_joiner.go @@ -367,7 +367,7 @@ func newInterleavedReaderJoiner( } if err := irj.initRowFetcher( - spec.Tables, tables, spec.Reverse, &irj.alloc, + spec.Tables, tables, spec.Reverse, spec.LockingStrength, &irj.alloc, ); err != nil { return nil, err } @@ -403,6 +403,7 @@ func (irj *interleavedReaderJoiner) initRowFetcher( tables []execinfrapb.InterleavedReaderJoinerSpec_Table, tableInfos []tableInfo, reverseScan bool, + lockStr sqlbase.ScanLockingStrength, alloc *sqlbase.DatumAlloc, ) error { args := make([]row.FetcherTableArgs, len(tables)) @@ -425,8 +426,14 @@ func (irj *interleavedReaderJoiner) initRowFetcher( } } - return irj.fetcher.Init(reverseScan, true /* returnRangeInfo */, true /* isCheck */, alloc, - args...) + return irj.fetcher.Init( + reverseScan, + lockStr, + true, /* returnRangeInfo */ + true, /* isCheck */ + alloc, + args..., + ) } func (irj *interleavedReaderJoiner) generateTrailingMeta( diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 6ec8f5391177..00f23a022599 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -206,7 +206,7 @@ func newJoinReader( var fetcher row.Fetcher _, _, err = initRowFetcher( &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ - neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, + neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, spec.LockingStrength, ) if err != nil { return nil, err diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index c50042d4ba6a..6016fae67785 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -62,6 +62,7 @@ func initRowFetcher( isCheck bool, alloc *sqlbase.DatumAlloc, scanVisibility execinfrapb.ScanVisibility, + lockStr sqlbase.ScanLockingStrength, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -82,7 +83,7 @@ func initRowFetcher( ValNeededForCol: valNeededForCol, } if err := fetcher.Init( - reverseScan, true /* returnRangeInfo */, isCheck, alloc, tableArgs, + reverseScan, lockStr, true /* returnRangeInfo */, isCheck, alloc, tableArgs, ); err != nil { return nil, false, err } diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index 71eb4b2a31f6..9bc5b2243c78 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -123,7 +123,7 @@ func newScrubTableReader( if _, _, err := initRowFetcher( &fetcher, &tr.tableDesc, int(spec.IndexIdx), tr.tableDesc.ColumnIdxMap(), spec.Reverse, neededColumns, true /* isCheck */, &tr.alloc, - execinfrapb.ScanVisibility_PUBLIC, + execinfrapb.ScanVisibility_PUBLIC, spec.LockingStrength, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 57999e19e653..6a55aea7a343 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -116,7 +116,7 @@ func newTableReader( columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) if _, _, err := initRowFetcher( &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, spec.Reverse, - neededColumns, spec.IsCheck, &tr.alloc, spec.Visibility, + neededColumns, spec.IsCheck, &tr.alloc, spec.Visibility, spec.LockingStrength, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 9d997a4a039f..a93d843fe17f 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -451,6 +451,9 @@ func (z *zigzagJoiner) setupInfo( false, /* check */ info.alloc, execinfrapb.ScanVisibility_PUBLIC, + // NB: zigzag joins are disabled when a row-level locking clause is + // supplied, so there is no locking strength on *ZigzagJoinerSpec. + sqlbase.ScanLockingStrength_FOR_NONE, ) if err != nil { return err diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 5978c72842ae..67fd5b817a19 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -116,6 +116,11 @@ type scanNode struct { // output. When there are no statistics to make the estimation, it will be // set to zero. estimatedRowCount uint64 + + // lockingStrength and lockingWaitPolicy represent the row-level locking + // mode of the Scan. + lockingStrength sqlbase.ScanLockingStrength + lockingWaitPolicy sqlbase.ScanLockingWaitPolicy } // scanVisibility represents which table columns should be included in a scan. diff --git a/pkg/sql/sqlbase/locking.go b/pkg/sql/sqlbase/locking.go new file mode 100644 index 000000000000..86f6c6d94a08 --- /dev/null +++ b/pkg/sql/sqlbase/locking.go @@ -0,0 +1,51 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sqlbase + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// ToScanLockingStrength converts a tree.LockingStrength to its corresponding +// ScanLockingStrength. +func ToScanLockingStrength(s tree.LockingStrength) ScanLockingStrength { + switch s { + case tree.ForNone: + return ScanLockingStrength_FOR_NONE + case tree.ForKeyShare: + return ScanLockingStrength_FOR_KEY_SHARE + case tree.ForShare: + return ScanLockingStrength_FOR_SHARE + case tree.ForNoKeyUpdate: + return ScanLockingStrength_FOR_NO_KEY_UPDATE + case tree.ForUpdate: + return ScanLockingStrength_FOR_UPDATE + default: + panic(fmt.Sprintf("unknown locking strength %s", s)) + } +} + +// ToScanLockingWaitPolicy converts a tree.LockingWaitPolicy to its +// corresponding ScanLockingWaitPolicy. +func ToScanLockingWaitPolicy(wp tree.LockingWaitPolicy) ScanLockingWaitPolicy { + switch wp { + case tree.LockWaitBlock: + return ScanLockingWaitPolicy_BLOCK + case tree.LockWaitSkip: + return ScanLockingWaitPolicy_SKIP + case tree.LockWaitError: + return ScanLockingWaitPolicy_ERROR + default: + panic(fmt.Sprintf("unknown locking wait policy %s", wp)) + } +} diff --git a/pkg/sql/sqlbase/locking.pb.go b/pkg/sql/sqlbase/locking.pb.go new file mode 100644 index 000000000000..5fbb9976ae5e --- /dev/null +++ b/pkg/sql/sqlbase/locking.pb.go @@ -0,0 +1,232 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: sql/sqlbase/locking.proto + +package sqlbase + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// ScanLockingStrength controls the row-level locking mode used by scans. +// +// Typically, SQL scans read sequential keys from the key-value layer without +// acquiring any locks. This means that two scans by different transactions will +// not conflict and cause one of the two transactions to block the other. This +// is usually desirable, as it increases concurrency between readers. +// +// However, there are cases where a SQL scan would like to acquire locks on each +// of the keys that it reads to more carefully control concurrent access to the +// data that it reads. The prototypical example of this is a scan that is used +// to fetch the initial value of a row that its transction intends to later +// update. In this case, it would be beneficial to acquire a lock on the row +// during the intial scan instead of waiting until the mutation to acquire a +// lock. This prevents the row from being modified between the scan and the +// mutation. It also prevents situations that can lead to deadlocks. +// +// Locking modes have differing levels of strength, growing from "weakest" to +// "strongest" in the order that the variants are presented in the enumeration. +// The "stronger" a locking mode, the more protection it provides for the lock +// holder but the more restrictive it is to concurrent transactions attempting +// to access the same keys. +// +// The following matrix presents the compatibility of locking strengths with one +// another. +// +// +-------------------+---------------+-----------+-------------------+------------+ +// | | FOR_KEY_SHARE | FOR_SHARE | FOR_NO_KEY_UPDATE | FOR_UPDATE | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_KEY_SHARE | | | | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_SHARE | | | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_NO_KEY_UPDATE | | X | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_UPDATE | X | X | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// +// A transaction can hold conflicting locks on the same row, but two different +// transactions can never hold conflicting locks on the same row. Once acquired, +// a lock is held until the end of the transaction. +type ScanLockingStrength int32 + +const ( + // FOR_NONE represents the default - no row-level locking. + ScanLockingStrength_FOR_NONE ScanLockingStrength = 0 + // FOR_KEY_SHARE represents the FOR KEY SHARE row-level locking mode. + // + // The mode behaves similarly to FOR SHARE, except that the lock is weaker: + // SELECT FOR UPDATE is blocked, but not SELECT FOR NO KEY UPDATE. A + // key-shared lock blocks other transactions from performing DELETE or any + // UPDATE that changes the key values, but not other UPDATE, and neither does + // it prevent SELECT FOR NO KEY UPDATE, SELECT FOR SHARE, or SELECT FOR KEY + // SHARE. + // + // The locking mode was introduced into Postgres as an alternative to FOR + // SHARE to improve concurrency between foreign key validation scans, which + // acquire FOR KEY SHARE locks, and UPDATEs to existing rows, which acquire + // FOR NO KEY UPDATE locks. + // + // NOTE: FOR_KEY_SHARE is currently ignored. No locks are acquired. + ScanLockingStrength_FOR_KEY_SHARE ScanLockingStrength = 1 + // FOR_SHARE represents the FOR SHARE row-level locking mode. + // + // The mode behaves similarly to FOR NO KEY UPDATE, except that it acquires a + // shared lock rather than exclusive lock on each retrieved row. A shared lock + // blocks other transactions from performing UPDATE, DELETE, SELECT FOR UPDATE + // or SELECT FOR NO KEY UPDATE on these rows, but it does not prevent them + // from performing SELECT FOR SHARE or SELECT FOR KEY SHARE. + // + // NOTE: FOR_SHARE is currently ignored. No locks are acquired. + ScanLockingStrength_FOR_SHARE ScanLockingStrength = 2 + // FOR_NO_KEY_UPDATE represents the FOR NO KEY UPDATE row-level locking mode. + // + // The mode behaves similarly to FOR UPDATE, except that the lock acquired is + // weaker: this lock will not block SELECT FOR KEY SHARE commands that attempt + // to acquire a lock on the same rows. This lock mode is also acquired by any + // UPDATE that does not acquire a FOR UPDATE lock. + // + // The locking mode was introduced into Postgres as an alternative to FOR + // UDPATE to improve concurrency between foreign key validation scans, which + // acquire FOR KEY SHARE locks, and UPDATEs to existing rows, which acquire + // FOR NO KEY UPDATE locks. + // + // NOTE: FOR_NO_KEY_UPDATE is currently promoted to FOR_UPDATE. + ScanLockingStrength_FOR_NO_KEY_UPDATE ScanLockingStrength = 3 + // FOR_UPDATE represents the FOR UPDATE row-level locking mode. + // + // The mode causes the rows retrieved by the scan to be locked as though for + // update. This prevents them from being locked, modified or deleted by other + // transactions until the current transaction ends. That is, other + // transactions that attempt UPDATE, DELETE, SELECT FOR UPDATE, SELECT FOR NO + // KEY UPDATE, SELECT FOR SHARE or SELECT FOR KEY SHARE of these rows will be + // blocked until the current transaction ends. Conversely, SELECT FOR UPDATE + // will wait for a concurrent transaction that has run any of those commands + // on the same row, and will then lock and return the updated row (or no row, + // if the row was deleted). + // + // NOTE: FOR_UPDATE is currently implemented by acquiring lock.Exclusive locks + // on each key scanned. + ScanLockingStrength_FOR_UPDATE ScanLockingStrength = 4 +) + +var ScanLockingStrength_name = map[int32]string{ + 0: "FOR_NONE", + 1: "FOR_KEY_SHARE", + 2: "FOR_SHARE", + 3: "FOR_NO_KEY_UPDATE", + 4: "FOR_UPDATE", +} +var ScanLockingStrength_value = map[string]int32{ + "FOR_NONE": 0, + "FOR_KEY_SHARE": 1, + "FOR_SHARE": 2, + "FOR_NO_KEY_UPDATE": 3, + "FOR_UPDATE": 4, +} + +func (x ScanLockingStrength) Enum() *ScanLockingStrength { + p := new(ScanLockingStrength) + *p = x + return p +} +func (x ScanLockingStrength) String() string { + return proto.EnumName(ScanLockingStrength_name, int32(x)) +} +func (x *ScanLockingStrength) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(ScanLockingStrength_value, data, "ScanLockingStrength") + if err != nil { + return err + } + *x = ScanLockingStrength(value) + return nil +} +func (ScanLockingStrength) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_locking_6062bb649214c66a, []int{0} +} + +// ScanLockingWaitPolicy controls the policy used by scans for dealing with rows +// being locked by FOR UPDATE/SHARE clauses. +type ScanLockingWaitPolicy int32 + +const ( + // BLOCK represents the default - wait for the lock to become available. + ScanLockingWaitPolicy_BLOCK ScanLockingWaitPolicy = 0 + // SKIP represents SKIP LOCKED - skip rows that can't be locked. + // + // NOTE: SKIP is not currently implemented and does not make it out of the SQL + // optimizer without throwing an error. + ScanLockingWaitPolicy_SKIP ScanLockingWaitPolicy = 1 + // ERROR represents NOWAIT - raise an error if a row cannot be locked. + // + // NOTE: ERROR is not currently implemented and does not make it out of the + // SQL optimizer without throwing an error. + ScanLockingWaitPolicy_ERROR ScanLockingWaitPolicy = 2 +) + +var ScanLockingWaitPolicy_name = map[int32]string{ + 0: "BLOCK", + 1: "SKIP", + 2: "ERROR", +} +var ScanLockingWaitPolicy_value = map[string]int32{ + "BLOCK": 0, + "SKIP": 1, + "ERROR": 2, +} + +func (x ScanLockingWaitPolicy) Enum() *ScanLockingWaitPolicy { + p := new(ScanLockingWaitPolicy) + *p = x + return p +} +func (x ScanLockingWaitPolicy) String() string { + return proto.EnumName(ScanLockingWaitPolicy_name, int32(x)) +} +func (x *ScanLockingWaitPolicy) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(ScanLockingWaitPolicy_value, data, "ScanLockingWaitPolicy") + if err != nil { + return err + } + *x = ScanLockingWaitPolicy(value) + return nil +} +func (ScanLockingWaitPolicy) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_locking_6062bb649214c66a, []int{1} +} + +func init() { + proto.RegisterEnum("cockroach.sql.sqlbase.ScanLockingStrength", ScanLockingStrength_name, ScanLockingStrength_value) + proto.RegisterEnum("cockroach.sql.sqlbase.ScanLockingWaitPolicy", ScanLockingWaitPolicy_name, ScanLockingWaitPolicy_value) +} + +func init() { proto.RegisterFile("sql/sqlbase/locking.proto", fileDescriptor_locking_6062bb649214c66a) } + +var fileDescriptor_locking_6062bb649214c66a = []byte{ + // 238 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2c, 0x2e, 0xcc, 0xd1, + 0x2f, 0x2e, 0xcc, 0x49, 0x4a, 0x2c, 0x4e, 0xd5, 0xcf, 0xc9, 0x4f, 0xce, 0xce, 0xcc, 0x4b, 0xd7, + 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x4d, 0xce, 0x4f, 0xce, 0x2e, 0xca, 0x4f, 0x4c, 0xce, + 0xd0, 0x2b, 0x2e, 0xcc, 0xd1, 0x83, 0x2a, 0xd2, 0xca, 0xe1, 0x12, 0x0e, 0x4e, 0x4e, 0xcc, 0xf3, + 0x81, 0xa8, 0x0d, 0x2e, 0x29, 0x4a, 0xcd, 0x4b, 0x2f, 0xc9, 0x10, 0xe2, 0xe1, 0xe2, 0x70, 0xf3, + 0x0f, 0x8a, 0xf7, 0xf3, 0xf7, 0x73, 0x15, 0x60, 0x10, 0x12, 0xe4, 0xe2, 0x05, 0xf1, 0xbc, 0x5d, + 0x23, 0xe3, 0x83, 0x3d, 0x1c, 0x83, 0x5c, 0x05, 0x18, 0x85, 0x78, 0xb9, 0x38, 0x41, 0x42, 0x10, + 0x2e, 0x93, 0x90, 0x28, 0x97, 0x20, 0x44, 0x3d, 0x58, 0x51, 0x68, 0x80, 0x8b, 0x63, 0x88, 0xab, + 0x00, 0xb3, 0x10, 0x1f, 0x17, 0x17, 0x48, 0x18, 0xca, 0x67, 0xd1, 0x32, 0xe7, 0x12, 0x45, 0xb2, + 0x2d, 0x3c, 0x31, 0xb3, 0x24, 0x20, 0x3f, 0x27, 0x33, 0xb9, 0x52, 0x88, 0x93, 0x8b, 0xd5, 0xc9, + 0xc7, 0xdf, 0xd9, 0x5b, 0x80, 0x41, 0x88, 0x83, 0x8b, 0x25, 0xd8, 0xdb, 0x33, 0x40, 0x80, 0x11, + 0x24, 0xe8, 0x1a, 0x14, 0xe4, 0x1f, 0x24, 0xc0, 0xe4, 0xa4, 0x79, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, + 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, + 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x0e, 0xf5, + 0x11, 0x20, 0x00, 0x00, 0xff, 0xff, 0x5c, 0xc5, 0xee, 0xb0, 0x04, 0x01, 0x00, 0x00, +} diff --git a/pkg/sql/sqlbase/locking.proto b/pkg/sql/sqlbase/locking.proto new file mode 100644 index 000000000000..6d7c5bc8882b --- /dev/null +++ b/pkg/sql/sqlbase/locking.proto @@ -0,0 +1,136 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto2"; +package cockroach.sql.sqlbase; +option go_package = "sqlbase"; + +// ScanLockingStrength controls the row-level locking mode used by scans. +// +// Typically, SQL scans read sequential keys from the key-value layer without +// acquiring any locks. This means that two scans by different transactions will +// not conflict and cause one of the two transactions to block the other. This +// is usually desirable, as it increases concurrency between readers. +// +// However, there are cases where a SQL scan would like to acquire locks on each +// of the keys that it reads to more carefully control concurrent access to the +// data that it reads. The prototypical example of this is a scan that is used +// to fetch the initial value of a row that its transction intends to later +// update. In this case, it would be beneficial to acquire a lock on the row +// during the intial scan instead of waiting until the mutation to acquire a +// lock. This prevents the row from being modified between the scan and the +// mutation. It also prevents situations that can lead to deadlocks. +// +// Locking modes have differing levels of strength, growing from "weakest" to +// "strongest" in the order that the variants are presented in the enumeration. +// The "stronger" a locking mode, the more protection it provides for the lock +// holder but the more restrictive it is to concurrent transactions attempting +// to access the same keys. +// +// The following matrix presents the compatibility of locking strengths with one +// another. +// +// +-------------------+---------------+-----------+-------------------+------------+ +// | | FOR_KEY_SHARE | FOR_SHARE | FOR_NO_KEY_UPDATE | FOR_UPDATE | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_KEY_SHARE | | | | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_SHARE | | | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_NO_KEY_UPDATE | | X | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// | FOR_UPDATE | X | X | X | X | +// +-------------------+---------------+-----------+-------------------+------------+ +// +// A transaction can hold conflicting locks on the same row, but two different +// transactions can never hold conflicting locks on the same row. Once acquired, +// a lock is held until the end of the transaction. +enum ScanLockingStrength { + // FOR_NONE represents the default - no row-level locking. + FOR_NONE = 0; + + // FOR_KEY_SHARE represents the FOR KEY SHARE row-level locking mode. + // + // The mode behaves similarly to FOR SHARE, except that the lock is weaker: + // SELECT FOR UPDATE is blocked, but not SELECT FOR NO KEY UPDATE. A + // key-shared lock blocks other transactions from performing DELETE or any + // UPDATE that changes the key values, but not other UPDATE, and neither does + // it prevent SELECT FOR NO KEY UPDATE, SELECT FOR SHARE, or SELECT FOR KEY + // SHARE. + // + // The locking mode was introduced into Postgres as an alternative to FOR + // SHARE to improve concurrency between foreign key validation scans, which + // acquire FOR KEY SHARE locks, and UPDATEs to existing rows, which acquire + // FOR NO KEY UPDATE locks. + // + // NOTE: FOR_KEY_SHARE is currently ignored. No locks are acquired. + FOR_KEY_SHARE = 1; + + // FOR_SHARE represents the FOR SHARE row-level locking mode. + // + // The mode behaves similarly to FOR NO KEY UPDATE, except that it acquires a + // shared lock rather than exclusive lock on each retrieved row. A shared lock + // blocks other transactions from performing UPDATE, DELETE, SELECT FOR UPDATE + // or SELECT FOR NO KEY UPDATE on these rows, but it does not prevent them + // from performing SELECT FOR SHARE or SELECT FOR KEY SHARE. + // + // NOTE: FOR_SHARE is currently ignored. No locks are acquired. + FOR_SHARE = 2; + + // FOR_NO_KEY_UPDATE represents the FOR NO KEY UPDATE row-level locking mode. + // + // The mode behaves similarly to FOR UPDATE, except that the lock acquired is + // weaker: this lock will not block SELECT FOR KEY SHARE commands that attempt + // to acquire a lock on the same rows. This lock mode is also acquired by any + // UPDATE that does not acquire a FOR UPDATE lock. + // + // The locking mode was introduced into Postgres as an alternative to FOR + // UDPATE to improve concurrency between foreign key validation scans, which + // acquire FOR KEY SHARE locks, and UPDATEs to existing rows, which acquire + // FOR NO KEY UPDATE locks. + // + // NOTE: FOR_NO_KEY_UPDATE is currently promoted to FOR_UPDATE. + FOR_NO_KEY_UPDATE = 3; + + // FOR_UPDATE represents the FOR UPDATE row-level locking mode. + // + // The mode causes the rows retrieved by the scan to be locked as though for + // update. This prevents them from being locked, modified or deleted by other + // transactions until the current transaction ends. That is, other + // transactions that attempt UPDATE, DELETE, SELECT FOR UPDATE, SELECT FOR NO + // KEY UPDATE, SELECT FOR SHARE or SELECT FOR KEY SHARE of these rows will be + // blocked until the current transaction ends. Conversely, SELECT FOR UPDATE + // will wait for a concurrent transaction that has run any of those commands + // on the same row, and will then lock and return the updated row (or no row, + // if the row was deleted). + // + // NOTE: FOR_UPDATE is currently implemented by acquiring lock.Exclusive locks + // on each key scanned. + FOR_UPDATE = 4; +} + +// ScanLockingWaitPolicy controls the policy used by scans for dealing with rows +// being locked by FOR UPDATE/SHARE clauses. +enum ScanLockingWaitPolicy { + // BLOCK represents the default - wait for the lock to become available. + BLOCK = 0; + + // SKIP represents SKIP LOCKED - skip rows that can't be locked. + // + // NOTE: SKIP is not currently implemented and does not make it out of the SQL + // optimizer without throwing an error. + SKIP = 1; + + // ERROR represents NOWAIT - raise an error if a row cannot be locked. + // + // NOTE: ERROR is not currently implemented and does not make it out of the + // SQL optimizer without throwing an error. + ERROR = 2; +} diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index d063f9c34bf2..e838a295ceca 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -164,7 +164,15 @@ func (td *tableDeleter) deleteAllRowsScan( ValNeededForCol: valNeededForCol, } if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, td.alloc, tableArgs, + false, /* reverse */ + // TODO(nvanbenschoten): it might make sense to use a FOR_UPDATE locking + // strength here. Consider hooking this in to the same knob that will + // control whether we perform locking implicitly during DELETEs. + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + td.alloc, + tableArgs, ); err != nil { return resume, err } @@ -276,7 +284,15 @@ func (td *tableDeleter) deleteIndexScan( ValNeededForCol: valNeededForCol, } if err := rf.Init( - false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, td.alloc, tableArgs, + false, /* reverse */ + // TODO(nvanbenschoten): it might make sense to use a FOR_UPDATE locking + // strength here. Consider hooking this in to the same knob that will + // control whether we perform locking implicitly during DELETEs. + sqlbase.ScanLockingStrength_FOR_NONE, + false, /* returnRangeInfo */ + false, /* isCheck */ + td.alloc, + tableArgs, ); err != nil { return resume, err } From 1cbcb78b9e344f1d9b1b6ebb1c76f4db32b0c6dd Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 26 Dec 2019 17:56:02 -0500 Subject: [PATCH 2/4] [DNM] storage/result: add UpdatedIntents and ResolvedIntents to LocalResult This allows the handling of added, updated, and resolved intents to become more reactive. It mirrors our handling of UpdatedTxns and their interaction with the TxnWaitQueue. --- pkg/storage/batcheval/cmd_conditional_put.go | 10 +++++-- pkg/storage/batcheval/cmd_delete.go | 6 +++- pkg/storage/batcheval/cmd_delete_range.go | 11 +++++-- pkg/storage/batcheval/cmd_end_transaction.go | 20 ++++++++----- pkg/storage/batcheval/cmd_increment.go | 5 +++- pkg/storage/batcheval/cmd_init_put.go | 10 +++++-- pkg/storage/batcheval/cmd_put.go | 10 +++++-- pkg/storage/batcheval/cmd_resolve_intent.go | 1 + .../batcheval/cmd_resolve_intent_range.go | 1 + pkg/storage/batcheval/result/intent.go | 29 +++++++++++++++++++ pkg/storage/batcheval/result/result.go | 26 +++++++++++++++-- pkg/storage/replica_proposal.go | 10 +++++++ 12 files changed, 119 insertions(+), 20 deletions(-) diff --git a/pkg/storage/batcheval/cmd_conditional_put.go b/pkg/storage/batcheval/cmd_conditional_put.go index 78bb945f8294..e965664f1c88 100644 --- a/pkg/storage/batcheval/cmd_conditional_put.go +++ b/pkg/storage/batcheval/cmd_conditional_put.go @@ -41,8 +41,14 @@ func ConditionalPut( } } handleMissing := engine.CPutMissingBehavior(args.AllowIfDoesNotExist) + var err error if args.Blind { - return result.Result{}, engine.MVCCBlindConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.ExpValue, handleMissing, h.Txn) + err = engine.MVCCBlindConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.ExpValue, handleMissing, h.Txn) + } else { + err = engine.MVCCConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.ExpValue, handleMissing, h.Txn) } - return result.Result{}, engine.MVCCConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.ExpValue, handleMissing, h.Txn) + if err != nil { + return result.Result{}, err + } + return result.FromUpdatedIntent(h.Txn, args.Key), nil } diff --git a/pkg/storage/batcheval/cmd_delete.go b/pkg/storage/batcheval/cmd_delete.go index 85fdfa79c27b..215f0962d9a4 100644 --- a/pkg/storage/batcheval/cmd_delete.go +++ b/pkg/storage/batcheval/cmd_delete.go @@ -29,5 +29,9 @@ func Delete( args := cArgs.Args.(*roachpb.DeleteRequest) h := cArgs.Header - return result.Result{}, engine.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn) + err := engine.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn) + if err != nil { + return result.Result{}, err + } + return result.FromUpdatedIntent(h.Txn, args.Key), nil } diff --git a/pkg/storage/batcheval/cmd_delete_range.go b/pkg/storage/batcheval/cmd_delete_range.go index c66143067ed3..8f7ffd16b866 100644 --- a/pkg/storage/batcheval/cmd_delete_range.go +++ b/pkg/storage/batcheval/cmd_delete_range.go @@ -51,10 +51,15 @@ func DeleteRange( if !args.Inline { timestamp = h.Timestamp } + returnKeys := args.ReturnKeys || h.Txn != nil deleted, resumeSpan, num, err := engine.MVCCDeleteRange( - ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, cArgs.MaxKeys, timestamp, h.Txn, args.ReturnKeys, + ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, cArgs.MaxKeys, timestamp, h.Txn, returnKeys, ) - if err == nil { + if err != nil { + return result.Result{}, err + } + + if args.ReturnKeys { reply.Keys = deleted } reply.NumKeys = num @@ -62,5 +67,5 @@ func DeleteRange( reply.ResumeSpan = resumeSpan reply.ResumeReason = roachpb.RESUME_KEY_LIMIT } - return result.Result{}, err + return result.FromUpdatedIntents(h.Txn, deleted), nil } diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index dc8b54b9bdc6..4e9be72c4efa 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -223,7 +223,7 @@ func EndTxn( // Do not return TransactionAbortedError since the client anyway // wanted to abort the transaction. desc := cArgs.EvalCtx.Desc() - externalIntents, err := resolveLocalIntents(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx) + externalIntents, resolvedIntents, err := resolveLocalIntents(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx) if err != nil { return result.Result{}, err } @@ -234,7 +234,9 @@ func EndTxn( } // Use alwaysReturn==true because the transaction is definitely // aborted, no matter what happens to this command. - return result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison), nil + res := result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison) + res.Local.ResolvedIntents = resolvedIntents + return res, nil } // If the transaction was previously aborted by a concurrent writer's // push, any intents written are still open. It's only now that we know @@ -324,7 +326,7 @@ func EndTxn( // This avoids the need for the intentResolver to have to return to this range // to resolve intents for this transaction in the future. desc := cArgs.EvalCtx.Desc() - externalIntents, err := resolveLocalIntents(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx) + externalIntents, resolvedIntents, err := resolveLocalIntents(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx) if err != nil { return result.Result{}, err } @@ -368,6 +370,7 @@ func EndTxn( // if the commit actually happens; otherwise, we risk losing writes. intentsResult := result.FromEndTxn(reply.Txn, false /* alwaysReturn */, args.Poison) intentsResult.Local.UpdatedTxns = []*roachpb.Transaction{reply.Txn} + intentsResult.Local.ResolvedIntents = resolvedIntents if err := pd.MergeAndDestroy(intentsResult); err != nil { return result.Result{}, err } @@ -451,7 +454,7 @@ func resolveLocalIntents( args *roachpb.EndTxnRequest, txn *roachpb.Transaction, evalCtx EvalContext, -) ([]roachpb.Span, error) { +) ([]roachpb.Span, []roachpb.Intent, error) { if mergeTrigger := args.InternalCommitTrigger.GetMergeTrigger(); mergeTrigger != nil { // If this is a merge, then use the post-merge descriptor to determine // which intents are local (note that for a split, we want to use the @@ -466,6 +469,7 @@ func resolveLocalIntents( defer iterAndBuf.Cleanup() var externalIntents []roachpb.Span + var resolvedIntents []roachpb.Intent var resolveAllowance int64 = intentResolutionBatchSize if args.InternalCommitTrigger != nil { // If this is a system transaction (such as a split or merge), don't enforce the resolve allowance. @@ -490,6 +494,7 @@ func resolveLocalIntents( ok, err := engine.MVCCResolveWriteIntentUsingIter(ctx, readWriter, iterAndBuf, resolveMS, intent) if ok { resolveAllowance-- + resolvedIntents = append(resolvedIntents, intent) } return err } @@ -514,21 +519,22 @@ func resolveLocalIntents( } externalIntents = append(externalIntents, *resumeSpan) } + resolvedIntents = append(resolvedIntents, intent) // TODO(nvanbenschoten): resume span return nil } return nil }(); err != nil { - return nil, errors.Wrapf(err, "resolving intent at %s on end transaction [%s]", span, txn.Status) + return nil, nil, errors.Wrapf(err, "resolving intent at %s on end transaction [%s]", span, txn.Status) } } removedAny := resolveAllowance != intentResolutionBatchSize if WriteAbortSpanOnResolve(txn.Status, args.Poison, removedAny) { if err := UpdateAbortSpan(ctx, evalCtx, readWriter, ms, txn.TxnMeta, args.Poison); err != nil { - return nil, err + return nil, nil, err } } - return externalIntents, nil + return externalIntents, resolvedIntents, nil } // updateStagingTxn persists the STAGING transaction record with updated status diff --git a/pkg/storage/batcheval/cmd_increment.go b/pkg/storage/batcheval/cmd_increment.go index 78c79a31cf8b..ec6151f0cad9 100644 --- a/pkg/storage/batcheval/cmd_increment.go +++ b/pkg/storage/batcheval/cmd_increment.go @@ -33,6 +33,9 @@ func Increment( reply := resp.(*roachpb.IncrementResponse) newVal, err := engine.MVCCIncrement(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn, args.Increment) + if err != nil { + return result.Result{}, err + } reply.NewValue = newVal - return result.Result{}, err + return result.FromUpdatedIntent(h.Txn, args.Key), nil } diff --git a/pkg/storage/batcheval/cmd_init_put.go b/pkg/storage/batcheval/cmd_init_put.go index bba3f0232d37..7fa11319e375 100644 --- a/pkg/storage/batcheval/cmd_init_put.go +++ b/pkg/storage/batcheval/cmd_init_put.go @@ -41,8 +41,14 @@ func InitPut( defer readWriter.Close() } } + var err error if args.Blind { - return result.Result{}, engine.MVCCBlindInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + err = engine.MVCCBlindInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + } else { + err = engine.MVCCInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) } - return result.Result{}, engine.MVCCInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + if err != nil { + return result.Result{}, err + } + return result.FromUpdatedIntent(h.Txn, args.Key), nil } diff --git a/pkg/storage/batcheval/cmd_put.go b/pkg/storage/batcheval/cmd_put.go index 51b41adacbce..bd6e0ab055fb 100644 --- a/pkg/storage/batcheval/cmd_put.go +++ b/pkg/storage/batcheval/cmd_put.go @@ -59,8 +59,14 @@ func Put( defer readWriter.Close() } } + var err error if args.Blind { - return result.Result{}, engine.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + err = engine.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + } else { + err = engine.MVCCPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + } + if err != nil { + return result.Result{}, err } - return result.Result{}, engine.MVCCPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + return result.FromUpdatedIntent(h.Txn, args.Key), nil } diff --git a/pkg/storage/batcheval/cmd_resolve_intent.go b/pkg/storage/batcheval/cmd_resolve_intent.go index 5b0836545361..cb06f49a59b1 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent.go +++ b/pkg/storage/batcheval/cmd_resolve_intent.go @@ -86,6 +86,7 @@ func ResolveIntent( } var res result.Result + res.Local.ResolvedIntents = []roachpb.Intent{intent} res.Local.Metrics = resolveToMetricType(args.Status, args.Poison) if WriteAbortSpanOnResolve(args.Status, args.Poison, ok) { diff --git a/pkg/storage/batcheval/cmd_resolve_intent_range.go b/pkg/storage/batcheval/cmd_resolve_intent_range.go index e4175053c7aa..65d70c9f09b7 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_range.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_range.go @@ -61,6 +61,7 @@ func ResolveIntentRange( } var res result.Result + res.Local.ResolvedIntents = []roachpb.Intent{intent} // TODO(nvanbenschoten): resume span res.Local.Metrics = resolveToMetricType(args.Status, args.Poison) if WriteAbortSpanOnResolve(args.Status, args.Poison, numKeys > 0) { diff --git a/pkg/storage/batcheval/result/intent.go b/pkg/storage/batcheval/result/intent.go index 22013c214004..bf8ae0b954bb 100644 --- a/pkg/storage/batcheval/result/intent.go +++ b/pkg/storage/batcheval/result/intent.go @@ -12,6 +12,35 @@ package result import "github.com/cockroachdb/cockroach/pkg/roachpb" +// FromUpdatedIntent creates a Result communicating that an intent was updated +// by the given request and should be handled. +func FromUpdatedIntent(txn *roachpb.Transaction, key roachpb.Key) Result { + var pd Result + if txn == nil { + return pd + } + pd.Local.UpdatedIntents = []roachpb.Intent{{ + Span: roachpb.Span{Key: key}, Txn: txn.TxnMeta, Status: roachpb.PENDING, + }} + return pd +} + +// FromUpdatedIntents creates a Result communicating that the intents were +// updated by the given request and should be handled. +func FromUpdatedIntents(txn *roachpb.Transaction, keys []roachpb.Key) Result { + var pd Result + if txn == nil { + return pd + } + pd.Local.UpdatedIntents = make([]roachpb.Intent, len(keys)) + for i := range pd.Local.UpdatedIntents { + pd.Local.UpdatedIntents[i] = roachpb.Intent{ + Span: roachpb.Span{Key: keys[i]}, Txn: txn.TxnMeta, Status: roachpb.PENDING, + } + } + return pd +} + // FromEncounteredIntents creates a Result communicating that the intents were encountered // by the given request and should be handled. func FromEncounteredIntents(intents []roachpb.Intent) Result { diff --git a/pkg/storage/batcheval/result/result.go b/pkg/storage/batcheval/result/result.go index afc01436a0ad..7e90fd091b10 100644 --- a/pkg/storage/batcheval/result/result.go +++ b/pkg/storage/batcheval/result/result.go @@ -28,6 +28,10 @@ import ( type LocalResult struct { Reply *roachpb.BatchResponse + // UpdatedIntents stores any newly created or updated intents. + UpdatedIntents []roachpb.Intent + // ResolvedIntents stores any resolved intents. + ResolvedIntents []roachpb.Intent // EncounteredIntents stores any intents encountered but not conflicted // with. They should be handed off to asynchronous intent processing on // the proposer, so that an attempt to resolve them is made. @@ -66,6 +70,8 @@ type LocalResult struct { func (lResult *LocalResult) IsZero() bool { // NB: keep in order. return lResult.Reply == nil && + lResult.UpdatedIntents == nil && + lResult.ResolvedIntents == nil && lResult.EncounteredIntents == nil && lResult.UpdatedTxns == nil && lResult.EndTxns == nil && @@ -80,11 +86,13 @@ func (lResult *LocalResult) String() string { if lResult == nil { return "LocalResult: nil" } - return fmt.Sprintf("LocalResult (reply: %v, #encountered intents: %d, "+ + return fmt.Sprintf("LocalResult (reply: %v, "+ + "#updated intents: %d #resolved intents: %d #encountered intents: %d, "+ "#updated txns: %d #end txns: %d, "+ "GossipFirstRange:%t MaybeGossipSystemConfig:%t MaybeAddToSplitQueue:%t "+ "MaybeGossipNodeLiveness:%s MaybeWatchForMerge:%t", - lResult.Reply, len(lResult.EncounteredIntents), + lResult.Reply, + len(lResult.UpdatedIntents), len(lResult.ResolvedIntents), len(lResult.EncounteredIntents), len(lResult.UpdatedTxns), len(lResult.EndTxns), lResult.GossipFirstRange, lResult.MaybeGossipSystemConfig, lResult.MaybeAddToSplitQueue, lResult.MaybeGossipNodeLiveness, lResult.MaybeWatchForMerge) @@ -277,6 +285,20 @@ func (p *Result) MergeAndDestroy(q Result) error { } q.Replicated.PrevLeaseProposal = nil + if p.Local.UpdatedIntents == nil { + p.Local.UpdatedIntents = q.Local.UpdatedIntents + } else { + p.Local.UpdatedIntents = append(p.Local.UpdatedIntents, q.Local.UpdatedIntents...) + } + q.Local.UpdatedIntents = nil + + if p.Local.ResolvedIntents == nil { + p.Local.ResolvedIntents = q.Local.ResolvedIntents + } else { + p.Local.ResolvedIntents = append(p.Local.ResolvedIntents, q.Local.ResolvedIntents...) + } + q.Local.ResolvedIntents = nil + if p.Local.EncounteredIntents == nil { p.Local.EncounteredIntents = q.Local.EncounteredIntents } else { diff --git a/pkg/storage/replica_proposal.go b/pkg/storage/replica_proposal.go index 2b5bdc16be67..6d8d82352454 100644 --- a/pkg/storage/replica_proposal.go +++ b/pkg/storage/replica_proposal.go @@ -609,6 +609,16 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re log.Fatalf(ctx, "LocalEvalResult.MaybeWatchForMerge should be false") } + if lResult.UpdatedIntents != nil { + // TODO(nvanbenschoten): handle UpdatedIntents. + lResult.UpdatedIntents = nil + } + + if lResult.ResolvedIntents != nil { + // TODO(nvanbenschoten): handle ResolvedIntents. + lResult.ResolvedIntents = nil + } + if lResult.UpdatedTxns != nil { for _, txn := range lResult.UpdatedTxns { r.txnWaitQueue.UpdateTxn(ctx, txn) From 7ada06ba610bfd188131ca5377c5ae6a2109d96f Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jan 2020 17:57:20 -0500 Subject: [PATCH 3/4] [DNM] storage/concurrency: introduce concurrency control package, prototype SFU Informs #41720. This commit creates a new concurrency package that provides a concurrency manager structure that encapsulates the details of concurrency control and contention handling for serializable key-value transactions. Any reader of this commit should start at `concurrency_control.go` and move out from there. The new package has a few primary objectives: 1. centralize the handling of request synchronization and transaction contention handling in a single location, allowing for the topic to be documented and understood in isolation. 2. rework contention handling to react to intent state transitions directly. This simplifies the transaction queueing story, reduces the frequency of transaction push RPCs, and allows waiters to proceed after intent resolution as soon as possible. 3. create a framework that naturally permits "update" locking, which is required for kv-level SELECT FOR UPDATE support (#6583). 4. provide stronger guarantees around fairness when transactions conflict, in order to reduce tail latencies under contended scenarios. 5. create a structure that can extend to address the long-term goals of a fully centralized lock-table laid out in #41720. WARNING: this is still a WIP. Notably, the lockTableImpl is mocked out with a working but incomplete implementation. See #43740 for a more complete strawman. Release note: None --- pkg/storage/batcheval/cmd_query_txn.go | 2 +- .../batcheval/cmd_resolve_intent_test.go | 4 +- pkg/storage/batcheval/eval_context.go | 4 +- .../concurrency/concurrency_control.go | 355 +++++++++++++++++ .../concurrency/concurrency_manager.go | 357 ++++++++++++++++++ pkg/storage/concurrency/latch_manager.go | 35 ++ pkg/storage/concurrency/lock_table.go | 335 ++++++++++++++++ pkg/storage/helpers_test.go | 4 - pkg/storage/intentresolver/intent_resolver.go | 4 +- pkg/storage/replica.go | 93 +---- pkg/storage/replica_eval_context_span.go | 8 +- pkg/storage/replica_init.go | 8 +- pkg/storage/replica_metrics.go | 7 +- pkg/storage/replica_proposal.go | 23 +- pkg/storage/replica_raft.go | 34 +- pkg/storage/replica_read.go | 29 +- pkg/storage/replica_send.go | 160 ++++---- pkg/storage/replica_test.go | 36 +- pkg/storage/replica_tscache.go | 3 + pkg/storage/replica_write.go | 49 +-- pkg/storage/spanset/spanset.go | 12 + pkg/storage/store.go | 8 + pkg/storage/store_merge.go | 6 +- pkg/storage/store_send.go | 36 -- pkg/storage/store_split.go | 2 +- pkg/storage/txnwait/queue.go | 20 +- 26 files changed, 1305 insertions(+), 329 deletions(-) create mode 100644 pkg/storage/concurrency/concurrency_control.go create mode 100644 pkg/storage/concurrency/concurrency_manager.go create mode 100644 pkg/storage/concurrency/latch_manager.go create mode 100644 pkg/storage/concurrency/lock_table.go diff --git a/pkg/storage/batcheval/cmd_query_txn.go b/pkg/storage/batcheval/cmd_query_txn.go index 5769d1f4386b..829e409f0c85 100644 --- a/pkg/storage/batcheval/cmd_query_txn.go +++ b/pkg/storage/batcheval/cmd_query_txn.go @@ -78,6 +78,6 @@ func QueryTxn( } // Get the list of txns waiting on this txn. - reply.WaitingTxns = cArgs.EvalCtx.GetTxnWaitQueue().GetDependents(args.Txn.ID) + reply.WaitingTxns = cArgs.EvalCtx.GetConcurrencyManager().GetDependents(args.Txn.ID) return result.Result{}, nil } diff --git a/pkg/storage/batcheval/cmd_resolve_intent_test.go b/pkg/storage/batcheval/cmd_resolve_intent_test.go index c3af46381c40..5a35d03b109b 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_test.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_test.go @@ -21,11 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/abortspan" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" - "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -71,7 +71,7 @@ func (m *mockEvalCtx) GetLimiters() *Limiters { func (m *mockEvalCtx) AbortSpan() *abortspan.AbortSpan { return m.abortSpan } -func (m *mockEvalCtx) GetTxnWaitQueue() *txnwait.Queue { +func (m *mockEvalCtx) GetConcurrencyManager() concurrency.Manager { panic("unimplemented") } func (m *mockEvalCtx) NodeID() roachpb.NodeID { diff --git a/pkg/storage/batcheval/eval_context.go b/pkg/storage/batcheval/eval_context.go index 6b15685b9c96..52b637880d21 100644 --- a/pkg/storage/batcheval/eval_context.go +++ b/pkg/storage/batcheval/eval_context.go @@ -19,10 +19,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/abortspan" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" - "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/limit" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -53,7 +53,7 @@ type EvalContext interface { Clock() *hlc.Clock DB() *client.DB AbortSpan() *abortspan.AbortSpan - GetTxnWaitQueue() *txnwait.Queue + GetConcurrencyManager() concurrency.Manager GetLimiters() *Limiters NodeID() roachpb.NodeID diff --git a/pkg/storage/concurrency/concurrency_control.go b/pkg/storage/concurrency/concurrency_control.go new file mode 100644 index 000000000000..9193806784f8 --- /dev/null +++ b/pkg/storage/concurrency/concurrency_control.go @@ -0,0 +1,355 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Package concurrency provides a concurrency manager structure that +// encapsulates the details of concurrency control and contention handling for +// serializable key-value transactions. +package concurrency + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/spanset" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/uuid" +) + +// Manager is a structure that sequences incoming requests and provides +// isolation between requests that intend to perform conflicting operations. +// During sequencing, conflicts are discovered and any found are resolved +// through a combination of passive queuing and active pushing. Once a request +// has been sequenced, it is free to evaluate without concerns of conflicting +// with other in-flight requests due to the isolation provided by the manager. +// This isolation is guaranteed for the lifetime of the request but terminates +// once the request completes. +// +// Transactions require isolation both within requests and across requests. The +// manager accommodates this by allowing transactional requests to acquire +// locks, which outlive the requests themselves. Locks extend the duration of +// the isolation provided over specific keys to the lifetime of the lock-holder +// transaction itself. They are (typically) only released when the transaction +// commits or aborts. Other requests that find these locks while being sequenced +// wait on them to be released in a queue before proceeding. Because locks are +// checked during sequencing, requests are guaranteed access to all declared +// keys after they have been sequenced. In other words, locks don't need to be +// checked again during evaluation. +// +// However, at the time of writing, not all locks are stored directly under the +// manager's control, so not all locks are discoverable during sequencing. +// Specifically, write intents (replicated, exclusive locks) are stored inline +// in the MVCC keyspace, so they are not detectable until request evaluation +// time. To accommodate this form of lock storage, the manager exposes a +// RetryReqAfterWriteIntentError method, which can be used in conjunction with a +// retry loop around evaluation to integrate external locks with the concurrency +// manager structure. In the future, we intend to pull all locks, including +// those associated with write intents, into the concurrency manager directly +// through a replicated lock table structure. +// +// Fairness is ensured between requests. If any two requests conflict then the +// request that arrived first will be sequenced first. As such, sequencing +// guarantees FIFO semantics. The one exception to this is that a request that +// is part of a transaction which has already acquired a lock does not need to +// wait on that lock during sequencing, and can therefore ignore any queue that +// has formed on the lock. +// +// Internal Components +// +// The concurrency manager is composed of a number of internal synchronization, +// bookkeeping, and queueing structures. Each of these is discussed in more +// detail on their interface definition. The following diagram details how the +// components are tied together: +// +// +---------------------+-------------------------------------------------+ +// | concurrency.Manager | | +// +---------------------+ | +// | | +// +------------+ acquire +--------------+ acquire | +// Sequence() |--->--->--->| latchManager |<---<---<---<---<---<---<--+ | +// +------------+ +--------------+ | | +// | / check locks + wait queues | | +// | v if conflict, enter q & drop latches ^ | +// | +---------------------------------------------------+ | | +// | | [ lockTable ] | | | +// | | [ key1 ] -------------+-----------------+ | ^ | +// | | [ key2 ] / MVCCLock: | lockWaitQueue: |--<------<---+ +// | | [ key3 ]-{ - lock type | +-[a]<-[b]<-[c] | | | | | +// | | [ key4 ] \ - txn meta | | (no latches) |-->-^ | | +// | | [ key5 ] -------------+-|---------------+ | | | +// | | [ ... ] v | | ^ +// | +---------------------------------|-----------------+ | | if lock found during eval +// | | | | | - enter lockWaitQueue +// | | +- may be remote -+--+ | | - drop latches +// | | | | | | - wait for lock holder +// | v v ^ | | +// | | +--------------------------+ | ^ +// | | | txnWaitQueue: | | | +// | | | (located on txn record's | | | +// | v | leaseholder replica) | | | +// | | |--------------------------| | ^ +// | | | [txn1] [txn2] [txn3] ... | | | +// | | +--------------------------+ | | +// | | | | +// | +--> hold latches ---> remain at head of queues -----> evaluate ... +// | | +// +----------+ | +// Finish() | ---> exit wait queues ---> drop latches ----------------------> respond ... +// +----------+ | +// | | +// +-----------------------------------------------------------------------+ +// +// See the comments on individual components for a more detailed look at their +// interface and inner-workings. +// +// At a high-level, requests enter the concurrency manager and immediately +// acquire latches from the latchManager to serialize access to the keys that +// they intend to touch. This latching takes into account the keys being +// accessed, the MVCC timestamp of accesses, and the access method being used +// (read vs. write) to allow for concurrency where possible. This has the effect +// of queuing on conflicting in-flight operations until their completion. +// +// Once latched, the request consults the lockTable to check for any conflicting +// locks owned by other transactions. If any are found, the request enters the +// corresponding lockWaitQueue and its latches are dropped. The head of the +// lockWaitQueue pushes the owner of the lock through a remote RPC that ends up +// in the pushee's txnWaitQueue. This queue exists on the leaseholder replica of +// the range that contains the pushee's transaction record. Other entries in the +// queue wait for the head of the queue, eventually pushing it to detect +// deadlocks. Once the lock is cleared, the head of the queue reacquires latches +// and attempts to proceed while remains at the head of that lockWaitQueue to +// ensure fairness. +// +// Once a request is latched and observes no conflicting locks in the lockTable +// and no conflicting lockWaitQueues that it is not already the head of, the +// request can proceed to evaluate. During evaluation, the request may insert or +// remove locks from the lockTable for its own transaction. This is performed +// transparently by a lockAwareBatch/lockAwareReadWriter. The request may also +// need to consult its own locks to properly interpret the corresponding intents +// in the MVCC keyspace. This is performed transparently by a lockAwareIter. +// +// When the request completes, it exits any lockWaitQueues that it was at the +// head of and releases its latches. However, as long as the request was +// successful, any locks that it inserted into the lockTable remain. +type Manager interface { + requestSequencer + contentionHandler + transactionManager + rangeUpdateListener + metricExporter +} + +// requestSequencer is concerned with the sequencing of concurrent requests. +type requestSequencer interface { + // SequenceReq acquires latches, checks for locks, and queues behind and/or + // pushes other transactions to resolve any conflicts. Once sequenced, the + // request is guaranteed sufficient isolation for the duration of its + // evaluation, until its guard is released. + // NOTE: this last part will not be true until replicated locks are pulled + // into the concurrency manager. + // + // An optional existing request guard can be provided to SequenceReq. This + // allows the request's position in lock wait-queues to be retained across + // sequencing attempts. If provided, the guard should not be holding latches + // already. This typically means that the guard was acquired through a call + // to a contentionHandler method. + // + // If the method returns a non-nil request guard then the caller must ensure + // that the guard is eventually released by passing it to FinishReq. + // + // Alternatively, the concurrency manager may be able to serve the request + // directly, in which case it will return a Response for the request. If it + // does so, it will not return a request guard. + SequenceReq(context.Context, *Guard, Request) (*Guard, Response, *Error) + // FinishReq marks the request as complete, releasing any protection + // the request had against conflicting requests and allowing conflicting + // requests that are blocked on this one to proceed. The guard should not + // be used after being released. + FinishReq(*Guard) +} + +// contentionHandler is concerned with handling contention-related errors. This +// typically involves preparing the request to be queued upon a retry. +type contentionHandler interface { + // HandleWriterIntentError consumes a WriteIntentError by informing the + // concurrency manager about the replicated write intent that was missing + // from its lock table. After doing so, it enqueues the request that hit the + // error in the lock's wait-queue (but does not wait) and releases the + // guard's latches. It returns an updated guard reflecting this change. + // After the method returns, the original guard should no longer be used. + HandleWriterIntentError(context.Context, *Guard, *roachpb.WriteIntentError) *Guard + // HandleTransactionPushError consumes a TransactionPushError by informing + // the concurrency manager about a transaction record that could not be + // pushed. After doing so, it releases the guard's latches. It returns an + // updated guard reflecting this change. After the method returns, the + // original guard should no longer be used. + HandleTransactionPushError(context.Context, *Guard, *roachpb.TransactionPushError) *Guard +} + +// transactionManager is concerned with tracking transactions that have their +// records stored on the manager's range. +type transactionManager interface { + // OnTransactionUpdated informs the concurrency manager that a transaction's + // status was updated by a successful transaction state transition. + OnTransactionUpdated(context.Context, *roachpb.Transaction) + // OnLockAcquired informs the concurrency manager that a transaction has + // acquired a new lock or updated an existing lock that it already held. + OnLockAcquired(context.Context, roachpb.Intent) + // OnLockReleased informs the concurrency manager that a transaction has + // released a lock that it previously held. + OnLockReleased(context.Context, roachpb.Intent) + // GetDependents returns a set of transactions waiting on the specified + // transaction either directly or indirectly. The method is used to perform + // deadlock detection. See txnWaitQueue for more. + GetDependents(uuid.UUID) []uuid.UUID +} + +// rangeUpdateListener is concerned with observing updates to the concurrency +// manager's range. +type rangeUpdateListener interface { + // OnDescriptorUpdated informs the manager that its range's descriptor has + // been updated. + OnDescriptorUpdated(*roachpb.RangeDescriptor) + // OnLeaseUpdated informs the concurrency manager that its range's lease has + // been updated. The argument indicates whether this manager's replica is + // the leaseholder going forward. + OnLeaseUpdated(bool) + // OnSplit informs the concurrency manager that its range has split off a + // new range to its RHS. + OnSplit() + // OnMerge informs the concurrency manager that its range has merged into + // its LHS neighbor. This is not called on the range being merged into. + OnMerge() +} + +// metricExporter is concerned with providing observability into the state of +// the concurrency manager. +type metricExporter interface { + // TODO + // LatchMetrics() + // LockTableMetrics() + // TxnWaitQueueMetrics() +} + +/////////////////////////////////// +// External API Type Definitions // +/////////////////////////////////// + +// Request is the input to Manager.SequenceReq. The struct contains all of the +// information necessary to sequence a KV request and determine which locks and +// other in-flight requests it conflicts with. +type Request struct { + // The (optional) transaction that sent the request. + Txn *roachpb.Transaction + // The timestamp that the request should evaluate at. + Timestamp hlc.Timestamp + // The priority of the request. Only set if Txn is nil. + Priority roachpb.UserPriority + // The consistency level of the request. Only set if Txn is nil. + ReadConsistency roachpb.ReadConsistencyType + // The individual requests in the batch. + Requests []roachpb.RequestUnion + // The maximal set of spans that the request will access. + Spans *spanset.SpanSet +} + +// Guard is returned from Manager.SequenceReq. The guard is passed back in to +// Manager.FinishReq to release the request's resources when it has completed. +type Guard struct { + req Request + lg latchGuard + wqgs []lockWaitQueueGuard +} + +// Response is a slice of responses to requests in a batch. +type Response = []roachpb.ResponseUnion + +// Error is an alias for a roachpb.Error. +type Error = roachpb.Error + +/////////////////////////////////// +// Internal Structure Interfaces // +/////////////////////////////////// + +// latchManager serializes access to keys and key ranges. +// +// See additional documentation in pkg/storage/spanlatch. +type latchManager interface { + // Acquires latches, providing mutual exclusion for conflicting requests. + Acquire(context.Context, Request) (latchGuard, *Error) + // Releases latches. + Release(latchGuard) +} + +// latchGuard is a handle to a set of acquired key latches. +type latchGuard interface{} + +// lockTable holds a collection of locks acquired by in-progress transactions. +// Each lock in the table has a possibly-empty lock wait-queue associated with +// it, where conflicting transactions can queue while waiting for the lock to be +// released. +// +// +---------------------------------------------------+ +// | [ lockTable ] | +// | [ key1 ] -------------+-----------------+ | +// | [ key2 ] / MVCCLock: | lockWaitQueue: | | +// | [ key3 ]-{ - lock type | <-[a]<-[b]<-[c] | | +// | [ key4 ] \ - txn meta | (no latches) | | +// | [ key5 ] -------------+-----------------+ | +// | [ ... ] | +// +---------------------------------------------------+ +// +// TODO(nvanbenschoten): document further. +type lockTable interface { + // AcquireLock informs the lockTable that a new lock was acquired or an + // existing lock was updated. + AcquireLock(roachpb.Intent) + // AcquireLock informs the lockTable that an existing lock was released. + ReleaseLock(roachpb.Intent) + // AddDiscoveredLock informs the lockTable of a lock that was discovered + // during evaluation that the lockTable wasn't previously tracking. + AddDiscoveredLock(Request, roachpb.Intent) lockWaitQueueGuard + // ScanAndEnqueue scans over the spans that the request will access and + // enqueues the request in the lock wait-queue of any conflicting locks + // encountered. + ScanAndEnqueue(Request, *Guard) []lockWaitQueueGuard + // Dequeue removes the guard from its lock wait-queue. The guard is not + // guaranteed to be at the front of hte wait-queue. + Dequeue(lockWaitQueueGuard) + // Clear removes all locks and lock wait-queues from the lockTable. + Clear() +} + +// lockWaitQueueWaiter is concerned with waiting in a lock wait-queues for +// an individual lock held by a conflicting transaction. +// +// TODO(nvanbenschoten): document further. +type lockWaitQueueWaiter interface { + MustWaitOnAny([]lockWaitQueueGuard) bool + WaitOn(context.Context, Request, lockWaitQueueGuard) *Error +} + +// lockWaitQueueGuard is a handle to an entry in a lock wait-queue. +type lockWaitQueueGuard interface{} + +// txnWaitQueue holds a collection of wait-queues for transaction records. +// +// TODO(nvanbenschoten): document further. +// TODO(nvanbenschoten): if we exposed a "queue guard" interface, we could +// make stronger guarantees around cleaning up enqueued txns when there +// are no waiters. +type txnWaitQueue interface { + EnqueueTxn(*roachpb.Transaction) + UpdateTxn(context.Context, *roachpb.Transaction) + GetDependents(uuid.UUID) []uuid.UUID + MaybeWaitForPush(context.Context, *roachpb.PushTxnRequest) (*roachpb.PushTxnResponse, *Error) + MaybeWaitForQuery(context.Context, *roachpb.QueryTxnRequest) *Error + Enable() + Clear(disable bool) +} diff --git a/pkg/storage/concurrency/concurrency_manager.go b/pkg/storage/concurrency/concurrency_manager.go new file mode 100644 index 000000000000..e948d16f8d62 --- /dev/null +++ b/pkg/storage/concurrency/concurrency_manager.go @@ -0,0 +1,357 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package concurrency + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/intentresolver" + "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" + "github.com/cockroachdb/cockroach/pkg/storage/spanset" + "github.com/cockroachdb/cockroach/pkg/storage/storagebase" + "github.com/cockroachdb/cockroach/pkg/storage/txnwait" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/google/btree" +) + +// managerImpl implements the Manager interface. +type managerImpl struct { + // Synchronizes conflicting in-flight requests. + lm latchManager + // Synchronizes conflicting in-progress transactions. + lt lockTable + // Ensures fair FIFO access to keys accessed by conflicting transactions. + lwq lockWaitQueueWaiter + // Waits for transaction completion and detects deadlocks. + twq txnWaitQueue + // The Store and Range that the manager is in charge of. + str Store + rng *roachpb.RangeDescriptor +} + +// Store provides some parts of a Store without incurring a dependency. +type Store interface { + // Components. + Clock() *hlc.Clock + Stopper() *stop.Stopper + DB() *client.DB + IntentResolver() *intentresolver.IntentResolver + // Knobs. + GetTxnWaitKnobs() txnwait.TestingKnobs + // Metrics. + GetTxnWaitMetrics() *txnwait.Metrics + GetSlowLatchGauge() *metric.Gauge +} + +// NewManager creates a new concurrency Manager structure. +func NewManager(store Store) Manager { + m := new(managerImpl) + // TODO(nvanbenschoten): move pkg/storage/spanlatch to a new + // pkg/storage/concurrency/latch package. Make it implement + // the latchManager interface directly, if possible. + m.lm = &latchManagerImpl{ + m: spanlatch.Make(store.Stopper(), store.GetSlowLatchGauge()), + } + m.lt = &lockTableImpl{ + qs: btree.New(16), + } + m.lwq = &lockWaitQueueWaiterImpl{} + // TODO(nvanbenschoten): move pkg/storage/txnwait to a new + // pkg/storage/concurrency/txnwait package. + m.twq = txnwait.NewQueue(store, m) + m.str = store + return m +} + +// SequenceReq implements the Manager interface. +func (m *managerImpl) SequenceReq( + ctx context.Context, prev *Guard, req Request, +) (g *Guard, resp Response, err *Error) { + if !shouldAcquireLatches(req) { + return nil, nil, nil + } + + // Ensure that we release the guard if we return a response or an error. + defer func() { + if g != nil && (resp != nil || err != nil) { + m.FinishReq(g) + g = nil + } + }() + + // Provide the manager with an opportunity to intercept the request. + resp, err = m.maybeInterceptReq(ctx, req) + if resp != nil || err != nil { + return nil, resp, err + } + + g = createOrReuseGuard(prev, req) + for { + // Acquire latches for the request. This synchronizes the request + // with all conflicting in-flight requests. + log.Event(ctx, "acquire latches") + g.lg, err = m.lm.Acquire(ctx, req) + if err != nil { + return g, nil, err + } + + // Some requests don't want the wait on locks. + if !shouldWaitOnConflicts(req) { + return g, nil, nil + } + + // Scan for conflicting locks. + log.Event(ctx, "scan for conflicting locks") + if wqgs := m.lt.ScanAndEnqueue(g.req, g); len(wqgs) > 0 { + g.wqgs = append(g.wqgs, wqgs...) + } + + // Wait on each newly conflicting lock, if applicable. + if m.lwq.MustWaitOnAny(g.wqgs) { + m.lm.Release(g.moveLatchGuard()) + + log.Event(ctx, "waiting in lock wait-queues") + for _, wqg := range g.wqgs { + if err := m.lwq.WaitOn(ctx, g.req, wqg); err != nil { + return g, nil, err + } + } + continue + } + return g, nil, nil + } +} + +// maybeInterceptReq allows the concurrency manager to intercept requests before +// sequencing and evaluation so that it can immediately act on them. This allows +// the concurrency manager to route certain concurrency control-related requests +// into queues and update its optionally internal state based on the requests. +func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Response, *Error) { + switch { + case req.isSingle(roachpb.PushTxn): + // If necessary, wait in the txnWaitQueue for the pushee transaction + // to expire or to move to a finalized state. + t := req.Requests[0].GetPushTxn() + resp, err := m.twq.MaybeWaitForPush(ctx, t) + if err != nil { + return nil, err + } else if resp != nil { + return makeResponse(resp), nil + } + case req.isSingle(roachpb.QueryTxn): + // If necessary, wait in the txnWaitQueue either for transaction + // update or for dependent transactions to change. + t := req.Requests[0].GetQueryTxn() + return nil, m.twq.MaybeWaitForQuery(ctx, t) + default: + // TODO(nvanbenschoten): in the future, use this hook to update the + // lock table to allow contending transactions to proceed. + // for _, arg := range req.Requests { + // switch t := arg.GetInner().(type) { + // case *roachpb.ResolveIntentRequest: + // _ = t + // case *roachpb.ResolveIntentRangeRequest: + // _ = t + // } + // } + } + return nil, nil +} + +// shouldAcquireLatches determines whether the request should acquire latches +// before proceeding to evaluate. Latches are used to synchronize with other +// conflicting requests, based on the Spans collected for the request. Most +// request types will want to acquire latches. +func shouldAcquireLatches(req Request) bool { + switch { + case req.ReadConsistency != roachpb.CONSISTENT: + // Only acquire latches for consistent operations. + return false + case req.isSingle(roachpb.RequestLease): + // Don't acquire latches for lease requests. These are run on replicas + // that do not hold the lease, so acquiring latches wouldn't help + // synchronize with other requests. + return false + } + return true +} + +// shouldWaitOnConflicts determines whether the request should wait on locks and +// wait-queues owned by other transactions before proceeding to evaluate. Most +// requests will want to wait on conflicting transactions to ensure that they +// are sufficiently isolated during their evaluation, but some "isolation aware" +// requests want to proceed to evaluation even in the presence of conflicts +// because they know how to handle them. +func shouldWaitOnConflicts(req Request) bool { + // TODO what's the best way to define this. There are request types that do + // want to wait of locks like (PutRequest and ScanRequest) and then there + // are those that don't want to wait on locks like (QueryIntentRequest, + // RefreshRequest, and ResolveIntentRequest). Should we define a flag for + // this? + should := false + for _, ru := range req.Requests { + arg := ru.GetInner() + if roachpb.IsTransactional(arg) { + switch arg.Method() { + case roachpb.HeartbeatTxn: + case roachpb.Refresh: + case roachpb.RefreshRange: + default: + should = true + } + } + } + return should +} + +// FinishReq implements the Manager interface. +func (m *managerImpl) FinishReq(g *Guard) { + for _, wqg := range g.moveWaitQueueGuards() { + m.lt.Dequeue(wqg) + } + if lg := g.moveLatchGuard(); lg != nil { + m.lm.Release(lg) + } +} + +// HandleWriterIntentError implements the Manager interface. +func (m *managerImpl) HandleWriterIntentError( + ctx context.Context, g *Guard, t *roachpb.WriteIntentError, +) *Guard { + // Create or enter a txnWaitQueue entry per intent. + for _, intent := range t.Intents { + g.wqgs = append(g.wqgs, m.lt.AddDiscoveredLock(g.req, intent)) + } + + // Release the Guard's latches but continue to remain in lock wait-queues by + // not releasing lockWaitQueueGuards. We expect the caller of this method to + // then re-sequence the Request by calling SequenceReq with the un-latched + // Guard. This is analogous to iterating through the loop in SequenceReq. + m.lm.Release(g.moveLatchGuard()) + return g +} + +// HandleTransactionPushError implements the Manager interface. +func (m *managerImpl) HandleTransactionPushError( + ctx context.Context, g *Guard, t *roachpb.TransactionPushError, +) *Guard { + m.twq.EnqueueTxn(&t.PusheeTxn) + + // Release the Guard's latches but continue to remain in lock wait-queues by + // not releasing lockWaitQueueGuards. We expect the caller of this method to + // then re-sequence the Request by calling SequenceReq with the un-latched + // Guard. This is analogous to iterating through the loop in SequenceReq. + m.lm.Release(g.moveLatchGuard()) + return g +} + +// OnTransactionUpdated implements the transactionManager interface. +func (m *managerImpl) OnTransactionUpdated(ctx context.Context, txn *roachpb.Transaction) { + m.twq.UpdateTxn(ctx, txn) +} + +// OnLockAcquired implements the transactionManager interface. +func (m *managerImpl) OnLockAcquired(ctx context.Context, in roachpb.Intent) { + m.lt.AcquireLock(in) +} + +// OnLockReleased implements the transactionManager interface. +func (m *managerImpl) OnLockReleased(ctx context.Context, in roachpb.Intent) { + m.lt.ReleaseLock(in) +} + +// GetDependents implements the transactionManager interface. +func (m *managerImpl) GetDependents(txnID uuid.UUID) []uuid.UUID { + return m.twq.GetDependents(txnID) +} + +// OnDescriptorUpdated implements the replicaStateContainer interface. +func (m *managerImpl) OnDescriptorUpdated(desc *roachpb.RangeDescriptor) { + m.rng = desc +} + +// OnLeaseUpdated implements the replicaStateContainer interface. +func (m *managerImpl) OnLeaseUpdated(iAmTheLeaseHolder bool) { + if iAmTheLeaseHolder { + m.twq.Enable() + } else { + m.lt.Clear() + m.twq.Clear(true /* disable */) + } +} + +// OnSplit implements the replicaStateContainer interface. +func (m *managerImpl) OnSplit() { + m.lt.Clear() + m.twq.Clear(false /* disable */) +} + +// OnMerge implements the replicaStateContainer interface. +func (m *managerImpl) OnMerge() { + m.lt.Clear() + m.twq.Clear(true /* disable */) +} + +// ContainsKey implements the txnwait.ReplicaInterface interface. +func (m *managerImpl) ContainsKey(key roachpb.Key) bool { + return storagebase.ContainsKey(m.rng, key) +} + +func (r *Request) isReadOnly() bool { + return !r.Spans.Contains(spanset.SpanReadWrite) +} + +func (r *Request) isSingle(m roachpb.Method) bool { + if len(r.Requests) != 1 { + return false + } + return r.Requests[0].GetInner().Method() == m +} + +func createOrReuseGuard(g *Guard, req Request) *Guard { + if g == nil { + // TODO(nvanbenschoten): Pool these guard objects. + g = &Guard{req: req} + } else { + g.assertNoLatches() + } + return g +} + +func (g *Guard) assertNoLatches() { + if g.lg != nil { + panic("unexpected latches held") + } +} + +func (g *Guard) moveLatchGuard() latchGuard { + lg := g.lg + g.lg = nil + return lg +} + +func (g *Guard) moveWaitQueueGuards() []lockWaitQueueGuard { + wqgs := g.wqgs + g.wqgs = nil + return wqgs +} + +func makeResponse(r roachpb.Response) Response { + ru := make(Response, 1) + ru[0].MustSetInner(r) + return ru +} diff --git a/pkg/storage/concurrency/latch_manager.go b/pkg/storage/concurrency/latch_manager.go new file mode 100644 index 000000000000..c65d0a0a7cec --- /dev/null +++ b/pkg/storage/concurrency/latch_manager.go @@ -0,0 +1,35 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package concurrency + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" +) + +// latchManagerImpl implements the latchManager interface. +type latchManagerImpl struct { + m spanlatch.Manager +} + +func (m *latchManagerImpl) Acquire(ctx context.Context, req Request) (latchGuard, *Error) { + lg, err := m.m.Acquire(ctx, req.Spans) + if err != nil { + return nil, roachpb.NewError(err) + } + return lg, nil +} + +func (m *latchManagerImpl) Release(lg latchGuard) { + m.m.Release(lg.(*spanlatch.Guard)) +} diff --git a/pkg/storage/concurrency/lock_table.go b/pkg/storage/concurrency/lock_table.go new file mode 100644 index 000000000000..d9fc5069e8a9 --- /dev/null +++ b/pkg/storage/concurrency/lock_table.go @@ -0,0 +1,335 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package concurrency + +import ( + "container/list" + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" + "github.com/cockroachdb/cockroach/pkg/storage/spanset" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/google/btree" +) + +// lockTableImpl implements the lockTable interface. +// +// WIP: this is not a real implementation. It's just to get things working. +// Sumeer is working on a better version that can actually detect deadlocks. +type lockTableImpl struct { + mu syncutil.Mutex + qs *btree.BTree + tmp1, tmp2 perKeyWaitQueue +} + +type perKeyWaitQueue struct { + in roachpb.Intent + held bool + ll list.List // List<*perKeyWaitQueueElem> +} + +type perKeyWaitQueueElem struct { + wq *perKeyWaitQueue + req Request + ref uint16 + done chan struct{} + closed bool + waited bool + + // WIP: remove the need for this + elem *list.Element +} + +// Less implements the btree.Item interface. +func (a *perKeyWaitQueue) Less(b btree.Item) bool { + return a.in.Key.Compare(b.(*perKeyWaitQueue).in.Key) < 0 +} + +func (lt *lockTableImpl) AcquireLock(in roachpb.Intent) { + if in.EndKey != nil { + panic("must acquire point locks") + } + + lt.mu.Lock() + defer lt.mu.Unlock() + lt.tmp1.in = in + wqI := lt.qs.Get(<.tmp1) + var wq *perKeyWaitQueue + if wqI == nil { + // TODO(nvanbenschoten): memory recycling. + wq = new(perKeyWaitQueue) + wq.in = in + wq.held = true + wq.ll.Init() + lt.qs.ReplaceOrInsert(wq) + } else { + wq = wqI.(*perKeyWaitQueue) + wq.in.Txn = in.Txn + wq.held = true + } +} + +func (lt *lockTableImpl) ReleaseLock(in roachpb.Intent) { + lt.mu.Lock() + defer lt.mu.Unlock() + lt.tmp1.in = in + wqI := lt.qs.Get(<.tmp1) + if wqI == nil { + return + } + wq := wqI.(*perKeyWaitQueue) + if !wq.held || wq.in.Txn.ID != in.Txn.ID { + return + } + if wq.ll.Len() == 0 { + lt.qs.Delete(wq) + return + } + wq.in.Txn = enginepb.TxnMeta{} + wq.held = false + front := wq.ll.Front().Value.(*perKeyWaitQueueElem) + if !front.closed { + close(front.done) + front.closed = true + } +} + +func (lt *lockTableImpl) AddDiscoveredLock(req Request, in roachpb.Intent) lockWaitQueueGuard { + lt.mu.Lock() + defer lt.mu.Unlock() + lt.tmp1.in = in + wqI := lt.qs.Get(<.tmp1) + var wq *perKeyWaitQueue + if wqI == nil { + // TODO(nvanbenschoten): memory recycling. + wq = new(perKeyWaitQueue) + wq.in = in + wq.held = true + wq.ll.Init() + lt.qs.ReplaceOrInsert(wq) + } else { + wq = wqI.(*perKeyWaitQueue) + wq.in.Txn = in.Txn + wq.held = true + } + + if req.isReadOnly() { + for e := wq.ll.Front(); e != nil; e = e.Next() { + elem := e.Value.(*perKeyWaitQueueElem) + if elem.req.isReadOnly() { + elem.ref++ + return elem + } + } + } + + // TODO(nvanbenschoten): memory recycling. + elem := &perKeyWaitQueueElem{ + wq: wq, + req: req, + ref: 1, + done: make(chan struct{}), + } + elem.elem = wq.ll.PushBack(elem) + return elem +} + +func (lt *lockTableImpl) ScanAndEnqueue(req Request, g *Guard) []lockWaitQueueGuard { + var elems []lockWaitQueueGuard + readOnly := req.isReadOnly() + + lt.mu.Lock() + defer lt.mu.Unlock() + for s := spanset.SpanScope(0); s < spanset.NumSpanScope; s++ { + for a := spanset.SpanAccess(0); a < spanset.NumSpanAccess; a++ { + ss := req.Spans.GetSpans(a, s) + for _, span := range ss { + shouldEnqueue := func(lock *enginepb.TxnMeta, held bool) bool { + if req.Txn != nil && req.Txn.ID == lock.ID { + return false + } + switch a { + case spanset.SpanReadOnly: + return held && !span.Timestamp.Less(lock.WriteTimestamp) + case spanset.SpanReadWrite: + return true + default: + panic("unknown access") + } + } + maybeEnqueue := func(i btree.Item) bool { + wq := i.(*perKeyWaitQueue) + if !shouldEnqueue(&wq.in.Txn, wq.held) { + return true + } + // WIP: this is awful. + for _, ex := range g.wqgs { + if ex.(*perKeyWaitQueueElem).wq == wq { + return true + } + } + // WIP: this is more awful. + if readOnly { + for e := wq.ll.Front(); e != nil; e = e.Next() { + elem := e.Value.(*perKeyWaitQueueElem) + if elem.req.isReadOnly() { + elem.ref++ + elems = append(elems, elem) + return true + } + } + } + elem := &perKeyWaitQueueElem{ + wq: wq, + req: req, + ref: 1, + done: make(chan struct{}), + } + elem.elem = wq.ll.PushBack(elem) + elems = append(elems, elem) + return true + } + + lt.tmp1.in.Key = span.Key + if span.EndKey == nil { + if i := lt.qs.Get(<.tmp1); i != nil { + maybeEnqueue(i) + } + } else { + lt.tmp2.in.Key = span.EndKey + lt.qs.AscendRange(<.tmp1, <.tmp2, maybeEnqueue) + } + } + } + } + return elems +} + +func (lt *lockTableImpl) Dequeue(wqg lockWaitQueueGuard) { + lt.mu.Lock() + defer lt.mu.Unlock() + elem := wqg.(*perKeyWaitQueueElem) + elem.ref-- + if elem.ref > 0 { + return + } + wq := elem.wq + front := wq.ll.Front() == elem.elem + wq.ll.Remove(elem.elem) + if !wq.held { + if wq.ll.Len() == 0 { + lt.qs.Delete(wq) + } else if front { + newFront := wq.ll.Front().Value.(*perKeyWaitQueueElem) + if !newFront.closed { + close(newFront.done) + newFront.closed = true + } + } + } +} + +func (lt *lockTableImpl) Clear() { + lt.mu.Lock() + defer lt.mu.Unlock() + lt.qs.Ascend(func(i btree.Item) bool { + wq := i.(*perKeyWaitQueue) + for e := wq.ll.Front(); e != nil; e = e.Next() { + elem := e.Value.(*perKeyWaitQueueElem) + if !elem.closed { + close(elem.done) + elem.closed = true + } + } + return true + }) + lt.qs.Clear(false) +} + +// lockWaitQueueWaiterImpl implements the lockWaitQueueWaiter interface. +type lockWaitQueueWaiterImpl struct{} + +func (wq *lockWaitQueueWaiterImpl) MustWaitOnAny(wqgs []lockWaitQueueGuard) bool { + // WIP: iterate in reverse. + for _, wqg := range wqgs { + if !wqg.(*perKeyWaitQueueElem).waited { + return true + } + } + return false +} + +func (wq *lockWaitQueueWaiterImpl) WaitOn( + ctx context.Context, req Request, wqg lockWaitQueueGuard, +) *Error { + elem := wqg.(*perKeyWaitQueueElem) + if elem.waited { + return nil + } + select { + case <-elem.done: + elem.waited = true + return nil + case <-ctx.Done(): + return roachpb.NewError(ctx.Err()) + } +} + +// // contentionQueuelockWaitQueueWaiterImpl implements the lockWaitQueueWaiter interface. +// type contentionQueuelockWaitQueueWaiterImpl struct { +// c *hlc.Clock +// ir *intentresolver.IntentResolver +// } + +// func (wq *contentionQueuelockWaitQueueWaiterImpl) waitOn( +// ctx context.Context, req Request, wqg lockWaitQueueGuard, +// ) *Error { +// pkwq := wqg.(perKeyWaitQueue) + +// // WIP: Just to get it working. +// wiErr := roachpb.NewError(&roachpb.WriteIntentError{Intents: []roachpb.Intent{pkwq.in}}) + +// h := roachpb.Header{ +// Timestamp: req.Timestamp, +// UserPriority: req.Priority, +// } +// if req.Txn != nil { +// // We must push at least to req.Timestamp, but in fact we want to +// // go all the way up to a timestamp which was taken off the HLC +// // after our operation started. This allows us to not have to +// // restart for uncertainty as we come back and read. +// h.Timestamp = wq.c.Now() +// // We are going to hand the header (and thus the transaction proto) +// // to the RPC framework, after which it must not be changed (since +// // that could race). Since the subsequent execution of the original +// // request might mutate the transaction, make a copy here. +// // +// // See #9130. +// h.Txn = req.Txn.Clone() +// } + +// var pushType roachpb.PushTxnType +// if req.Spans.Contains(spanset.SpanReadWrite) { +// pushType = roachpb.PUSH_ABORT +// } else { +// pushType = roachpb.PUSH_TIMESTAMP +// } + +// var err *Error +// if _, err = wq.ir.ProcessWriteIntentError(ctx, wiErr, h, pushType); err != nil { +// // Do not propagate ambiguous results; assume success and retry original op. +// if _, ok := err.GetDetail().(*roachpb.AmbiguousResultError); !ok { +// return err +// } +// } +// return nil +// } diff --git a/pkg/storage/helpers_test.go b/pkg/storage/helpers_test.go index cb3010f58fd5..a1b282f02ecd 100644 --- a/pkg/storage/helpers_test.go +++ b/pkg/storage/helpers_test.go @@ -449,10 +449,6 @@ func (r *Replica) IsQuiescent() bool { return r.mu.quiescent } -func (r *Replica) IsTxnWaitQueueEnabled() bool { - return r.txnWaitQueue.IsEnabled() -} - // GetQueueLastProcessed returns the last processed timestamp for the // specified queue, or the zero timestamp if not available. func (r *Replica) GetQueueLastProcessed(ctx context.Context, queue string) (hlc.Timestamp, error) { diff --git a/pkg/storage/intentresolver/intent_resolver.go b/pkg/storage/intentresolver/intent_resolver.go index acd9cda729ee..8819fa923b57 100644 --- a/pkg/storage/intentresolver/intent_resolver.go +++ b/pkg/storage/intentresolver/intent_resolver.go @@ -883,9 +883,7 @@ func (ir *IntentResolver) ResolveIntents( } var resolveReqs []resolveReq var resolveRangeReqs []roachpb.Request - for i := range intents { - intent := intents[i] // avoids a race in `i, intent := range ...` - + for _, intent := range intents { if len(intent.EndKey) == 0 { resolveReqs = append(resolveReqs, resolveReq{ diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 1023d5b3bba8..061a981bf1fd 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -30,16 +30,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/batcheval" "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/rangefeed" - "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" - "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/split" "github.com/cockroachdb/cockroach/pkg/storage/stateloader" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" - "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -176,9 +174,8 @@ type Replica struct { // TODO(tschottdorf): Duplicates r.mu.state.desc.RangeID; revisit that. RangeID roachpb.RangeID // Only set by the constructor - store *Store - abortSpan *abortspan.AbortSpan // Avoids anomalous reads after abort - txnWaitQueue *txnwait.Queue // Queues push txn attempts by txn ID + store *Store + abortSpan *abortspan.AbortSpan // Avoids anomalous reads after abort // leaseholderStats tracks all incoming BatchRequests to the replica and which // localities they come from in order to aid in lease rebalancing decisions. @@ -224,11 +221,10 @@ type Replica struct { // Contains the lease history when enabled. leaseHistory *leaseHistory - // Enforces at most one command is running per key(s) within each span - // scope. The globally-scoped component tracks user writes (i.e. all - // keys for which keys.Addr is the identity), the locally-scoped component - // the rest (e.g. RangeDescriptor, transaction record, Lease, ...). - latchMgr spanlatch.Manager + // concMgr sequences incoming requests and provides isoaltion between + // requests that intend to perform conflicting operations. It is the + // centerpiece of transaction contention handling. + concMgr concurrency.Manager mu struct { // Protects all fields in the mu struct. @@ -690,9 +686,9 @@ func (r *Replica) GetLimiters() *batcheval.Limiters { return &r.store.limiters } -// GetTxnWaitQueue returns the Replica's txnwait.Queue. -func (r *Replica) GetTxnWaitQueue() *txnwait.Queue { - return r.txnWaitQueue +// GetConcurrencyManager returns the Replica's concurrency.Manager. +func (r *Replica) GetConcurrencyManager() concurrency.Manager { + return r.concMgr } // GetTerm returns the term of the given index in the raft log. @@ -953,14 +949,14 @@ func (r *Replica) assertStateLocked(ctx context.Context, reader engine.Reader) { // able to serve traffic or that the request is not compatible with the state of // the Range. // -// The method accepts a spanlatch.Guard and a LeaseStatus parameter. These are +// The method accepts a concurrency.Guard and a LeaseStatus parameter. These are // used to indicate whether the caller has acquired latches and checked the // Range lease. The method will only check for a pending merge if both of these // conditions are true. If either lg == nil or st == nil then the method will // not check for a pending merge. Callers might be ok with this if they know // that they will end up checking for a pending merge at some later time. func (r *Replica) checkExecutionCanProceed( - ba *roachpb.BatchRequest, lg *spanlatch.Guard, st *storagepb.LeaseStatus, + ba *roachpb.BatchRequest, g *concurrency.Guard, st *storagepb.LeaseStatus, ) error { rSpan, err := keys.Range(ba.Requests) if err != nil { @@ -975,7 +971,7 @@ func (r *Replica) checkExecutionCanProceed( return err } else if err := r.checkTSAboveGCThresholdRLocked(ba.Timestamp); err != nil { return err - } else if lg != nil && st != nil { + } else if g != nil && st != nil { // Only check for a pending merge if latches are held and the Range // lease is held by this Replica. Without both of these conditions, // checkForPendingMergeRLocked could return false negatives. @@ -1151,7 +1147,7 @@ func (r *Replica) isNewerThanSplitRLocked(split *roachpb.SplitTrigger) bool { // command processing. type endCmds struct { repl *Replica - lg *spanlatch.Guard + g *concurrency.Guard } // move moves the endCmds into the return value, clearing and making @@ -1179,67 +1175,16 @@ func (ec *endCmds) done( // Update the timestamp cache if the request is not being re-evaluated. Each // request is considered in turn; only those marked as affecting the cache are // processed. Inconsistent reads are excluded. - if ba.ReadConsistency == roachpb.CONSISTENT { - ec.repl.updateTimestampCache(ctx, ba, br, pErr) - } + ec.repl.updateTimestampCache(ctx, ba, br, pErr) // Release the latches acquired by the request back to the spanlatch - // manager. Must be done AFTER the timestamp cache is updated. - if ec.lg != nil { - ec.repl.latchMgr.Release(ec.lg) + // manager. Must be done AFTER the timestamp cache is updated. Only set + // during when the proposal has assumed responsibility for releasing latches. + if ec.g != nil { + ec.repl.concMgr.FinishReq(ec.g) } } -// beginCmds waits for any in-flight, conflicting commands to complete. More -// specifically, beginCmds acquires latches for the request based on keys -// affected by the batched commands. This gates subsequent commands with -// overlapping keys or key ranges. It returns a cleanup function to be called -// when the commands are done and can release their latches. -func (r *Replica) beginCmds( - ctx context.Context, ba *roachpb.BatchRequest, spans *spanset.SpanSet, -) (*spanlatch.Guard, error) { - // Only acquire latches for consistent operations. - if ba.ReadConsistency != roachpb.CONSISTENT { - log.Event(ctx, "operation accepts inconsistent results") - return nil, nil - } - - // Don't acquire latches for lease requests. These are run on replicas that - // do not hold the lease, so acquiring latches wouldn't help synchronize - // with other requests. - if ba.IsLeaseRequest() { - return nil, nil - } - - var beforeLatch time.Time - if log.ExpensiveLogEnabled(ctx, 2) { - beforeLatch = timeutil.Now() - } - - // Acquire latches for all the request's declared spans to ensure - // protected access and to avoid interacting requests from operating at - // the same time. The latches will be held for the duration of request. - log.Event(ctx, "acquire latches") - lg, err := r.latchMgr.Acquire(ctx, spans) - if err != nil { - return nil, err - } - - if !beforeLatch.IsZero() { - dur := timeutil.Since(beforeLatch) - log.VEventf(ctx, 2, "waited %s to acquire latches", dur) - } - - if filter := r.store.cfg.TestingKnobs.TestingLatchFilter; filter != nil { - if pErr := filter(*ba); pErr != nil { - r.latchMgr.Release(lg) - return nil, pErr.GoError() - } - } - - return lg, nil -} - // maybeWatchForMerge checks whether a merge of this replica into its left // neighbor is in its critical phase and, if so, arranges to block all requests // until the merge completes. diff --git a/pkg/storage/replica_eval_context_span.go b/pkg/storage/replica_eval_context_span.go index d91e7f3cc075..33b77b0fcbee 100644 --- a/pkg/storage/replica_eval_context_span.go +++ b/pkg/storage/replica_eval_context_span.go @@ -20,11 +20,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/abortspan" "github.com/cockroachdb/cockroach/pkg/storage/batcheval" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" - "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -74,9 +74,9 @@ func (rec *SpanSetReplicaEvalContext) DB() *client.DB { return rec.i.DB() } -// GetTxnWaitQueue returns the txnwait.Queue. -func (rec *SpanSetReplicaEvalContext) GetTxnWaitQueue() *txnwait.Queue { - return rec.i.GetTxnWaitQueue() +// GetConcurrencyManager returns the concurrency.Manager. +func (rec *SpanSetReplicaEvalContext) GetConcurrencyManager() concurrency.Manager { + return rec.i.GetConcurrencyManager() } // NodeID returns the NodeID. diff --git a/pkg/storage/replica_init.go b/pkg/storage/replica_init.go index de5f069422da..f4b11169b395 100644 --- a/pkg/storage/replica_init.go +++ b/pkg/storage/replica_init.go @@ -18,11 +18,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/storage/abortspan" - "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/split" "github.com/cockroachdb/cockroach/pkg/storage/stateloader" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" - "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -42,7 +41,7 @@ func newReplica(rangeID roachpb.RangeID, store *Store) *Replica { RangeID: rangeID, store: store, abortSpan: abortspan.New(rangeID), - txnWaitQueue: txnwait.NewQueue(store), + concMgr: concurrency.NewManager(store), } r.mu.pendingLeaseRequest = makePendingLeaseRequest(r) r.mu.stateLoader = stateloader.Make(rangeID) @@ -95,7 +94,6 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked( return errors.Errorf("replicaID must be 0 when creating an initialized replica") } - r.latchMgr = spanlatch.Make(r.store.stopper, r.store.metrics.SlowLatchRequests) r.mu.proposals = map[storagebase.CmdIDKey]*ProposalData{} r.mu.checksums = map[uuid.UUID]ReplicaChecksum{} // Clear the internal raft group in case we're being reset. Since we're @@ -147,6 +145,7 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked( } r.rangeStr.store(replicaID, r.mu.state.Desc) r.connectionClass.set(rpc.ConnectionClassForKey(desc.StartKey)) + r.concMgr.OnDescriptorUpdated(r.mu.state.Desc) if r.mu.replicaID == 0 { if err := r.setReplicaIDRaftMuLockedMuLocked(ctx, replicaID); err != nil { return err @@ -299,5 +298,6 @@ func (r *Replica) setDesc(ctx context.Context, desc *roachpb.RangeDescriptor) { r.rangeStr.store(r.mu.replicaID, desc) r.connectionClass.set(rpc.ConnectionClassForKey(desc.StartKey)) + r.concMgr.OnDescriptorUpdated(desc) r.mu.state.Desc = desc } diff --git a/pkg/storage/replica_metrics.go b/pkg/storage/replica_metrics.go index 21d21452d61d..4f5740f18e7a 100644 --- a/pkg/storage/replica_metrics.go +++ b/pkg/storage/replica_metrics.go @@ -64,7 +64,8 @@ func (r *Replica) Metrics( _, ticking := r.store.unquiescedReplicas.m[r.RangeID] r.store.unquiescedReplicas.Unlock() - latchInfoGlobal, latchInfoLocal := r.latchMgr.Info() + // TODO WIP + // latchInfoGlobal, latchInfoLocal := r.latchMgr.Info() return calcReplicaMetrics( ctx, @@ -79,8 +80,8 @@ func (r *Replica) Metrics( r.store.StoreID(), quiescent, ticking, - latchInfoLocal, - latchInfoGlobal, + storagepb.LatchManagerInfo{}, + storagepb.LatchManagerInfo{}, raftLogSize, ) } diff --git a/pkg/storage/replica_proposal.go b/pkg/storage/replica_proposal.go index 6d8d82352454..6eba6c15f2ff 100644 --- a/pkg/storage/replica_proposal.go +++ b/pkg/storage/replica_proposal.go @@ -87,7 +87,7 @@ type ProposalData struct { tmpFooter storagepb.RaftCommandFooter // ec.done is called after command application to update the timestamp - // cache and release latches. + // cache and optionally release latches. ec endCmds // applied is set when the a command finishes application. It is used to @@ -413,12 +413,6 @@ func (r *Replica) leasePostApply(ctx context.Context, newLease roachpb.Lease, pe } } - if leaseChangingHands && !iAmTheLeaseHolder { - // Also clear and disable the push transaction queue. Any waiters - // must be redirected to the new lease holder. - r.txnWaitQueue.Clear(true /* disable */) - } - // If we're the current raft leader, may want to transfer the leadership to // the new leaseholder. Note that this condition is also checked periodically // when ticking the replica. @@ -439,6 +433,9 @@ func (r *Replica) leasePostApply(ctx context.Context, newLease roachpb.Lease, pe } } + // Inform the concurrency manager that the lease holder has been updated. + r.concMgr.OnLeaseUpdated(iAmTheLeaseHolder) + // Potentially re-gossip if the range contains system data (e.g. system // config or node liveness). We need to perform this gossip at startup as // soon as possible. Trying to minimize how often we gossip is a fool's @@ -453,8 +450,6 @@ func (r *Replica) leasePostApply(ctx context.Context, newLease roachpb.Lease, pe if err := r.MaybeGossipNodeLiveness(ctx, keys.NodeLivenessSpan); err != nil { log.Error(ctx, err) } - // Make sure the push transaction queue is enabled. - r.txnWaitQueue.Enable() // Emit an MLAI on the leaseholder replica, as follower will be looking // for one and if we went on to quiesce, they wouldn't necessarily get @@ -610,18 +605,22 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re } if lResult.UpdatedIntents != nil { - // TODO(nvanbenschoten): handle UpdatedIntents. + for _, intent := range lResult.UpdatedIntents { + r.concMgr.OnLockAcquired(ctx, intent) + } lResult.UpdatedIntents = nil } if lResult.ResolvedIntents != nil { - // TODO(nvanbenschoten): handle ResolvedIntents. + for _, intent := range lResult.ResolvedIntents { + r.concMgr.OnLockReleased(ctx, intent) + } lResult.ResolvedIntents = nil } if lResult.UpdatedTxns != nil { for _, txn := range lResult.UpdatedTxns { - r.txnWaitQueue.UpdateTxn(ctx, txn) + r.concMgr.OnTransactionUpdated(ctx, txn) } lResult.UpdatedTxns = nil } diff --git a/pkg/storage/replica_raft.go b/pkg/storage/replica_raft.go index 0260737f493a..6d006970d90f 100644 --- a/pkg/storage/replica_raft.go +++ b/pkg/storage/replica_raft.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/apply" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/stateloader" @@ -68,21 +69,14 @@ func (r *Replica) evalAndPropose( lease *roachpb.Lease, ba *roachpb.BatchRequest, spans *spanset.SpanSet, - ec endCmds, -) (_ chan proposalResult, _ func(), _ int64, pErr *roachpb.Error) { + g *concurrency.Guard, +) (_ chan proposalResult, _ func(), _ int64, async bool, pErr *roachpb.Error) { idKey := makeIDKey() proposal, pErr := r.requestToProposal(ctx, idKey, ba, spans) log.Event(proposal.ctx, "evaluated request") - // Attach the endCmds to the proposal. This moves responsibility of - // releasing latches to "below Raft" machinery. However, we make sure - // we clean up this resource if the proposal doesn't make it to Raft. - proposal.ec = ec.move() - defer func() { - if pErr != nil { - proposal.ec.done(ctx, ba, nil /* br */, pErr) - } - }() + // Attach the endCmds to the proposal. + proposal.ec = endCmds{repl: r} // Pull out proposal channel to return. proposal.doneCh may be set to // nil if it is signaled in this function. @@ -106,9 +100,13 @@ func (r *Replica) evalAndPropose( EndTxns: endTxns, } proposal.finishApplication(ctx, pr) - return proposalCh, func() {}, 0, nil + return proposalCh, func() {}, 0, async, nil } + // Assume responsibility for releasing the concurrency guard. + proposal.ec.g = g + async = true + // If the request requested that Raft consensus be performed asynchronously, // return a proposal result immediately on the proposal's done channel. // The channel's capacity will be large enough to accommodate this. @@ -117,7 +115,7 @@ func (r *Replica) evalAndPropose( // Disallow async consensus for commands with EndTxnIntents because // any !Always EndTxnIntent can't be cleaned up until after the // command succeeds. - return nil, nil, 0, roachpb.NewErrorf("cannot perform consensus asynchronously for "+ + return nil, nil, 0, false, roachpb.NewErrorf("cannot perform consensus asynchronously for "+ "proposal with EndTxnIntents=%v; %v", ets, ba) } @@ -154,14 +152,14 @@ func (r *Replica) evalAndPropose( // behavior. quotaSize := uint64(proposal.command.Size()) if maxSize := uint64(MaxCommandSize.Get(&r.store.cfg.Settings.SV)); quotaSize > maxSize { - return nil, nil, 0, roachpb.NewError(errors.Errorf( + return nil, nil, 0, false, roachpb.NewError(errors.Errorf( "command is too large: %d bytes (max: %d)", quotaSize, maxSize, )) } var err error proposal.quotaAlloc, err = r.maybeAcquireProposalQuota(ctx, quotaSize) if err != nil { - return nil, nil, 0, roachpb.NewError(err) + return nil, nil, 0, false, roachpb.NewError(err) } // Make sure we clean up the proposal if we fail to insert it into the // proposal buffer successfully. This ensures that we always release any @@ -180,13 +178,13 @@ func (r *Replica) evalAndPropose( Req: *ba, } if pErr := filter(filterArgs); pErr != nil { - return nil, nil, 0, pErr + return nil, nil, 0, false, pErr } } maxLeaseIndex, pErr := r.propose(ctx, proposal) if pErr != nil { - return nil, nil, 0, pErr + return nil, nil, 0, false, pErr } // Abandoning a proposal unbinds its context so that the proposal's client // is free to terminate execution. However, it does nothing to try to @@ -208,7 +206,7 @@ func (r *Replica) evalAndPropose( // We'd need to make sure the span is finished eventually. proposal.ctx = r.AnnotateCtx(context.TODO()) } - return proposalCh, abandon, maxLeaseIndex, nil + return proposalCh, abandon, maxLeaseIndex, async, nil } // propose encodes a command, starts tracking it, and proposes it to raft. The diff --git a/pkg/storage/replica_read.go b/pkg/storage/replica_read.go index 6b8d4b037fbb..0c75d6963f3c 100644 --- a/pkg/storage/replica_read.go +++ b/pkg/storage/replica_read.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" @@ -29,22 +29,15 @@ import ( // iterator to evaluate the batch and then updates the timestamp cache to // reflect the key spans that it read. func (r *Replica) executeReadOnlyBatch( - ctx context.Context, ba *roachpb.BatchRequest, spans *spanset.SpanSet, lg *spanlatch.Guard, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { - // Guarantee we release the provided latches. This is wrapped to delay pErr - // evaluation to its value when returning. - ec := endCmds{repl: r, lg: lg} - defer func() { - ec.done(ctx, ba, br, pErr) - }() - + ctx context.Context, ba *roachpb.BatchRequest, spans *spanset.SpanSet, g *concurrency.Guard, +) (br *roachpb.BatchResponse, _ bool, pErr *roachpb.Error) { // If the read is not inconsistent, the read requires the range lease or // permission to serve via follower reads. var status storagepb.LeaseStatus if ba.ReadConsistency.RequiresReadLease() { if status, pErr = r.redirectOnOrAcquireLease(ctx); pErr != nil { if nErr := r.canServeFollowerRead(ctx, ba, pErr); nErr != nil { - return nil, nErr + return nil, false, nErr } r.store.metrics.FollowerReadsCount.Inc(1) } @@ -56,8 +49,8 @@ func (r *Replica) executeReadOnlyBatch( defer r.readOnlyCmdMu.RUnlock() // Verify that the batch can be executed. - if err := r.checkExecutionCanProceed(ba, ec.lg, &status); err != nil { - return nil, roachpb.NewError(err) + if err := r.checkExecutionCanProceed(ba, g, &status); err != nil { + return nil, false, roachpb.NewError(err) } // Evaluate read-only batch command. @@ -76,13 +69,14 @@ func (r *Replica) executeReadOnlyBatch( if err := r.handleReadOnlyLocalEvalResult(ctx, ba, result.Local); err != nil { pErr = roachpb.NewError(err) } + r.updateTimestampCache(ctx, ba, br, pErr) if pErr != nil { log.VErrEvent(ctx, 3, pErr.String()) } else { log.Event(ctx, "read completed") } - return br, pErr + return br, false, pErr } func (r *Replica) handleReadOnlyLocalEvalResult( @@ -105,6 +99,13 @@ func (r *Replica) handleReadOnlyLocalEvalResult( lResult.MaybeWatchForMerge = false } + if lResult.UpdatedIntents != nil { + for _, intent := range lResult.UpdatedIntents { + r.concMgr.OnLockAcquired(ctx, intent) + } + lResult.UpdatedIntents = nil + } + if intents := lResult.DetachEncounteredIntents(); len(intents) > 0 { log.Eventf(ctx, "submitting %d intents to asynchronous processing", len(intents)) // We only allow synchronous intent resolution for consistent requests. diff --git a/pkg/storage/replica_send.go b/pkg/storage/replica_send.go index cae0bd8f8ece..e9f4e315aff0 100644 --- a/pkg/storage/replica_send.go +++ b/pkg/storage/replica_send.go @@ -16,8 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/batcheval" - "github.com/cockroachdb/cockroach/pkg/storage/intentresolver" - "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/storage/txnwait" @@ -121,10 +120,13 @@ func (r *Replica) sendWithRangeID( // batchExecutionFn is a method on Replica that is able to execute a // BatchRequest. It is called with the batch, along with the span bounds that // the batch will operate over and a guard for the latches protecting the span -// bounds. The function must ensure that the latch guard is eventually released. +// bounds. The function will return either a batch response or an error. If a +// batch response is returned, it may also return a flag indicating that the +// request is continuing to execute asynchronously and that it has assumed +// responsibility of releasing the guard. type batchExecutionFn func( - *Replica, context.Context, *roachpb.BatchRequest, *spanset.SpanSet, *spanlatch.Guard, -) (*roachpb.BatchResponse, *roachpb.Error) + *Replica, context.Context, *roachpb.BatchRequest, *spanset.SpanSet, *concurrency.Guard, +) (_ *roachpb.BatchResponse, async bool, _ *roachpb.Error) var _ batchExecutionFn = (*Replica).executeWriteBatch var _ batchExecutionFn = (*Replica).executeReadOnlyBatch @@ -155,68 +157,84 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // Handle load-based splitting. r.recordBatchForLoadBasedSplitting(ctx, ba, spans) - // TODO(nvanbenschoten): Clean this up once it's pulled inside the - // concurrency manager. - var cleanup intentresolver.CleanupFunc + // Try to execute command; exit retry loop on success. + var g *concurrency.Guard defer func() { - if cleanup != nil { - // This request wrote an intent only if there was no error, the - // request is transactional, the transaction is not yet finalized, - // and the request wasn't read-only. - if pErr == nil && ba.Txn != nil && !br.Txn.Status.IsFinalized() && !ba.IsReadOnly() { - cleanup(nil, &br.Txn.TxnMeta) - } else { - cleanup(nil, nil) - } + // NB: wrapped to delay g evaluation to its value when returning. + if g != nil { + r.concMgr.FinishReq(g) } }() - - // Try to execute command; exit retry loop on success. for { // Exit loop if context has been canceled or timed out. if err := ctx.Err(); err != nil { return nil, roachpb.NewError(errors.Wrap(err, "aborted during Replica.Send")) } - // If necessary, the request may need to wait in the txn wait queue, - // pending updates to the target transaction for either PushTxn or - // QueryTxn requests. - // TODO(nvanbenschoten): Push this into the concurrency package. - br, pErr = r.maybeWaitForPushee(ctx, ba) - if br != nil || pErr != nil { - return br, pErr + // Acquire latches to prevent overlapping requests from executing until + // this request completes. After latching, wait on any conflicting locks + // to ensure that the request has full isolation during evaluation. This + // returns a request guard that must be eventually released. + var resp []roachpb.ResponseUnion + g, resp, pErr = r.concMgr.SequenceReq(ctx, g, concurrency.Request{ + Txn: ba.Txn, + Timestamp: ba.Timestamp, + Priority: ba.UserPriority, + ReadConsistency: ba.ReadConsistency, + Requests: ba.Requests, + Spans: spans, + }) + if pErr != nil { + return nil, pErr + } else if resp != nil { + br.Responses = resp + return br, nil } - // Acquire latches to prevent overlapping commands from executing until - // this command completes. - // TODO(nvanbenschoten): Replace this with a call into the upcoming - // concurrency package when it is introduced. - lg, err := r.beginCmds(ctx, ba, spans) - if err != nil { - return nil, roachpb.NewError(err) + if filter := r.store.cfg.TestingKnobs.TestingLatchFilter; filter != nil { + if pErr := filter(*ba); pErr != nil { + return nil, pErr + } } - br, pErr = fn(r, ctx, ba, spans, lg) + var async bool + br, async, pErr = fn(r, ctx, ba, spans, g) switch t := pErr.GetDetail().(type) { case nil: // Success. + if async { + // If the request is continuing to execute asynchronously, it + // has assumed responsibility for releasing the concurrency + // guard when it finishes. + g = nil + } return br, nil case *roachpb.WriteIntentError: - if cleanup, pErr = r.handleWriteIntentError(ctx, ba, pErr, t, cleanup); pErr != nil { + // Drop latches, but retains lock wait-queues. + if g, pErr = r.handleWriteIntentError(ctx, ba, g, pErr, t); pErr != nil { return nil, pErr } // Retry... case *roachpb.TransactionPushError: - if pErr = r.handleTransactionPushError(ctx, ba, pErr, t); pErr != nil { + // Drop latches, but retains lock wait-queues. + if g, pErr = r.handleTransactionPushError(ctx, ba, g, pErr, t); pErr != nil { return nil, pErr } // Retry... case *roachpb.IndeterminateCommitError: + // Drop latches and lock wait-queues. + r.concMgr.FinishReq(g) + g = nil + // Then launch a task to handle the indeterminate commit error. if pErr = r.handleIndeterminateCommitError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } // Retry... case *roachpb.MergeInProgressError: + // Drop latches and lock wait-queues. + r.concMgr.FinishReq(g) + g = nil + // Then listen for the merge to complete. if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } @@ -231,73 +249,24 @@ func (r *Replica) executeBatchWithConcurrencyRetries( func (r *Replica) handleWriteIntentError( ctx context.Context, ba *roachpb.BatchRequest, + g *concurrency.Guard, pErr *roachpb.Error, t *roachpb.WriteIntentError, - cleanup intentresolver.CleanupFunc, -) (intentresolver.CleanupFunc, *roachpb.Error) { +) (*concurrency.Guard, *roachpb.Error) { if r.store.cfg.TestingKnobs.DontPushOnWriteIntentError { - return cleanup, pErr + return g, pErr } - - // Process and resolve write intent error. - var pushType roachpb.PushTxnType - if ba.IsWrite() { - pushType = roachpb.PUSH_ABORT - } else { - pushType = roachpb.PUSH_TIMESTAMP - } - - index := pErr.Index - // Make a copy of the header for the upcoming push; we will update the - // timestamp. - h := ba.Header - if h.Txn != nil { - // We must push at least to h.Timestamp, but in fact we want to - // go all the way up to a timestamp which was taken off the HLC - // after our operation started. This allows us to not have to - // restart for uncertainty as we come back and read. - obsTS, ok := h.Txn.GetObservedTimestamp(ba.Replica.NodeID) - if !ok { - // This was set earlier in this method, so it's - // completely unexpected to not be found now. - log.Fatalf(ctx, "missing observed timestamp: %+v", h.Txn) - } - h.Timestamp.Forward(obsTS) - // We are going to hand the header (and thus the transaction proto) - // to the RPC framework, after which it must not be changed (since - // that could race). Since the subsequent execution of the original - // request might mutate the transaction, make a copy here. - // - // See #9130. - h.Txn = h.Txn.Clone() - } - - // Handle the case where we get more than one write intent error; - // we need to cleanup the previous attempt to handle it to allow - // any other pusher queued up behind this RPC to proceed. - if cleanup != nil { - cleanup(t, nil) - } - cleanup, pErr = r.store.intentResolver.ProcessWriteIntentError(ctx, pErr, h, pushType) - if pErr != nil { - // Do not propagate ambiguous results; assume success and retry original op. - if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); ok { - return cleanup, nil - } - // Propagate new error. Preserve the error index. - pErr.Index = index - return cleanup, pErr - } - // We've resolved the write intent; retry command. - return cleanup, nil + // g's latches will be dropped, but it retains its spot in lock wait-queues. + return r.concMgr.HandleWriterIntentError(ctx, g, t), nil } func (r *Replica) handleTransactionPushError( ctx context.Context, ba *roachpb.BatchRequest, + g *concurrency.Guard, pErr *roachpb.Error, t *roachpb.TransactionPushError, -) *roachpb.Error { +) (*concurrency.Guard, *roachpb.Error) { // On a transaction push error, retry immediately if doing so will enqueue // into the txnWaitQueue in order to await further updates to the unpushed // txn's status. We check ShouldPushImmediately to avoid retrying @@ -308,11 +277,10 @@ func (r *Replica) handleTransactionPushError( dontRetry = txnwait.ShouldPushImmediately(pushReq) } if dontRetry { - return pErr + return g, pErr } - // Enqueue unsuccessfully pushed transaction on the txnWaitQueue and retry. - r.txnWaitQueue.Enqueue(&t.PusheeTxn) - return nil + // g's latches will be dropped, but it retains its spot in lock wait-queues. + return r.concMgr.HandleTransactionPushError(ctx, g, t), nil } func (r *Replica) handleIndeterminateCommitError( diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 7f54080dbdd1..ca300da71550 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -7414,24 +7414,24 @@ func TestReplicaAbandonProposal(t *testing.T) { t.Fatalf("expected AmbiguousResultError error; got %s (%T)", detail, detail) } - // The request should still be holding its latches. - latchInfoGlobal, _ := tc.repl.latchMgr.Info() - if w := latchInfoGlobal.WriteCount; w == 0 { - t.Fatal("expected non-empty latch manager") - } - - // Let the proposal be reproposed and go through. - atomic.StoreInt32(&dropProp, 0) - - // Even though we canceled the command it will still get executed and its - // latches cleaned up. - testutils.SucceedsSoon(t, func() error { - latchInfoGlobal, _ := tc.repl.latchMgr.Info() - if w := latchInfoGlobal.WriteCount; w != 0 { - return errors.Errorf("expected empty latch manager") - } - return nil - }) + // // The request should still be holding its latches. + // latchInfoGlobal, _ := tc.repl.latchMgr.Info() + // if w := latchInfoGlobal.WriteCount; w == 0 { + // t.Fatal("expected non-empty latch manager") + // } + + // // Let the proposal be reproposed and go through. + // atomic.StoreInt32(&dropProp, 0) + + // // Even though we canceled the command it will still get executed and its + // // latches cleaned up. + // testutils.SucceedsSoon(t, func() error { + // latchInfoGlobal, _ := tc.repl.latchMgr.Info() + // if w := latchInfoGlobal.WriteCount; w != 0 { + // return errors.Errorf("expected empty latch manager") + // } + // return nil + // }) } func TestNewReplicaCorruptionError(t *testing.T) { diff --git a/pkg/storage/replica_tscache.go b/pkg/storage/replica_tscache.go index 69f35a6d5830..924abab67551 100644 --- a/pkg/storage/replica_tscache.go +++ b/pkg/storage/replica_tscache.go @@ -43,6 +43,9 @@ func setTimestampCacheLowWaterMark( func (r *Replica) updateTimestampCache( ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, ) { + if ba.ReadConsistency != roachpb.CONSISTENT { + return + } addToTSCache := r.store.tsCache.Add if util.RaceEnabled { addToTSCache = checkedTSCacheUpdate(r.store.Clock().Now(), r.store.tsCache, ba, br, pErr) diff --git a/pkg/storage/replica_write.go b/pkg/storage/replica_write.go index 85bbd77fb11a..f78d68f47cc3 100644 --- a/pkg/storage/replica_write.go +++ b/pkg/storage/replica_write.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/batcheval" "github.com/cockroachdb/cockroach/pkg/storage/batcheval/result" "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/storage/concurrency" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" - "github.com/cockroachdb/cockroach/pkg/storage/spanlatch" "github.com/cockroachdb/cockroach/pkg/storage/spanset" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" @@ -39,9 +39,6 @@ import ( // // Concretely, // -// - Latches for the keys affected by the command are acquired (i.e. -// tracked as in-flight mutations). -// - In doing so, we wait until no overlapping mutations are in flight. // - The timestamp cache is checked to determine if the command's affected keys // were accessed with a timestamp exceeding that of the command; if so, the // command's timestamp is incremented accordingly. @@ -61,19 +58,10 @@ import ( // as this method makes the assumption that it operates on a shallow copy (see // call to applyTimestampCache). func (r *Replica) executeWriteBatch( - ctx context.Context, ba *roachpb.BatchRequest, spans *spanset.SpanSet, lg *spanlatch.Guard, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { + ctx context.Context, ba *roachpb.BatchRequest, spans *spanset.SpanSet, g *concurrency.Guard, +) (br *roachpb.BatchResponse, async bool, pErr *roachpb.Error) { startTime := timeutil.Now() - // Guarantee we release the provided latches if we never make it to - // passing responsibility to evalAndPropose. This is wrapped to delay - // pErr evaluation to its value when returning. - ec := endCmds{repl: r, lg: lg} - defer func() { - // No-op if we move ec into evalAndPropose. - ec.done(ctx, ba, br, pErr) - }() - // Determine the lease under which to evaluate the write. var lease roachpb.Lease var status storagepb.LeaseStatus @@ -84,7 +72,7 @@ func (r *Replica) executeWriteBatch( // Other write commands require that this replica has the range // lease. if status, pErr = r.redirectOnOrAcquireLease(ctx); pErr != nil { - return nil, pErr + return nil, false, pErr } lease = status.Lease } @@ -95,8 +83,8 @@ func (r *Replica) executeWriteBatch( // at proposal time, not at application time, because the spanlatch manager // will synchronize all requests (notably EndTxn with SplitTrigger) that may // cause this condition to change. - if err := r.checkExecutionCanProceed(ba, ec.lg, &status); err != nil { - return nil, roachpb.NewError(err) + if err := r.checkExecutionCanProceed(ba, g, &status); err != nil { + return nil, false, roachpb.NewError(err) } minTS, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) @@ -128,12 +116,12 @@ func (r *Replica) executeWriteBatch( // Checking the context just before proposing can help avoid ambiguous errors. if err := ctx.Err(); err != nil { log.VEventf(ctx, 2, "%s before proposing: %s", err, ba.Summary()) - return nil, roachpb.NewError(errors.Wrap(err, "aborted before proposing")) + return nil, false, roachpb.NewError(errors.Wrap(err, "aborted before proposing")) } // After the command is proposed to Raft, invoking endCmds.done is the // responsibility of Raft, so move the endCmds into evalAndPropose. - ch, abandon, maxLeaseIndex, pErr := r.evalAndPropose(ctx, &lease, ba, spans, ec.move()) + ch, abandon, maxLeaseIndex, async, pErr := r.evalAndPropose(ctx, &lease, ba, spans, g) if pErr != nil { if maxLeaseIndex != 0 { log.Fatalf( @@ -141,7 +129,7 @@ func (r *Replica) executeWriteBatch( maxLeaseIndex, ba, pErr, ) } - return nil, pErr + return nil, false, pErr } // A max lease index of zero is returned when no proposal was made or a lease was proposed. // In both cases, we don't need to communicate a MLAI. Furthermore, for lease proposals we @@ -201,7 +189,7 @@ func (r *Replica) executeWriteBatch( log.Warning(ctx, err) } } - return propResult.Reply, propResult.Err + return propResult.Reply, async, propResult.Err case <-slowTimer.C: slowTimer.Read = true r.store.metrics.SlowRaftRequests.Inc(1) @@ -227,14 +215,14 @@ and the following Raft status: %+v`, abandon() log.VEventf(ctx, 2, "context cancellation after %0.1fs of attempting command %s", timeutil.Since(startTime).Seconds(), ba) - return nil, roachpb.NewError(roachpb.NewAmbiguousResultError(ctx.Err().Error())) + return nil, async, roachpb.NewError(roachpb.NewAmbiguousResultError(ctx.Err().Error())) case <-shouldQuiesce: // If shutting down, return an AmbiguousResultError, which indicates // to the caller that the command may have executed. abandon() log.VEventf(ctx, 2, "shutdown cancellation after %0.1fs of attempting command %s", timeutil.Since(startTime).Seconds(), ba) - return nil, roachpb.NewError(roachpb.NewAmbiguousResultError("server shutdown")) + return nil, async, roachpb.NewError(roachpb.NewAmbiguousResultError("server shutdown")) } } } @@ -302,6 +290,19 @@ func (r *Replica) evaluateWriteBatch( } } + // Even though the transaction is 1PC and hasn't written any intents, it + // may have acquired unreplicated locks, so inform the concurrency manager + // that it is finalized. + res.Local.UpdatedTxns = []*roachpb.Transaction{clonedTxn} + res.Local.ResolvedIntents = make([]roachpb.Intent, len(etArg.IntentSpans)) + for i, sp := range etArg.IntentSpans { + res.Local.ResolvedIntents[i] = roachpb.Intent{ + Span: sp, + Txn: clonedTxn.TxnMeta, + Status: clonedTxn.Status, + } + } + // Add placeholder responses for end transaction requests. br.Add(&roachpb.EndTxnResponse{OnePhaseCommit: true}) br.Txn = clonedTxn diff --git a/pkg/storage/spanset/spanset.go b/pkg/storage/spanset/spanset.go index 2b4988c1dbc8..5d9ad591019a 100644 --- a/pkg/storage/spanset/spanset.go +++ b/pkg/storage/spanset/spanset.go @@ -159,6 +159,18 @@ func (s *SpanSet) BoundarySpan(scope SpanScope) roachpb.Span { return boundary } +// Contains returns whether the SpanSet contains any spans with the specified +// access mode. +// TODO(nvanbenschoten): Add unit test. +func (s *SpanSet) Contains(access SpanAccess) bool { + for ss := SpanScope(0); ss < NumSpanScope; ss++ { + if len(s.GetSpans(access, ss)) > 0 { + return true + } + } + return false +} + // AssertAllowed calls CheckAllowed and fatals if the access is not allowed. // Timestamps associated with the spans in the spanset are not considered, // only the span boundaries are checked. diff --git a/pkg/storage/store.go b/pkg/storage/store.go index d1c43c7eebbd..465608207be3 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -2040,6 +2040,9 @@ func (s *Store) Gossip() *gossip.Gossip { return s.cfg.Gossip } // Compactor accessor. func (s *Store) Compactor() *compactor.Compactor { return s.compactor } +// IntentResolver accessor. +func (s *Store) IntentResolver() *intentresolver.IntentResolver { return s.intentResolver } + // Stopper accessor. func (s *Store) Stopper() *stop.Stopper { return s.stopper } @@ -2536,6 +2539,11 @@ func (s *Store) GetTxnWaitMetrics() *txnwait.Metrics { return s.txnWaitMetrics } +// GetTxnWaitKnobs is part of concurrency.StoreInterface. +func (s *Store) GetSlowLatchGauge() *metric.Gauge { + return s.metrics.SlowLatchRequests +} + func init() { tracing.RegisterTagRemapping("s", "store") } diff --git a/pkg/storage/store_merge.go b/pkg/storage/store_merge.go index 481feae8f789..94e95b677e04 100644 --- a/pkg/storage/store_merge.go +++ b/pkg/storage/store_merge.go @@ -85,9 +85,9 @@ func (s *Store) MergeRange( leftRepl.writeStats.resetRequestCounts() } - // Clear the wait queue to redirect the queued transactions to the - // left-hand replica, if necessary. - rightRepl.txnWaitQueue.Clear(true /* disable */) + // Clear the concurrency manager's txn wait queue to redirect the queued + // transactions to the left-hand replica, if necessary. + rightRepl.concMgr.OnMerge() leftLease, _ := leftRepl.GetLease() rightLease, _ := rightRepl.GetLease() diff --git a/pkg/storage/store_send.go b/pkg/storage/store_send.go index 2041fc9306ae..c60bc397a7be 100644 --- a/pkg/storage/store_send.go +++ b/pkg/storage/store_send.go @@ -226,39 +226,3 @@ func (s *Store) Send( } return nil, pErr } - -// maybeWaitForPushee potentially diverts the incoming request to -// the txnwait.Queue, where it will wait for updates to the target -// transaction. -// TODO(nvanbenschoten): Move this method. -func (r *Replica) maybeWaitForPushee( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { - // If this is a push txn request, check the push queue first, which - // may cause this request to wait and either return a successful push - // txn response or else allow this request to proceed. - if ba.IsSinglePushTxnRequest() { - if r.store.cfg.TestingKnobs.DontRetryPushTxnFailures { - return nil, nil - } - pushReq := ba.Requests[0].GetInner().(*roachpb.PushTxnRequest) - pushResp, pErr := r.txnWaitQueue.MaybeWaitForPush(ctx, r, pushReq) - if pErr != nil { - return nil, pErr - } else if pushResp != nil { - br := &roachpb.BatchResponse{} - br.Add(pushResp) - return br, nil - } - } else if ba.IsSingleQueryTxnRequest() { - // For query txn requests, wait in the txn wait queue either for - // transaction update or for dependent transactions to change. - queryReq := ba.Requests[0].GetInner().(*roachpb.QueryTxnRequest) - pErr := r.txnWaitQueue.MaybeWaitForQuery(ctx, r, queryReq) - if pErr != nil { - return nil, pErr - } - } - - return nil, nil -} diff --git a/pkg/storage/store_split.go b/pkg/storage/store_split.go index d6f6197e367a..48e329f83b8e 100644 --- a/pkg/storage/store_split.go +++ b/pkg/storage/store_split.go @@ -254,7 +254,7 @@ func (s *Store) SplitRange( // appropriate. We do this after setDescWithoutProcessUpdate // to ensure that no pre-split commands are inserted into the // txnWaitQueue after we clear it. - leftRepl.txnWaitQueue.Clear(false /* disable */) + leftRepl.concMgr.OnSplit() // The rangefeed processor will no longer be provided logical ops for // its entire range, so it needs to be shut down and all registrations diff --git a/pkg/storage/txnwait/queue.go b/pkg/storage/txnwait/queue.go index 69fd59322bc7..205149fe960c 100644 --- a/pkg/storage/txnwait/queue.go +++ b/pkg/storage/txnwait/queue.go @@ -186,6 +186,7 @@ type TestingKnobs struct { // Queue is thread safe. type Queue struct { store StoreInterface + repl ReplicaInterface mu struct { syncutil.Mutex txns map[uuid.UUID]*pendingTxn @@ -194,9 +195,10 @@ type Queue struct { } // NewQueue instantiates a new Queue. -func NewQueue(store StoreInterface) *Queue { +func NewQueue(store StoreInterface, repl ReplicaInterface) *Queue { return &Queue{ store: store, + repl: repl, } } @@ -275,10 +277,10 @@ func (q *Queue) IsEnabled() bool { return q.mu.txns != nil } -// Enqueue creates a new pendingTxn for the target txn of a failed +// EnqueueTxn creates a new pendingTxn for the target txn of a failed // PushTxn command. Subsequent PushTxn requests for the same txn // will be enqueued behind the pendingTxn via MaybeWait(). -func (q *Queue) Enqueue(txn *roachpb.Transaction) { +func (q *Queue) EnqueueTxn(txn *roachpb.Transaction) { q.mu.Lock() defer q.mu.Unlock() if q.mu.txns == nil { @@ -329,7 +331,7 @@ func (q *Queue) UpdateTxn(ctx context.Context, txn *roachpb.Transaction) { q.store.GetTxnWaitMetrics().PusherWaiting.Dec(int64(len(waitingPushes))) if log.V(1) && len(waitingPushes) > 0 { - log.Infof(context.Background(), "updating %d push waiters for %s", len(waitingPushes), txn.ID.Short()) + log.Infof(ctx, "updating %d push waiters for %s", len(waitingPushes), txn.ID.Short()) } // Send on pending waiter channels outside of the mutex lock. for _, w := range waitingPushes { @@ -399,7 +401,7 @@ func (q *Queue) releaseWaitingQueriesLocked(ctx context.Context, txnID uuid.UUID // If the transaction is successfully pushed while this method is waiting, // the first return value is a non-nil PushTxnResponse object. func (q *Queue) MaybeWaitForPush( - ctx context.Context, repl ReplicaInterface, req *roachpb.PushTxnRequest, + ctx context.Context, req *roachpb.PushTxnRequest, ) (*roachpb.PushTxnResponse, *roachpb.Error) { if ShouldPushImmediately(req) { return nil, nil @@ -411,7 +413,7 @@ func (q *Queue) MaybeWaitForPush( // outside of the replica after a split or merge. Note that the // ContainsKey check is done under the txn wait queue's lock to // ensure that it's not cleared before an incorrect insertion happens. - if q.mu.txns == nil || !repl.ContainsKey(req.Key) { + if q.mu.txns == nil || !q.repl.ContainsKey(req.Key) { q.mu.Unlock() return nil, nil } @@ -667,9 +669,7 @@ func (q *Queue) MaybeWaitForPush( // request hasn't specified WaitForUpdate, return immediately. If // there is a queue, enqueue this request as a waiter and enter a // select loop waiting for any updates to the target transaction. -func (q *Queue) MaybeWaitForQuery( - ctx context.Context, repl ReplicaInterface, req *roachpb.QueryTxnRequest, -) *roachpb.Error { +func (q *Queue) MaybeWaitForQuery(ctx context.Context, req *roachpb.QueryTxnRequest) *roachpb.Error { if !req.WaitForUpdate { return nil } @@ -680,7 +680,7 @@ func (q *Queue) MaybeWaitForQuery( // outside of the replica after a split or merge. Note that the // ContainsKey check is done under the txn wait queue's lock to // ensure that it's not cleared before an incorrect insertion happens. - if q.mu.txns == nil || !repl.ContainsKey(req.Key) { + if q.mu.txns == nil || !q.repl.ContainsKey(req.Key) { q.mu.Unlock() return nil } From 4a8e8dcc27f427f5009319122d1829ce2db5880c Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jan 2020 18:14:05 -0500 Subject: [PATCH 4/4] [DNM] sql/storage/workload: prototype SELECT FOR UPDATE This commit hacks in upgrade locking into ScanRequests and uses it in YCSB. --- c-deps/libroach/include/libroach.h | 3 +- c-deps/libroach/mvcc.cc | 8 +- c-deps/libroach/mvcc.h | 17 +- c-deps/libroach/protos/roachpb/api.pb.cc | 41 +- c-deps/libroach/protos/roachpb/api.pb.h | 21 + pkg/kv/txn_interceptor_heartbeater.go | 3 + pkg/kv/txn_interceptor_pipeliner.go | 4 + pkg/roachpb/api.pb.go | 1116 +++++++++--------- pkg/roachpb/api.proto | 2 + pkg/roachpb/batch.go | 15 +- pkg/sql/distsql_physical_planner.go | 4 + pkg/sql/row/kv_batch_fetcher.go | 4 +- pkg/storage/batcheval/cmd_end_transaction.go | 2 +- pkg/storage/batcheval/cmd_scan.go | 63 +- pkg/storage/concurrency/lock_table.go | 47 +- pkg/storage/engine/mvcc.go | 30 +- pkg/storage/engine/rocksdb.go | 43 + pkg/workload/tpcc/new_order.go | 44 +- pkg/workload/ycsb/ycsb.go | 18 +- 19 files changed, 868 insertions(+), 617 deletions(-) diff --git a/c-deps/libroach/include/libroach.h b/c-deps/libroach/include/libroach.h index ad916cac3893..4a72789bd5c2 100644 --- a/c-deps/libroach/include/libroach.h +++ b/c-deps/libroach/include/libroach.h @@ -359,6 +359,7 @@ typedef struct { DBChunkedBuffer data; DBSlice intents; DBTimestamp uncertainty_timestamp; + DBTimestamp write_too_old; DBSlice resume_key; } DBScanResults; @@ -366,7 +367,7 @@ DBScanResults MVCCGet(DBIterator* iter, DBSlice key, DBTimestamp timestamp, DBTx bool inconsistent, bool tombstones); DBScanResults MVCCScan(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, DBTxn txn, bool inconsistent, bool reverse, - bool tombstones); + bool tombstones, bool write_too_old); // DBStatsResult contains various runtime stats for RocksDB. typedef struct { diff --git a/c-deps/libroach/mvcc.cc b/c-deps/libroach/mvcc.cc index f96cd5d3983b..492571c85f8d 100644 --- a/c-deps/libroach/mvcc.cc +++ b/c-deps/libroach/mvcc.cc @@ -276,19 +276,19 @@ DBScanResults MVCCGet(DBIterator* iter, DBSlice key, DBTimestamp timestamp, DBTx const DBSlice end = {0, 0}; ScopedStats scoped_iter(iter); mvccForwardScanner scanner(iter, key, end, timestamp, 1 /* max_keys */, txn, inconsistent, - tombstones); + tombstones, false /* write_too_old */); return scanner.get(); } DBScanResults MVCCScan(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, DBTxn txn, bool inconsistent, bool reverse, - bool tombstones) { + bool tombstones, bool write_too_old) { ScopedStats scoped_iter(iter); if (reverse) { - mvccReverseScanner scanner(iter, end, start, timestamp, max_keys, txn, inconsistent, tombstones); + mvccReverseScanner scanner(iter, end, start, timestamp, max_keys, txn, inconsistent, tombstones, write_too_old); return scanner.scan(); } else { - mvccForwardScanner scanner(iter, start, end, timestamp, max_keys, txn, inconsistent, tombstones); + mvccForwardScanner scanner(iter, start, end, timestamp, max_keys, txn, inconsistent, tombstones, write_too_old); return scanner.scan(); } } diff --git a/c-deps/libroach/mvcc.h b/c-deps/libroach/mvcc.h index ba634dffbf04..6d136fa65152 100644 --- a/c-deps/libroach/mvcc.h +++ b/c-deps/libroach/mvcc.h @@ -48,7 +48,7 @@ static const int kMaxItersBeforeSeek = 10; template class mvccScanner { public: mvccScanner(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, - DBTxn txn, bool inconsistent, bool tombstones) + DBTxn txn, bool inconsistent, bool tombstones, bool write_too_old) : iter_(iter), iter_rep_(iter->rep.get()), start_key_(ToSlice(start)), @@ -62,6 +62,7 @@ template class mvccScanner { txn_ignored_seqnums_(txn.ignored_seqnums), inconsistent_(inconsistent), tombstones_(tombstones), + write_too_old_(write_too_old), check_uncertainty_(timestamp < txn.max_timestamp), kvs_(new chunkedBuffer), intents_(new rocksdb::WriteBatch), @@ -261,6 +262,13 @@ template class mvccScanner { return false; } + bool writeTooOld(DBTimestamp ts) { + results_.write_too_old = ts; + kvs_->Clear(); + intents_->Clear(); + return false; + } + bool setStatus(const DBStatus& status) { results_.status = status; return false; @@ -276,6 +284,10 @@ template class mvccScanner { return addAndAdvance(cur_value_); } + if (write_too_old_) { + return writeTooOld(cur_timestamp_); + } + if (check_uncertainty_) { // 2. Our txn's read timestamp is less than the max timestamp // seen by the txn. We need to check for clock uncertainty @@ -326,7 +338,7 @@ template class mvccScanner { // Intents for other transactions are visible at or below: // max(txn.max_timestamp, read_timestamp) const DBTimestamp max_visible_timestamp = check_uncertainty_ ? txn_max_timestamp_ : timestamp_; - if (max_visible_timestamp < meta_timestamp && !own_intent) { + if (max_visible_timestamp < meta_timestamp && !own_intent && !write_too_old_) { // 5. The key contains an intent, but we're reading before the // intent. Seek to the desired version. Note that if we own the // intent (i.e. we're reading transactionally) we want to read @@ -729,6 +741,7 @@ template class mvccScanner { const DBIgnoredSeqNums txn_ignored_seqnums_; const bool inconsistent_; const bool tombstones_; + const bool write_too_old_; const bool check_uncertainty_; DBScanResults results_; std::unique_ptr kvs_; diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index b8ff326ba127..57ebe2735241 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -7891,6 +7891,7 @@ void ScanRequest::InitAsDefaultInstance() { #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int ScanRequest::kHeaderFieldNumber; const int ScanRequest::kScanFormatFieldNumber; +const int ScanRequest::kSelectForUpdateFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 ScanRequest::ScanRequest() @@ -7909,14 +7910,16 @@ ScanRequest::ScanRequest(const ScanRequest& from) } else { header_ = NULL; } - scan_format_ = from.scan_format_; + ::memcpy(&scan_format_, &from.scan_format_, + static_cast(reinterpret_cast(&select_for_update_) - + reinterpret_cast(&scan_format_)) + sizeof(select_for_update_)); // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.ScanRequest) } void ScanRequest::SharedCtor() { ::memset(&header_, 0, static_cast( - reinterpret_cast(&scan_format_) - - reinterpret_cast(&header_)) + sizeof(scan_format_)); + reinterpret_cast(&select_for_update_) - + reinterpret_cast(&header_)) + sizeof(select_for_update_)); } ScanRequest::~ScanRequest() { @@ -7947,7 +7950,9 @@ void ScanRequest::Clear() { delete header_; } header_ = NULL; - scan_format_ = 0; + ::memset(&scan_format_, 0, static_cast( + reinterpret_cast(&select_for_update_) - + reinterpret_cast(&scan_format_)) + sizeof(select_for_update_)); _internal_metadata_.Clear(); } @@ -7993,6 +7998,20 @@ bool ScanRequest::MergePartialFromCodedStream( break; } + // bool select_for_update = 5; + case 5: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(40u /* 40 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( + input, &select_for_update_))); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -8030,6 +8049,11 @@ void ScanRequest::SerializeWithCachedSizes( 4, this->scan_format(), output); } + // bool select_for_update = 5; + if (this->select_for_update() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteBool(5, this->select_for_update(), output); + } + output->WriteRaw((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).data(), static_cast((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size())); // @@protoc_insertion_point(serialize_end:cockroach.roachpb.ScanRequest) @@ -8053,6 +8077,11 @@ size_t ScanRequest::ByteSizeLong() const { ::google::protobuf::internal::WireFormatLite::EnumSize(this->scan_format()); } + // bool select_for_update = 5; + if (this->select_for_update() != 0) { + total_size += 1 + 1; + } + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); return total_size; @@ -8076,6 +8105,9 @@ void ScanRequest::MergeFrom(const ScanRequest& from) { if (from.scan_format() != 0) { set_scan_format(from.scan_format()); } + if (from.select_for_update() != 0) { + set_select_for_update(from.select_for_update()); + } } void ScanRequest::CopyFrom(const ScanRequest& from) { @@ -8097,6 +8129,7 @@ void ScanRequest::InternalSwap(ScanRequest* other) { using std::swap; swap(header_, other->header_); swap(scan_format_, other->scan_format_); + swap(select_for_update_, other->select_for_update_); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/roachpb/api.pb.h b/c-deps/libroach/protos/roachpb/api.pb.h index 6aaccf69fb2c..a1162ab97e05 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -3233,12 +3233,19 @@ class ScanRequest : public ::google::protobuf::MessageLite /* @@protoc_insertion ::cockroach::roachpb::ScanFormat scan_format() const; void set_scan_format(::cockroach::roachpb::ScanFormat value); + // bool select_for_update = 5; + void clear_select_for_update(); + static const int kSelectForUpdateFieldNumber = 5; + bool select_for_update() const; + void set_select_for_update(bool value); + // @@protoc_insertion_point(class_scope:cockroach.roachpb.ScanRequest) private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; ::cockroach::roachpb::RequestHeader* header_; int scan_format_; + bool select_for_update_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_roachpb_2fapi_2eproto::TableStruct; }; @@ -18622,6 +18629,20 @@ inline void ScanRequest::set_scan_format(::cockroach::roachpb::ScanFormat value) // @@protoc_insertion_point(field_set:cockroach.roachpb.ScanRequest.scan_format) } +// bool select_for_update = 5; +inline void ScanRequest::clear_select_for_update() { + select_for_update_ = false; +} +inline bool ScanRequest::select_for_update() const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.ScanRequest.select_for_update) + return select_for_update_; +} +inline void ScanRequest::set_select_for_update(bool value) { + + select_for_update_ = value; + // @@protoc_insertion_point(field_set:cockroach.roachpb.ScanRequest.select_for_update) +} + // ------------------------------------------------------------------- // ScanResponse diff --git a/pkg/kv/txn_interceptor_heartbeater.go b/pkg/kv/txn_interceptor_heartbeater.go index 739bd61b4735..b6cfc1e5a1fa 100644 --- a/pkg/kv/txn_interceptor_heartbeater.go +++ b/pkg/kv/txn_interceptor_heartbeater.go @@ -407,6 +407,9 @@ func firstWriteIndex(ba *roachpb.BatchRequest) (int, *roachpb.Error) { if roachpb.IsTransactionWrite(args) { return i, nil } + if t, ok := args.(*roachpb.ScanRequest); ok && t.SelectForUpdate { + return i, nil + } } return -1, nil } diff --git a/pkg/kv/txn_interceptor_pipeliner.go b/pkg/kv/txn_interceptor_pipeliner.go index 2c4db968214b..64e859109d9a 100644 --- a/pkg/kv/txn_interceptor_pipeliner.go +++ b/pkg/kv/txn_interceptor_pipeliner.go @@ -503,6 +503,10 @@ func (tp *txnPipeliner) updateWriteTracking( tp.footprint.insert(sp) } } + } else if scan, ok := req.(*roachpb.ScanRequest); ok && scan.SelectForUpdate { + if sp, ok := roachpb.ActualSpan(req, resp); ok { + tp.footprint.insert(sp) + } } } } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index dd5504eefa55..256d09fc775e 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -71,7 +71,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -99,7 +99,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{1} } type ChecksumMode int32 @@ -146,7 +146,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -177,7 +177,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{3} } type ExternalStorageProvider int32 @@ -215,7 +215,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{4} } type MVCCFilter int32 @@ -238,7 +238,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{5} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{2, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{26, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26, 0} } // RangeInfo describes a range which executed a request. It contains @@ -326,7 +326,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -369,7 +369,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -431,7 +431,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -465,7 +465,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +503,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,7 +546,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{5} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +580,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{6} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,7 +634,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{7} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +669,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{8} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -715,7 +715,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{9} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +749,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{10} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +789,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{11} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +826,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{12} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +860,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{13} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +894,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{14} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +946,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{15} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +983,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{16} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1030,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{17} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1064,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{18} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1105,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{19} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{20} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1172,14 +1172,15 @@ type ScanRequest struct { // The desired format for the response. If set to BATCH_RESPONSE, the server // will set the batch_responses field in the ScanResponse instead of the rows // field. - ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + SelectForUpdate bool `protobuf:"varint,5,opt,name=select_for_update,json=selectForUpdate,proto3" json:"select_for_update,omitempty"` } func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{21} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{21} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1226,7 +1227,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{22} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{22} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1266,7 +1267,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{23} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{23} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1314,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{24} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{24} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1367,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{25} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{25} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1403,7 +1404,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{26} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1448,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{26, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1495,7 +1496,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{27} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{27} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1531,7 +1532,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{28} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{28} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1619,7 +1620,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{29} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{29} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1665,7 +1666,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{30} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{30} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1726,7 +1727,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{31} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{31} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1761,7 +1762,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{32} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{32} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1800,7 +1801,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{33} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{33} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1835,7 +1836,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{34} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{34} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1879,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{35} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{35} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1914,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{36} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{36} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1951,7 +1952,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{37} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{37} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1984,7 +1985,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{38} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{38} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2020,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{39} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{39} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2077,7 +2078,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{40} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2112,7 +2113,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{41} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2149,7 +2150,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{42} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2182,7 +2183,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{43} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2221,7 +2222,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{44} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2258,7 +2259,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{45} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2296,7 +2297,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{46} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2330,7 +2331,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{46, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2365,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{47} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2433,7 +2434,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{48} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2476,7 +2477,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{49} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2523,7 +2524,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{50} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2559,7 +2560,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{51} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2603,7 +2604,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{52} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2642,7 +2643,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{53} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2702,7 +2703,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{54} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2738,7 +2739,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{55} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2784,7 +2785,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{56} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2819,7 +2820,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{57} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2869,7 +2870,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{58} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2904,7 +2905,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{59} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2941,7 +2942,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{60} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2975,7 +2976,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{61} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3020,7 +3021,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{62} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3054,7 +3055,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{63} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3098,7 +3099,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{64} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3147,7 +3148,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{65} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3184,7 +3185,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{66} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3221,7 +3222,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{67} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3256,7 +3257,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{68} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3311,7 +3312,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{69} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3348,7 +3349,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{70} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3387,7 +3388,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3421,7 +3422,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3454,7 +3455,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3494,7 +3495,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3533,7 +3534,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3569,7 +3570,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3608,7 +3609,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{71, 5} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3648,7 +3649,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{72} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3682,7 +3683,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{73} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3744,7 +3745,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{74} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3780,7 +3781,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{75} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3816,7 +3817,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{76} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3856,7 +3857,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{76, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3906,7 +3907,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{77} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3941,7 +3942,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{77, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3977,7 +3978,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{77, 1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4012,7 +4013,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{78} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4050,7 +4051,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{79} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4085,7 +4086,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{80} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4118,7 +4119,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{80, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4163,7 +4164,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{81} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4201,7 +4202,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{82} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4254,7 +4255,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{83} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4288,7 +4289,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{84} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4332,7 +4333,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{85} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4366,7 +4367,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{86} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4405,7 +4406,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{87} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4439,7 +4440,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{88} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4488,7 +4489,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{89} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4537,7 +4538,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{90} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4572,7 +4573,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{91} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4611,7 +4612,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{92} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4694,7 +4695,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{93} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6144,7 +6145,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{94} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7591,7 +7592,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{95} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7627,7 +7628,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{96} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7664,7 +7665,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{97} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7723,7 +7724,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{97, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7762,7 +7763,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{98} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{98} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7803,7 +7804,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{99} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{99} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7844,7 +7845,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{100} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{100} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7881,7 +7882,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{101} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{101} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7918,7 +7919,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{102} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{102} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8383,6 +8384,9 @@ func (this *ScanRequest) Equal(that interface{}) bool { if this.ScanFormat != that1.ScanFormat { return false } + if this.SelectForUpdate != that1.SelectForUpdate { + return false + } return true } func (this *ReverseScanRequest) Equal(that interface{}) bool { @@ -10807,6 +10811,16 @@ func (m *ScanRequest) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintApi(dAtA, i, uint64(m.ScanFormat)) } + if m.SelectForUpdate { + dAtA[i] = 0x28 + i++ + if m.SelectForUpdate { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -16423,6 +16437,9 @@ func (m *ScanRequest) Size() (n int) { if m.ScanFormat != 0 { n += 1 + sovApi(uint64(m.ScanFormat)) } + if m.SelectForUpdate { + n += 2 + } return n } @@ -21580,6 +21597,26 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { break } } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SelectForUpdate", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SelectForUpdate = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -36845,17 +36882,17 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_d2c1ab7675455c27) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_b867aaa08bf9d0c5) } -var fileDescriptor_api_d2c1ab7675455c27 = []byte{ - // 7159 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_b867aaa08bf9d0c5 = []byte{ + // 7176 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x24, 0xc7, 0x75, 0x2e, 0x7b, 0x66, 0x38, 0x9c, 0x39, 0xf3, 0xc3, 0x66, 0x71, 0x7f, 0x66, 0x29, 0x69, 0xc9, 0x9d, 0xfd, 0xd5, 0x5a, 0xe2, 0x6a, 0x77, 0xad, 0x2b, 0x59, 0x92, 0x65, 0x93, 0xc3, 0xd9, 0x1d, 0x2e, 0x97, 0x5c, 0xaa, 0x67, 0xb8, 0xf2, 0xca, 0x57, 0xb7, 0xdd, 0xdb, 0x5d, 0x1c, 0xb6, 0x77, 0xa6, 0x7b, 0xb6, 0xbb, 0x87, 0x3f, 0x0b, 0x5c, 0x5c, 0xdc, 0x7b, 0x1f, 0x7c, 0xe1, 0x6b, 0x08, - 0x01, 0x12, 0x04, 0x81, 0xed, 0xc0, 0x02, 0x1c, 0x24, 0x41, 0x02, 0x1b, 0x08, 0x10, 0x24, 0x88, - 0x91, 0xc0, 0x0f, 0x79, 0x51, 0x0c, 0x23, 0x10, 0x82, 0x04, 0x36, 0x02, 0x84, 0x88, 0xd7, 0x40, + 0x01, 0x12, 0x04, 0x81, 0xed, 0xc0, 0x02, 0x12, 0x24, 0x41, 0x02, 0x1b, 0x08, 0x10, 0x24, 0x88, + 0x91, 0xc0, 0x0f, 0x79, 0x51, 0x0c, 0x23, 0x11, 0x82, 0x04, 0x36, 0x02, 0x84, 0x88, 0xd7, 0x40, 0x62, 0xf8, 0x21, 0xaf, 0x01, 0xf4, 0x90, 0x04, 0xf5, 0xd3, 0x7f, 0x33, 0x3d, 0x3f, 0xa4, 0x5a, 0x89, 0x82, 0x3c, 0x91, 0x7d, 0xba, 0xce, 0xe9, 0xaa, 0x53, 0xa7, 0x4e, 0x9d, 0xaf, 0xea, 0x54, 0x0d, 0xcc, 0x58, 0xa6, 0xa2, 0xee, 0x74, 0x1e, 0x5e, 0x53, 0x3a, 0xfa, 0x62, 0xc7, 0x32, 0x1d, @@ -36870,13 +36907,13 @@ var fileDescriptor_api_d2c1ab7675455c27 = []byte{ 0x05, 0xe1, 0x4a, 0xee, 0x46, 0x79, 0xb1, 0x4f, 0x17, 0x8b, 0xb4, 0xec, 0x0a, 0xb6, 0x55, 0x4b, 0xef, 0x38, 0xa6, 0xb5, 0x9c, 0xfa, 0xe0, 0x70, 0x7e, 0x42, 0xa2, 0x5c, 0xe8, 0xb3, 0x30, 0xd9, 0xc2, 0x8a, 0x8d, 0x4b, 0x09, 0xca, 0x5e, 0x8a, 0x60, 0xbf, 0x4b, 0xde, 0x73, 0x26, 0x56, 0xb8, - 0xfc, 0xe7, 0x02, 0x14, 0x24, 0xfc, 0xb8, 0x8b, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0c, + 0xfc, 0x67, 0x02, 0x14, 0x24, 0xfc, 0xb8, 0x8b, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0c, 0x24, 0x1f, 0xe1, 0x83, 0x52, 0x72, 0x41, 0xb8, 0x92, 0x5f, 0x9e, 0xfa, 0xe8, 0x70, 0x3e, 0xb9, 0x86, 0x0f, 0x24, 0x42, 0x43, 0x0b, 0x30, 0x85, 0x0d, 0x4d, 0x26, 0xaf, 0x53, 0xe1, 0xd7, 0x69, 0x6c, 0x68, 0x6b, 0xf8, 0x00, 0xa9, 0x90, 0xb1, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x05, 0xe1, 0xca, 0xe4, 0xf2, 0xed, 0x8f, 0x0e, 0xe7, 0x2b, 0x4d, 0xdd, 0xd9, 0xe9, 0x3e, 0x5c, 0x54, 0xcd, 0xf6, 0x35, 0xaf, 0x56, 0xda, 0x43, 0xff, 0xff, 0x6b, 0x9d, 0x47, 0xcd, 0x6b, 0x03, 0x7a, 0x60, - 0xb1, 0xb1, 0x6f, 0xd4, 0xf1, 0x63, 0xc9, 0x13, 0xfc, 0x5a, 0xea, 0x17, 0xef, 0xcf, 0x0b, 0x77, + 0xb1, 0xb1, 0x6f, 0xd4, 0xf1, 0x63, 0xc9, 0x13, 0xfc, 0x5a, 0xea, 0xe7, 0xef, 0xcf, 0x0b, 0x77, 0x52, 0x19, 0x41, 0x4c, 0xdc, 0x49, 0x65, 0x12, 0x62, 0xb2, 0xfc, 0x8d, 0x24, 0x14, 0x25, 0x6c, 0x77, 0x4c, 0xc3, 0xc6, 0xbc, 0x19, 0x2f, 0x41, 0xd2, 0xd9, 0x37, 0x68, 0x33, 0x72, 0x37, 0xce, 0x46, 0x28, 0xa3, 0x61, 0x29, 0x86, 0xad, 0xa8, 0x8e, 0x6e, 0x1a, 0x12, 0x29, 0x8a, 0x5e, 0x85, @@ -36890,411 +36927,412 @@ var fileDescriptor_api_d2c1ab7675455c27 = []byte{ 0xeb, 0x5b, 0xeb, 0x55, 0x79, 0x6b, 0x63, 0x6d, 0xe3, 0xde, 0xdb, 0x1b, 0xe2, 0x04, 0x3a, 0x01, 0x22, 0xa7, 0xad, 0x55, 0x1f, 0xc8, 0x77, 0x57, 0xd7, 0x57, 0x1b, 0xa2, 0x80, 0xce, 0xc0, 0x49, 0x4e, 0x95, 0x96, 0x36, 0x6e, 0x57, 0xe5, 0xe5, 0x7b, 0x5b, 0x1b, 0x2b, 0x4b, 0xd2, 0x03, 0x31, - 0x31, 0x97, 0xfa, 0x7f, 0xdf, 0x3d, 0x3b, 0x51, 0xbe, 0x0f, 0x70, 0x1b, 0x3b, 0xdc, 0xac, 0xd0, - 0x32, 0xa4, 0x77, 0x68, 0x6d, 0xb8, 0x61, 0x2f, 0x44, 0x56, 0x3b, 0x60, 0x82, 0xcb, 0x19, 0xa2, - 0x81, 0x0f, 0x0f, 0xe7, 0x05, 0x89, 0x73, 0xb2, 0x2e, 0x2f, 0xff, 0x50, 0x80, 0x1c, 0x15, 0xcc, - 0xda, 0x88, 0x2a, 0x3d, 0x92, 0xcf, 0x8d, 0x54, 0x48, 0xbf, 0x68, 0xb4, 0x08, 0x93, 0xbb, 0x4a, - 0xab, 0x3b, 0x6c, 0xdc, 0xdc, 0x27, 0xef, 0x25, 0x56, 0x0c, 0xbd, 0x0e, 0x79, 0xdd, 0x70, 0xb0, - 0xe1, 0xc8, 0x8c, 0x2d, 0x39, 0x82, 0x2d, 0xc7, 0x4a, 0xd3, 0x87, 0xf2, 0x0f, 0x04, 0x80, 0xcd, - 0x6e, 0x9c, 0xaa, 0x21, 0xe3, 0x7e, 0xac, 0xfa, 0xbb, 0xe3, 0x9e, 0xb5, 0xe2, 0x14, 0xa4, 0x75, - 0xa3, 0xa5, 0x1b, 0xac, 0xfe, 0x19, 0x89, 0x3f, 0xa1, 0x13, 0x30, 0xf9, 0xb0, 0xa5, 0x1b, 0x1a, - 0x35, 0xff, 0x8c, 0xc4, 0x1e, 0xb8, 0xfa, 0x25, 0xc8, 0xd1, 0xba, 0xc7, 0xa8, 0xfd, 0xf2, 0x37, - 0x13, 0x70, 0xb2, 0x62, 0x1a, 0x9a, 0x4e, 0xc6, 0xa1, 0xd2, 0xfa, 0x54, 0xe8, 0xe6, 0x65, 0xc8, - 0xe2, 0xfd, 0xce, 0x98, 0xdd, 0x9b, 0xc1, 0xfb, 0x1d, 0xfa, 0x5f, 0xb4, 0xea, 0xd0, 0x67, 0xe1, - 0xb4, 0xd2, 0x6a, 0x99, 0x7b, 0xb2, 0xbe, 0x2d, 0x6b, 0x26, 0xb6, 0x65, 0xc3, 0x74, 0x64, 0xbc, - 0xaf, 0xdb, 0x0e, 0x75, 0x15, 0x19, 0x69, 0x96, 0xbe, 0x5e, 0xdd, 0x5e, 0x31, 0xb1, 0xbd, 0x61, - 0x3a, 0x55, 0xf2, 0x8a, 0x2b, 0xfc, 0x5d, 0x38, 0xd5, 0xab, 0x9b, 0x38, 0x75, 0xff, 0x57, 0x02, - 0x14, 0x57, 0x0d, 0xdd, 0xf9, 0x54, 0x28, 0xdd, 0xd3, 0x5e, 0x32, 0xa8, 0xbd, 0xab, 0x20, 0x6e, - 0x2b, 0x7a, 0xeb, 0x9e, 0xd1, 0x30, 0xdb, 0x0f, 0x6d, 0xc7, 0x34, 0xb0, 0xcd, 0xd5, 0xdb, 0x47, - 0xe7, 0x3a, 0xbb, 0x0f, 0xd3, 0x5e, 0x9b, 0xe2, 0x54, 0xd6, 0x13, 0x10, 0x57, 0x0d, 0xd5, 0xc2, - 0x6d, 0x6c, 0xc4, 0xaa, 0xad, 0x67, 0x21, 0xab, 0xbb, 0x72, 0xa9, 0xc6, 0x92, 0x92, 0x4f, 0xe0, - 0x6d, 0xea, 0xc2, 0x4c, 0xe0, 0xdb, 0x71, 0x3a, 0xbf, 0x67, 0x20, 0x6b, 0xe0, 0x3d, 0xd9, 0xef, - 0xaf, 0xa4, 0x94, 0x31, 0xf0, 0x1e, 0x73, 0x56, 0x0f, 0xa0, 0xb0, 0x82, 0x5b, 0xd8, 0xc1, 0xf1, - 0x7b, 0xf2, 0x2d, 0x28, 0xba, 0xa2, 0xe3, 0xec, 0xa4, 0xdf, 0x14, 0x00, 0x71, 0xb9, 0x64, 0xf6, - 0x8c, 0xb3, 0x9f, 0xe6, 0x49, 0x74, 0xe0, 0x74, 0x2d, 0x83, 0x4d, 0xf3, 0xcc, 0x4a, 0x81, 0x91, - 0xe8, 0x4c, 0xef, 0x7b, 0xd4, 0x54, 0xd0, 0xa3, 0x7a, 0xd1, 0x0a, 0x89, 0x53, 0xf6, 0x60, 0x36, - 0x54, 0xbd, 0x78, 0xbb, 0x32, 0x45, 0x6b, 0x96, 0x58, 0x48, 0x06, 0x23, 0x33, 0x4a, 0x2c, 0xbf, - 0x0b, 0x33, 0x95, 0x16, 0x56, 0xac, 0xb8, 0xd5, 0xc2, 0xbb, 0xf3, 0x01, 0xa0, 0xa0, 0xf8, 0x38, - 0xbb, 0xf4, 0xb7, 0x04, 0x40, 0x12, 0xde, 0xc5, 0x96, 0x13, 0x7b, 0x97, 0xae, 0x40, 0xce, 0x51, - 0xac, 0x26, 0x76, 0x64, 0x12, 0xce, 0x73, 0x77, 0xf5, 0x5c, 0x40, 0x10, 0x09, 0xea, 0x17, 0x77, - 0x5a, 0xea, 0x62, 0xc3, 0x0d, 0xf7, 0xdd, 0xe0, 0x8c, 0xf1, 0x11, 0x32, 0xd7, 0xc0, 0x3b, 0x30, - 0x1b, 0xaa, 0x65, 0x9c, 0x2a, 0xf8, 0xae, 0x00, 0xb9, 0xba, 0xaa, 0x18, 0x71, 0xb6, 0xfd, 0x4d, - 0xc8, 0xd9, 0xaa, 0x62, 0xc8, 0xdb, 0xa6, 0xd5, 0x56, 0x1c, 0x6a, 0xb2, 0xc5, 0x50, 0xdb, 0xbd, - 0x60, 0x57, 0x55, 0x8c, 0x5b, 0xb4, 0x90, 0x04, 0xb6, 0xf7, 0x7f, 0xd0, 0xaa, 0xef, 0xa4, 0x32, - 0x49, 0x31, 0x55, 0xfe, 0x67, 0x01, 0xf2, 0xac, 0x96, 0x71, 0x5a, 0xf5, 0xcb, 0x90, 0xb2, 0xcc, - 0x3d, 0x66, 0xd5, 0xb9, 0x1b, 0xcf, 0x44, 0x88, 0x58, 0xc3, 0x07, 0xc1, 0xe9, 0x84, 0x16, 0x47, - 0xcb, 0xc0, 0xc3, 0x2e, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x97, 0x61, - 0xfa, 0xa1, 0xe2, 0xa8, 0x3b, 0xb2, 0xc5, 0x2b, 0x49, 0xa6, 0x9e, 0xe4, 0x95, 0xbc, 0x54, 0xa4, - 0x64, 0xb7, 0xea, 0x76, 0xf9, 0xf7, 0x5c, 0x13, 0xb5, 0xf1, 0xa7, 0xbf, 0x9b, 0xfe, 0x45, 0xe0, - 0x96, 0xea, 0x56, 0xf6, 0xbf, 0x5a, 0x6f, 0x7d, 0x27, 0x01, 0xa7, 0x2b, 0x3b, 0x58, 0x7d, 0x54, - 0x31, 0x0d, 0x5b, 0xb7, 0x1d, 0x6c, 0xa8, 0x07, 0x71, 0x76, 0xd9, 0x33, 0x90, 0xdd, 0xd3, 0x9d, - 0x1d, 0x59, 0xd3, 0xb7, 0xb7, 0xa9, 0x4f, 0xc9, 0x48, 0x19, 0x42, 0x58, 0xd1, 0xb7, 0xb7, 0xd1, - 0x4d, 0x48, 0xb5, 0x4d, 0x8d, 0x45, 0x95, 0xc5, 0x1b, 0xf3, 0x11, 0xe2, 0x69, 0xd5, 0xec, 0x6e, - 0x7b, 0xdd, 0xd4, 0xb0, 0x44, 0x0b, 0xa3, 0xb3, 0x00, 0x2a, 0xa1, 0x76, 0x4c, 0xdd, 0x70, 0xf8, - 0xec, 0x12, 0xa0, 0xa0, 0x1a, 0x64, 0x1d, 0x6c, 0xb5, 0x75, 0x43, 0x71, 0x08, 0xea, 0x26, 0xca, - 0xbb, 0x10, 0x59, 0xf1, 0x4e, 0x4b, 0x57, 0x95, 0xbe, 0xe5, 0x03, 0x9f, 0x99, 0xfb, 0xb2, 0xf7, - 0x52, 0x50, 0xea, 0xd7, 0x50, 0x9c, 0x76, 0xb2, 0x09, 0x69, 0x82, 0x45, 0x5b, 0x0e, 0xb7, 0x94, - 0x1b, 0x83, 0x14, 0x11, 0x51, 0x03, 0x8a, 0x69, 0x5b, 0x0e, 0xaf, 0x3c, 0x97, 0x33, 0xf7, 0xa7, - 0x02, 0xa4, 0xd9, 0x0b, 0x74, 0x1d, 0x32, 0x1c, 0x72, 0x6b, 0xb4, 0x8e, 0xc9, 0xe5, 0x53, 0x4f, - 0x0f, 0xe7, 0xa7, 0x18, 0xc0, 0x5e, 0xf9, 0xc8, 0xff, 0x57, 0x9a, 0x62, 0x18, 0x5b, 0x23, 0x7d, - 0x66, 0x3b, 0x8a, 0xe5, 0xd0, 0xa5, 0x0d, 0xd2, 0x67, 0x79, 0x29, 0x43, 0x09, 0x6b, 0xf8, 0x00, - 0xdd, 0x81, 0xb4, 0xed, 0x28, 0x4e, 0xd7, 0xe6, 0xbd, 0x76, 0xa4, 0xca, 0xd6, 0x29, 0xa7, 0xc4, - 0x25, 0x90, 0x20, 0x41, 0xc3, 0x8e, 0xa2, 0xb7, 0x68, 0x37, 0x66, 0x25, 0xfe, 0x54, 0xfe, 0x96, - 0x00, 0x69, 0x56, 0x14, 0x9d, 0x86, 0x59, 0x06, 0xab, 0x57, 0x37, 0x56, 0xaa, 0x8d, 0xaa, 0xb4, - 0xbe, 0xba, 0xb1, 0xd4, 0xa8, 0x8a, 0x13, 0xe8, 0x14, 0x20, 0xf7, 0x45, 0xe5, 0xde, 0x46, 0x7d, - 0xb5, 0xde, 0xa8, 0x6e, 0x10, 0x38, 0x4e, 0x40, 0x3a, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x01, 0x16, - 0x7a, 0xa9, 0x72, 0xbd, 0xb1, 0xd4, 0xa8, 0xcb, 0xd5, 0x7a, 0x63, 0x75, 0x7d, 0xa9, 0x51, 0x5d, - 0x11, 0x93, 0x43, 0x4a, 0x91, 0x8f, 0x48, 0x52, 0xb5, 0xd2, 0x10, 0x53, 0xe5, 0x27, 0x70, 0x52, - 0xc2, 0xaa, 0xd9, 0xee, 0x74, 0x1d, 0x4c, 0x6a, 0x69, 0xc7, 0x39, 0x5e, 0x4e, 0xc3, 0x94, 0x66, - 0x1d, 0xc8, 0x56, 0xd7, 0xe0, 0xa3, 0x25, 0xad, 0x59, 0x07, 0x52, 0xd7, 0xe0, 0xc6, 0xf8, 0xc7, - 0x02, 0x9c, 0xea, 0xfd, 0x78, 0x9c, 0xa6, 0xf8, 0x25, 0xc8, 0x29, 0x9a, 0x86, 0x35, 0x59, 0xc3, - 0x2d, 0x47, 0xe1, 0x41, 0xc0, 0xf5, 0x80, 0x24, 0xbe, 0x2c, 0xb5, 0xc8, 0xd6, 0xa3, 0x16, 0xbd, - 0x65, 0xa9, 0xf5, 0xfb, 0x95, 0x0a, 0xad, 0xcf, 0x0a, 0x61, 0x74, 0x3d, 0x12, 0x95, 0x45, 0x29, - 0xe5, 0x6f, 0xa7, 0xa0, 0x50, 0x35, 0xb4, 0xc6, 0x7e, 0xac, 0x33, 0xc2, 0x29, 0x48, 0xab, 0x66, - 0xbb, 0xad, 0x3b, 0xae, 0xb6, 0xd8, 0x13, 0xfa, 0x1c, 0x64, 0x34, 0xac, 0x68, 0x1e, 0xa4, 0x1f, - 0x15, 0xc9, 0x48, 0x5e, 0x71, 0xf4, 0x15, 0x38, 0x4d, 0x1c, 0xa9, 0x65, 0x28, 0x2d, 0x99, 0x49, - 0x93, 0x1d, 0x4b, 0x6f, 0x36, 0xb1, 0xc5, 0x17, 0xc1, 0xae, 0x44, 0xd4, 0x73, 0x95, 0x73, 0x54, - 0x28, 0x43, 0x83, 0x95, 0x97, 0x4e, 0xea, 0x51, 0x64, 0xf4, 0x45, 0x6f, 0xcd, 0xc4, 0xee, 0x28, - 0x86, 0xcd, 0x9d, 0xd4, 0xa0, 0xb5, 0x35, 0xae, 0x4b, 0x3e, 0x27, 0x10, 0x8a, 0x8d, 0xae, 0x91, - 0xf0, 0xfb, 0x71, 0x57, 0xb7, 0xb0, 0x7c, 0xbd, 0xa3, 0x96, 0xd2, 0xa4, 0xed, 0xcb, 0xc5, 0xa7, - 0x87, 0xf3, 0x20, 0x31, 0xf2, 0xf5, 0xcd, 0x0a, 0x09, 0xc7, 0xd9, 0xff, 0x1d, 0x15, 0x5d, 0x01, - 0xd1, 0x30, 0x65, 0x0b, 0x6f, 0x5b, 0xd8, 0xde, 0xe1, 0x9f, 0xcd, 0x50, 0x8d, 0x15, 0x0d, 0x53, - 0x62, 0x64, 0x26, 0xfa, 0x14, 0xa4, 0x3b, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xcb, 0x34, 0xca, 0x9e, - 0xd0, 0x5b, 0x20, 0xea, 0x86, 0xbc, 0xdd, 0xd2, 0x9b, 0x3b, 0x8e, 0xbc, 0x67, 0xe9, 0x0e, 0xb6, - 0x4b, 0x33, 0xb4, 0xe2, 0x51, 0x86, 0x56, 0xe7, 0xab, 0x93, 0xda, 0xdb, 0xa4, 0x24, 0x6f, 0x42, - 0x51, 0x37, 0x6e, 0x51, 0x7e, 0x4a, 0xb4, 0xbd, 0xe9, 0x78, 0x4a, 0xcc, 0x94, 0xff, 0x4e, 0x80, - 0xa2, 0x6b, 0x1e, 0x71, 0x1a, 0xf4, 0x15, 0x10, 0x4d, 0x03, 0xcb, 0x9d, 0x1d, 0xc5, 0xc6, 0xbc, - 0x3b, 0xf9, 0x9c, 0x51, 0x34, 0x0d, 0xbc, 0x49, 0xc8, 0xac, 0x73, 0xd0, 0x26, 0xcc, 0xd8, 0x8e, - 0xd2, 0xd4, 0x8d, 0xa6, 0xec, 0xad, 0x67, 0xd3, 0x45, 0x89, 0x31, 0xa3, 0x60, 0x91, 0x73, 0x7b, - 0xf4, 0x50, 0xa0, 0xf1, 0x13, 0x01, 0x66, 0x96, 0xb4, 0xb6, 0x6e, 0xd4, 0x3b, 0x2d, 0x3d, 0x56, - 0xc8, 0x7c, 0x01, 0xb2, 0x36, 0x91, 0xe9, 0x7b, 0x6b, 0x1f, 0xee, 0x64, 0xe8, 0x1b, 0xe2, 0xb6, - 0xef, 0xc2, 0x34, 0xde, 0xef, 0xe8, 0x96, 0xe2, 0xe8, 0xa6, 0xc1, 0x22, 0xfc, 0xd4, 0xf8, 0x6d, - 0x2b, 0xfa, 0xbc, 0x7e, 0x94, 0xcf, 0x5b, 0xf6, 0x00, 0x50, 0xb0, 0x61, 0x71, 0x86, 0xfa, 0x32, - 0xcc, 0x52, 0xd1, 0x5b, 0x86, 0x1d, 0xb3, 0xd6, 0xb8, 0x3b, 0xfd, 0x32, 0x9c, 0x08, 0x7f, 0x20, - 0xce, 0xda, 0xbf, 0xcb, 0x7b, 0x7c, 0x1d, 0x5b, 0x9f, 0x10, 0xca, 0x0c, 0x8a, 0x8f, 0xb3, 0xe6, - 0x5f, 0x17, 0xe0, 0x0c, 0x95, 0x4d, 0x77, 0x05, 0xb6, 0xb1, 0x45, 0xb7, 0x4a, 0xe2, 0x34, 0xda, - 0xf3, 0x90, 0x66, 0xa0, 0x91, 0x5a, 0xec, 0xe4, 0x72, 0x8e, 0x04, 0x22, 0x75, 0xc7, 0xb4, 0x48, - 0x20, 0xc2, 0x5f, 0xf1, 0x76, 0x2a, 0x30, 0x17, 0x55, 0x97, 0x98, 0x51, 0xf5, 0x0c, 0x8f, 0x07, - 0x89, 0x89, 0x57, 0x76, 0x48, 0x20, 0x84, 0xaa, 0x90, 0x53, 0xe9, 0x7f, 0xb2, 0x73, 0xd0, 0xc1, - 0x54, 0x7e, 0x71, 0x58, 0x28, 0xc9, 0xd8, 0x1a, 0x07, 0x1d, 0x4c, 0xe2, 0x51, 0xf7, 0x7f, 0xa2, - 0xae, 0x40, 0x53, 0x87, 0x06, 0xa3, 0x74, 0x7c, 0xd1, 0xb2, 0x6e, 0x3c, 0x17, 0xd2, 0xc4, 0x9f, - 0x24, 0xb9, 0x2a, 0xd8, 0x97, 0x38, 0x53, 0xac, 0xe1, 0xc7, 0x3b, 0x70, 0x4a, 0xc3, 0x1d, 0x0b, - 0xab, 0x8a, 0x83, 0x35, 0x39, 0xd8, 0xfc, 0xc4, 0x11, 0x9a, 0x7f, 0xc2, 0x97, 0xe1, 0x53, 0xd1, - 0x03, 0x40, 0x01, 0xd9, 0xac, 0x65, 0x2e, 0xbc, 0x39, 0x8a, 0x52, 0x66, 0x7c, 0x29, 0x8c, 0x6e, - 0xa3, 0x0a, 0x64, 0xf0, 0x7e, 0x47, 0xa6, 0xfb, 0x85, 0xa9, 0x23, 0xee, 0x17, 0x4e, 0xe1, 0xfd, - 0x0e, 0x21, 0xa2, 0x2d, 0x32, 0xc3, 0xb9, 0x13, 0x3f, 0xad, 0xb6, 0x3d, 0x1a, 0x3f, 0xf8, 0xf6, - 0xc2, 0xc5, 0x4d, 0x7b, 0x73, 0x3e, 0x13, 0xc1, 0xfb, 0xee, 0x7d, 0x01, 0x9e, 0x89, 0xec, 0xbb, - 0x38, 0x27, 0x3b, 0x77, 0xcb, 0x34, 0x71, 0x9c, 0x2d, 0xd3, 0xf2, 0xef, 0xbb, 0xa3, 0x5e, 0xc2, - 0x2d, 0x93, 0xa8, 0xf7, 0x13, 0x58, 0x62, 0x9a, 0x72, 0xbb, 0x3d, 0x71, 0xe4, 0x6e, 0x77, 0x59, - 0x7b, 0xdc, 0x42, 0x4f, 0x65, 0xe3, 0x74, 0x0b, 0xbf, 0x2e, 0xc0, 0x6c, 0x0d, 0x2b, 0x96, 0xf3, - 0x10, 0x2b, 0x4e, 0xcc, 0x81, 0xeb, 0xcb, 0x90, 0x34, 0xcc, 0xbd, 0xa3, 0xac, 0xb2, 0x91, 0xf2, - 0xfe, 0xb4, 0x15, 0xae, 0x57, 0x9c, 0xad, 0xfe, 0x8b, 0x04, 0x64, 0x6f, 0x57, 0xe2, 0x6c, 0xeb, - 0x1b, 0x7c, 0x2d, 0x96, 0x0d, 0xf5, 0x28, 0xb3, 0xf4, 0xbe, 0xb7, 0x78, 0xbb, 0xb2, 0x86, 0x0f, - 0x5c, 0xb3, 0x24, 0x5c, 0x68, 0x09, 0xb2, 0xce, 0x0e, 0x89, 0x4f, 0xcd, 0x96, 0x76, 0x94, 0x98, - 0xc5, 0xe7, 0x9a, 0x7b, 0x04, 0x93, 0x54, 0xae, 0xbb, 0x9b, 0x2f, 0x44, 0xec, 0xe6, 0x93, 0xcf, - 0x78, 0x61, 0x5f, 0xe2, 0x28, 0x9f, 0x71, 0x09, 0xac, 0x73, 0xbc, 0xd8, 0x68, 0x52, 0x4c, 0x97, - 0xdf, 0x02, 0x20, 0x4d, 0x8b, 0xb3, 0x7b, 0x7e, 0x35, 0x09, 0xc5, 0xcd, 0xae, 0xbd, 0x13, 0xb3, - 0x3d, 0x56, 0x00, 0x3a, 0x5d, 0x7b, 0x07, 0x5b, 0xb2, 0xb3, 0x6f, 0xf0, 0xf6, 0x8f, 0xc8, 0x13, - 0x70, 0x15, 0xc0, 0xf8, 0x1a, 0xfb, 0x06, 0xba, 0xc7, 0x85, 0x60, 0xd9, 0x4f, 0x36, 0xb8, 0x3a, - 0x06, 0x78, 0x6c, 0xec, 0x1b, 0xeb, 0xd8, 0x43, 0x8d, 0x4c, 0x20, 0x26, 0x02, 0xdf, 0x80, 0x29, - 0xf2, 0x20, 0x3b, 0xe6, 0x51, 0x7a, 0x3e, 0x4d, 0x78, 0x1a, 0x26, 0x7a, 0x1d, 0xb2, 0x8c, 0x9b, - 0xcc, 0x5f, 0x69, 0x3a, 0x7f, 0x45, 0x35, 0x89, 0x6b, 0x93, 0xce, 0x5c, 0x19, 0xca, 0x4a, 0x66, - 0xab, 0x13, 0x30, 0xb9, 0x6d, 0x5a, 0x2a, 0xa6, 0xd9, 0x05, 0x19, 0x89, 0x3d, 0x04, 0x3b, 0xf7, - 0x4e, 0x2a, 0x93, 0x11, 0xb3, 0x77, 0x52, 0x99, 0xac, 0x08, 0xe5, 0x6f, 0x09, 0x30, 0xed, 0xf5, - 0x4a, 0x9c, 0x2e, 0xbd, 0x12, 0x52, 0xe9, 0xd1, 0xfb, 0x85, 0xa8, 0xb1, 0xfc, 0xd7, 0x34, 0xbe, - 0x51, 0xcd, 0x5d, 0xda, 0x4d, 0x71, 0x9a, 0xcd, 0x32, 0xcb, 0x2b, 0x49, 0x1c, 0xb3, 0xab, 0x69, - 0xa6, 0xc9, 0x75, 0x38, 0xa1, 0xb7, 0x89, 0xcf, 0xd7, 0x9d, 0xd6, 0x01, 0xc7, 0x68, 0x0e, 0x76, - 0xb7, 0x3e, 0x67, 0xfd, 0x77, 0x15, 0xf7, 0x15, 0x77, 0x83, 0x6c, 0x33, 0xc4, 0x6f, 0x56, 0x9c, - 0x7a, 0x5f, 0x85, 0x82, 0xc5, 0x44, 0x93, 0x58, 0xe5, 0x88, 0xaa, 0xcf, 0x7b, 0xac, 0x44, 0xfb, - 0xdf, 0x4f, 0xc0, 0xf4, 0x5b, 0x5d, 0x6c, 0x1d, 0x7c, 0x0a, 0x75, 0x7f, 0x09, 0xa6, 0xf7, 0x14, - 0xdd, 0x91, 0xb7, 0x4d, 0x4b, 0xee, 0x76, 0x34, 0xc5, 0x71, 0x33, 0x20, 0x0a, 0x84, 0x7c, 0xcb, - 0xb4, 0xb6, 0x28, 0x11, 0x61, 0x40, 0x8f, 0x0c, 0x73, 0xcf, 0x90, 0x09, 0x99, 0x42, 0xe4, 0x7d, - 0x83, 0x2f, 0x29, 0x2f, 0xbf, 0xf2, 0xb7, 0x87, 0xf3, 0x37, 0xc7, 0xca, 0x69, 0xa2, 0x69, 0x61, - 0xdd, 0xae, 0xae, 0x2d, 0x6e, 0x6d, 0xad, 0xae, 0x48, 0x22, 0x15, 0xf9, 0x36, 0x93, 0xd8, 0xd8, - 0x37, 0xdc, 0xa9, 0xfd, 0x23, 0x01, 0x44, 0x5f, 0x61, 0x71, 0xf6, 0x6a, 0x15, 0x72, 0x8f, 0xbb, - 0xd8, 0xd2, 0x8f, 0xd1, 0xa7, 0xc0, 0x19, 0x89, 0x5b, 0x7a, 0x07, 0xf2, 0x21, 0x3d, 0x24, 0x3f, - 0x9e, 0x1e, 0x72, 0x7b, 0xbe, 0x0a, 0xca, 0x3f, 0x16, 0x00, 0xd1, 0xc6, 0xaf, 0xb2, 0xd5, 0xfc, - 0x4f, 0x99, 0xc1, 0x5c, 0x01, 0x91, 0x66, 0x19, 0xca, 0xfa, 0xb6, 0xdc, 0xd6, 0x6d, 0x5b, 0x37, - 0x9a, 0xdc, 0x62, 0x8a, 0x94, 0xbe, 0xba, 0xbd, 0xce, 0xa8, 0xbc, 0x2f, 0xff, 0x27, 0xcc, 0x86, - 0x5a, 0x13, 0x67, 0x6f, 0x9e, 0x83, 0xfc, 0xb6, 0xd9, 0x35, 0x34, 0x99, 0x2d, 0x8d, 0xf1, 0x25, - 0xc0, 0x1c, 0xa5, 0xb1, 0xef, 0x95, 0xff, 0x35, 0x01, 0x27, 0x24, 0x6c, 0x9b, 0xad, 0x5d, 0x1c, - 0xbf, 0x3e, 0xef, 0x01, 0xdf, 0x72, 0x91, 0x3f, 0x8e, 0x5a, 0xb3, 0x4c, 0x06, 0x9b, 0xee, 0xc2, - 0x6b, 0xeb, 0x17, 0x86, 0x5b, 0x66, 0xff, 0x6a, 0x3a, 0x5f, 0xb9, 0x4b, 0x85, 0x56, 0xee, 0x76, - 0x61, 0x5a, 0x6f, 0x1a, 0x26, 0x71, 0x64, 0x36, 0x7e, 0x6c, 0x74, 0xdb, 0x2e, 0xac, 0x79, 0x79, - 0x8c, 0xba, 0xae, 0x32, 0xce, 0x3a, 0x7e, 0xbc, 0xd1, 0x6d, 0xd3, 0x18, 0x7b, 0xf9, 0x14, 0xa9, - 0xf6, 0xd3, 0xc3, 0xf9, 0x62, 0xe8, 0x9d, 0x2d, 0x15, 0x75, 0xef, 0x99, 0x7c, 0x84, 0x1b, 0xc0, - 0x7f, 0x87, 0x93, 0x3d, 0x1d, 0x10, 0x67, 0x34, 0xf4, 0x97, 0x49, 0x38, 0x13, 0x16, 0x1f, 0x37, - 0x66, 0xf9, 0x4f, 0xd2, 0xc9, 0x35, 0x28, 0xb4, 0x75, 0xe3, 0x78, 0x2b, 0x97, 0xf9, 0xb6, 0x6e, - 0x78, 0xb4, 0x28, 0x73, 0x49, 0xff, 0xfb, 0x99, 0x8b, 0x02, 0x73, 0x51, 0xfd, 0x19, 0xa7, 0xcd, - 0xbc, 0x27, 0x40, 0x3e, 0xee, 0x35, 0xb9, 0xe3, 0xa5, 0x79, 0xf1, 0x36, 0x37, 0xa0, 0xf0, 0x09, - 0x2c, 0xe2, 0xfd, 0x8e, 0x00, 0xa8, 0x61, 0x75, 0x0d, 0x02, 0x8e, 0xef, 0x9a, 0xcd, 0x38, 0x1b, - 0x7b, 0x02, 0x26, 0x75, 0x43, 0xc3, 0xfb, 0xb4, 0xb1, 0x29, 0x89, 0x3d, 0x84, 0x76, 0x1a, 0x93, - 0x63, 0xed, 0x34, 0xfa, 0xd9, 0x22, 0xa1, 0x8a, 0xc6, 0xa9, 0x85, 0xef, 0x27, 0x60, 0x96, 0x37, - 0x27, 0xf6, 0x45, 0xcc, 0x63, 0xe5, 0x98, 0xa3, 0xcf, 0x03, 0x74, 0x2c, 0xbc, 0x2b, 0x33, 0xd6, - 0xe4, 0x58, 0xac, 0x59, 0xc2, 0x41, 0x09, 0xe8, 0x4b, 0x30, 0x4d, 0x06, 0x7a, 0xc7, 0x32, 0x3b, - 0xa6, 0x4d, 0xe2, 0x18, 0x7b, 0x3c, 0x68, 0x34, 0xf3, 0xf4, 0x70, 0xbe, 0xb0, 0xae, 0x1b, 0x9b, - 0x9c, 0xb1, 0x51, 0x97, 0x88, 0xc7, 0xf0, 0x1e, 0xdd, 0x01, 0xf8, 0x37, 0x02, 0x9c, 0xf8, 0xc4, - 0x96, 0x7d, 0xff, 0x23, 0x34, 0xe6, 0xcd, 0x43, 0x22, 0x7d, 0x5c, 0x35, 0xb6, 0xcd, 0xf8, 0x17, - 0xe3, 0xdf, 0x13, 0x60, 0x26, 0x20, 0x3e, 0xce, 0x28, 0xe7, 0x78, 0x27, 0x19, 0xbe, 0x4c, 0xe2, - 0x9e, 0xa0, 0xd9, 0xc7, 0x39, 0xa8, 0xfe, 0x2c, 0x01, 0xa7, 0x2a, 0x6c, 0x0f, 0xda, 0x4d, 0xd0, - 0x88, 0xd3, 0x4a, 0x4a, 0x30, 0xb5, 0x8b, 0x2d, 0x5b, 0x37, 0xd9, 0x7c, 0x5b, 0x90, 0xdc, 0x47, - 0x34, 0x07, 0x19, 0xdb, 0x50, 0x3a, 0xf6, 0x8e, 0xe9, 0xee, 0xe2, 0x79, 0xcf, 0x5e, 0x32, 0xc9, - 0xe4, 0xf1, 0x93, 0x49, 0xd2, 0xc3, 0x93, 0x49, 0xa6, 0x3e, 0x76, 0x32, 0x09, 0xdf, 0x32, 0xfb, - 0x91, 0x00, 0xa7, 0xfb, 0xf4, 0x17, 0xa7, 0xcd, 0x7c, 0x15, 0x72, 0x2a, 0x17, 0x4c, 0xbc, 0x31, - 0xdb, 0x15, 0x5c, 0x25, 0xc5, 0x8e, 0x89, 0x51, 0x9e, 0x1e, 0xce, 0x83, 0x5b, 0xd5, 0xd5, 0x15, - 0xae, 0x22, 0xf2, 0xbf, 0x56, 0xfe, 0xff, 0x39, 0x98, 0xae, 0xee, 0xb3, 0x35, 0xef, 0x3a, 0x0b, - 0x0b, 0xd0, 0x2d, 0xc8, 0x74, 0x2c, 0x73, 0x57, 0x77, 0x9b, 0x51, 0x0c, 0x85, 0x4c, 0x6e, 0x33, - 0x7a, 0xb8, 0x36, 0x39, 0x87, 0xe4, 0xf1, 0xa2, 0x06, 0x64, 0xef, 0x9a, 0xaa, 0xd2, 0xba, 0xa5, - 0xb7, 0x5c, 0xfb, 0x7f, 0x69, 0xb4, 0xa0, 0x45, 0x8f, 0x67, 0x53, 0x71, 0x76, 0xdc, 0xae, 0xf0, - 0x88, 0x68, 0x15, 0x32, 0x35, 0xc7, 0xe9, 0x90, 0x97, 0xdc, 0x9b, 0x5c, 0x1e, 0x43, 0x28, 0x61, - 0xe1, 0xb2, 0x3c, 0x76, 0xd4, 0x80, 0x99, 0xdb, 0xa6, 0xd9, 0x6c, 0xe1, 0x4a, 0xcb, 0xec, 0x6a, - 0x15, 0xd3, 0xd8, 0xd6, 0x9b, 0xdc, 0x1f, 0x5f, 0x1a, 0x43, 0xe6, 0xed, 0x4a, 0x5d, 0xea, 0x17, - 0x80, 0x96, 0x20, 0x53, 0xbf, 0xc9, 0x85, 0xb1, 0x38, 0xee, 0xe2, 0x18, 0xc2, 0xea, 0x37, 0x25, - 0x8f, 0x0d, 0xdd, 0x81, 0xdc, 0xd2, 0x93, 0xae, 0x85, 0xb9, 0x94, 0xf4, 0xc0, 0xcc, 0x85, 0x5e, - 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x3a, 0x14, 0xdf, 0x36, 0xad, 0x47, 0x2d, 0x53, 0x71, 0x5b, - 0x38, 0x45, 0xc5, 0x7d, 0x66, 0x0c, 0x71, 0x2e, 0xa3, 0xd4, 0x23, 0x62, 0xee, 0x4b, 0x50, 0x08, - 0x75, 0x13, 0x42, 0x90, 0xea, 0x90, 0x1e, 0x11, 0x68, 0x2a, 0x10, 0xfd, 0x1f, 0xbd, 0x08, 0x53, - 0x86, 0xa9, 0x61, 0xd7, 0x86, 0x0b, 0xcb, 0x27, 0x9e, 0x1e, 0xce, 0xa7, 0x37, 0x4c, 0x8d, 0x05, - 0x14, 0xfc, 0x3f, 0x29, 0x4d, 0x0a, 0xb9, 0xe1, 0xc4, 0xdc, 0x25, 0x48, 0x91, 0xfe, 0x21, 0x6e, - 0xe4, 0xa1, 0x62, 0xe3, 0x2d, 0x4b, 0xe7, 0x32, 0xdd, 0x47, 0x5e, 0xee, 0xa7, 0x02, 0x24, 0xea, - 0x37, 0x49, 0x44, 0xfd, 0xb0, 0xab, 0x3e, 0xc2, 0x0e, 0x2f, 0xc5, 0x9f, 0x68, 0xa4, 0x6d, 0xe1, - 0x6d, 0x9d, 0x45, 0x39, 0x59, 0x89, 0x3f, 0xa1, 0xe7, 0x00, 0x14, 0x55, 0xc5, 0xb6, 0x2d, 0xbb, - 0x07, 0xc3, 0xb2, 0x52, 0x96, 0x51, 0xd6, 0xf0, 0x01, 0x61, 0xb3, 0xb1, 0x6a, 0x61, 0xc7, 0xcd, - 0x69, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xed, 0x8e, 0xec, 0x98, 0x8f, 0xb0, 0x41, 0x7b, 0x35, 0x4b, - 0xdc, 0x43, 0xbb, 0xd3, 0x20, 0x04, 0xe2, 0xd9, 0xb0, 0xa1, 0xf9, 0x6e, 0x28, 0x2b, 0x79, 0xcf, - 0x44, 0xa4, 0x85, 0x9b, 0x3a, 0x3f, 0xe9, 0x94, 0x95, 0xf8, 0x13, 0xd1, 0x98, 0xd2, 0x75, 0x76, - 0x68, 0x22, 0x47, 0x56, 0xa2, 0xff, 0xf3, 0xa6, 0x7d, 0x53, 0x80, 0xe4, 0xed, 0x4a, 0xfd, 0xc8, - 0x6d, 0x73, 0x25, 0x26, 0x7d, 0x89, 0x34, 0x95, 0x50, 0x6f, 0xb5, 0x74, 0xa3, 0x49, 0x82, 0x8e, - 0xaf, 0x62, 0xd5, 0x6d, 0x59, 0x91, 0x93, 0x37, 0x19, 0x15, 0x2d, 0x40, 0x4e, 0xb5, 0xb0, 0x86, - 0x0d, 0x47, 0x57, 0x5a, 0x36, 0x6f, 0x62, 0x90, 0xc4, 0x2b, 0xf7, 0x35, 0x01, 0x26, 0xa9, 0x79, - 0xa1, 0x67, 0x21, 0xab, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0xf7, 0x13, 0x59, 0xc9, 0x27, 0x0c, 0xac, - 0xe4, 0x39, 0xc8, 0x2b, 0xaa, 0x6a, 0x76, 0x0d, 0x47, 0x36, 0x94, 0x36, 0xe6, 0x95, 0xcd, 0x71, - 0xda, 0x86, 0xd2, 0xc6, 0x68, 0x1e, 0xdc, 0x47, 0xef, 0x78, 0x5e, 0x56, 0x02, 0x4e, 0x5a, 0xc3, - 0x07, 0xbc, 0x26, 0x3f, 0x12, 0x20, 0xe3, 0x9a, 0x25, 0xa9, 0x4c, 0x13, 0x1b, 0xd8, 0x52, 0x1c, - 0xd3, 0xab, 0x8c, 0x47, 0xe8, 0x9d, 0x93, 0xb2, 0xfe, 0x9c, 0x74, 0x02, 0x26, 0x1d, 0xe5, 0x61, - 0xcb, 0xad, 0x07, 0x7b, 0xa0, 0xcb, 0xc7, 0x2d, 0xa5, 0xc9, 0xd6, 0xc8, 0xb2, 0x12, 0x7b, 0x20, - 0x4d, 0xe2, 0xb9, 0xab, 0x4c, 0x3b, 0xfc, 0x89, 0xd4, 0x97, 0xa5, 0x6b, 0x3e, 0xc4, 0x4d, 0xdd, - 0xa0, 0x06, 0x90, 0x94, 0x80, 0x92, 0x96, 0x09, 0x05, 0x3d, 0x03, 0x59, 0x56, 0x00, 0x1b, 0x1a, - 0xb5, 0x82, 0xa4, 0x94, 0xa1, 0x84, 0xaa, 0x7b, 0x1e, 0x89, 0x07, 0x22, 0xdf, 0x13, 0x60, 0x86, - 0xe5, 0xdc, 0xb0, 0x3c, 0xcf, 0xf8, 0x66, 0xe5, 0xd7, 0x20, 0xab, 0x29, 0x8e, 0xc2, 0x8e, 0x03, - 0x26, 0x86, 0x1e, 0x07, 0x74, 0xdd, 0x24, 0x29, 0x4f, 0x8f, 0x04, 0x22, 0x48, 0x91, 0xff, 0xd9, - 0x31, 0x4a, 0x89, 0xfe, 0xef, 0x67, 0x31, 0x04, 0xab, 0x1b, 0x67, 0x94, 0xf2, 0xe1, 0x24, 0x14, - 0xaa, 0xfb, 0x1d, 0xd3, 0x8a, 0x79, 0x11, 0x6d, 0x8a, 0x83, 0xdf, 0x21, 0xdb, 0xac, 0x3d, 0x5e, - 0xd0, 0xdd, 0xc1, 0xe4, 0x8c, 0x68, 0x19, 0x80, 0x25, 0x58, 0xd2, 0x3c, 0x9c, 0xe4, 0x11, 0x36, - 0x9b, 0x28, 0x1b, 0xa1, 0xa2, 0x0d, 0xc8, 0xb5, 0x77, 0x55, 0x55, 0xde, 0xd6, 0x5b, 0x0e, 0x4f, - 0x4d, 0x8b, 0xce, 0x85, 0x5e, 0xbf, 0x5f, 0xa9, 0xdc, 0xa2, 0x85, 0x58, 0x86, 0x98, 0xff, 0x2c, - 0x01, 0x91, 0xc0, 0xfe, 0x47, 0x2f, 0x00, 0x3f, 0xbe, 0x21, 0xdb, 0xee, 0x61, 0xac, 0xe5, 0xc2, - 0xd3, 0xc3, 0xf9, 0xac, 0x44, 0xa9, 0xf5, 0x7a, 0x43, 0xca, 0xb2, 0x02, 0x75, 0xdb, 0x41, 0xe7, - 0xa1, 0x60, 0xb6, 0x75, 0x47, 0x76, 0xe3, 0x00, 0x1e, 0x3a, 0xe5, 0x09, 0xd1, 0x8d, 0x13, 0x50, - 0x03, 0x2e, 0x63, 0x83, 0x8c, 0x06, 0xda, 0x4e, 0xf9, 0x21, 0x5b, 0xab, 0x73, 0xd8, 0x88, 0x92, - 0xcd, 0x8e, 0xa3, 0xb7, 0xf5, 0x27, 0x74, 0xa3, 0x97, 0x6f, 0xb2, 0x9c, 0x67, 0xc5, 0x49, 0xfb, - 0x96, 0xe9, 0x22, 0x1e, 0x2f, 0x7b, 0x2f, 0x50, 0x14, 0x7d, 0x4d, 0x80, 0x53, 0x5c, 0x91, 0xf2, - 0xc3, 0x03, 0xb9, 0x45, 0x26, 0x11, 0xdd, 0x39, 0x90, 0x1f, 0xed, 0x96, 0x32, 0x34, 0x40, 0xfb, - 0x5c, 0x64, 0x87, 0x04, 0xec, 0x60, 0xd1, 0xed, 0x96, 0x83, 0xbb, 0x9c, 0x79, 0x6d, 0xb7, 0x6a, - 0x38, 0xd6, 0xc1, 0xf2, 0xe9, 0xa7, 0x87, 0xf3, 0xb3, 0xfd, 0x6f, 0xef, 0x4b, 0xb3, 0x76, 0x3f, - 0xcb, 0xdc, 0x57, 0xa1, 0x34, 0x48, 0x12, 0x12, 0xfd, 0x9d, 0xc6, 0x2c, 0xdb, 0x60, 0x7c, 0x35, - 0xbc, 0x42, 0x30, 0x86, 0xd9, 0xb8, 0xab, 0x04, 0x89, 0x57, 0xdd, 0xd1, 0xfd, 0x0d, 0x01, 0x0a, - 0xcb, 0xdd, 0xd6, 0xa3, 0x7b, 0x9d, 0x7a, 0xb7, 0xdd, 0x56, 0xac, 0x03, 0xe2, 0x18, 0xd8, 0xa8, - 0xd4, 0x9f, 0xb0, 0x14, 0x95, 0x24, 0x1f, 0x76, 0xfa, 0x13, 0x4c, 0x86, 0x1d, 0x4f, 0x40, 0x27, - 0x74, 0x96, 0x5d, 0x7e, 0x1e, 0x0a, 0x14, 0xae, 0xcb, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0x55, 0xa8, - 0xa4, 0x94, 0xa7, 0xc4, 0x2a, 0xa3, 0xa1, 0x8b, 0x50, 0xb4, 0x0f, 0x6c, 0x07, 0xb7, 0x65, 0x76, - 0x78, 0x9b, 0x61, 0xcc, 0xa4, 0x54, 0x60, 0x54, 0x89, 0x11, 0xcb, 0x3f, 0x4b, 0x42, 0xd1, 0xd5, - 0x6c, 0x9c, 0xe1, 0xeb, 0x32, 0x4c, 0x6e, 0xeb, 0x2d, 0xec, 0x66, 0x09, 0x5c, 0x1a, 0xd2, 0xa1, - 0x3c, 0xb1, 0x98, 0x44, 0x11, 0x2e, 0x00, 0xa2, 0xac, 0x71, 0x8c, 0xb1, 0xb9, 0xff, 0x9d, 0x80, - 0x14, 0x8d, 0x18, 0xaf, 0x43, 0x8a, 0xba, 0x3d, 0x61, 0x1c, 0xb7, 0x47, 0x8b, 0x7a, 0xe1, 0x4c, - 0x22, 0x10, 0xce, 0x90, 0xd8, 0x60, 0x47, 0x79, 0xf9, 0xfa, 0x0d, 0x3a, 0xbe, 0xf2, 0x12, 0x7f, - 0x42, 0xcb, 0x34, 0x7d, 0xc5, 0xb4, 0x1c, 0xac, 0xf1, 0x48, 0x2d, 0xca, 0x33, 0x85, 0x3a, 0xde, - 0x75, 0xb1, 0x2e, 0x1f, 0x3a, 0x03, 0x49, 0x32, 0x70, 0xa7, 0xd8, 0xd6, 0xf6, 0xd3, 0xc3, 0xf9, - 0x24, 0x19, 0xb2, 0x84, 0x86, 0xae, 0x41, 0x2e, 0x3c, 0x4a, 0x84, 0x2b, 0x59, 0xe6, 0x0b, 0x02, - 0x16, 0x0e, 0x2d, 0xcf, 0x82, 0x19, 0x4a, 0xb9, 0x93, 0xca, 0xa4, 0xc4, 0xc9, 0xf2, 0x1f, 0xa5, - 0xa0, 0xb0, 0xda, 0x8e, 0xdb, 0x8b, 0x2e, 0x85, 0x7b, 0x38, 0x2a, 0xbc, 0x0d, 0x7d, 0x34, 0xa2, - 0x83, 0x43, 0xf3, 0x51, 0xf2, 0x68, 0xf3, 0xd1, 0x2a, 0x89, 0xa8, 0xf8, 0x01, 0xf5, 0xe4, 0x80, - 0x48, 0x36, 0xfc, 0xfd, 0x06, 0x71, 0x4c, 0x12, 0xe1, 0xf1, 0x53, 0xed, 0x69, 0x7a, 0xc2, 0x9b, - 0x34, 0x70, 0x63, 0x56, 0x96, 0x1e, 0xdf, 0xca, 0xa6, 0xb0, 0xa1, 0x51, 0x1b, 0x7b, 0xc2, 0x4d, - 0xec, 0x35, 0x48, 0x6a, 0xba, 0x35, 0xe4, 0xb6, 0x83, 0xe8, 0x39, 0x85, 0x30, 0x8d, 0xb0, 0xb5, - 0x54, 0xd0, 0xd6, 0x82, 0x38, 0x74, 0xee, 0x1e, 0x80, 0xdf, 0x2e, 0xb4, 0x00, 0x69, 0xb3, 0xa5, - 0xb9, 0xe7, 0x04, 0x0a, 0xcb, 0xd9, 0xa7, 0x87, 0xf3, 0x93, 0xf7, 0x5a, 0xda, 0xea, 0x8a, 0x34, - 0x69, 0xb6, 0xb4, 0x55, 0x8d, 0x9e, 0xec, 0xc7, 0x7b, 0xb2, 0x97, 0x63, 0x94, 0x97, 0xa6, 0x0c, - 0xbc, 0x47, 0x50, 0x6f, 0x4f, 0xee, 0x03, 0x31, 0x9c, 0xef, 0x08, 0x50, 0x74, 0x75, 0x18, 0xaf, - 0x73, 0xc8, 0xe8, 0x6d, 0x3e, 0x58, 0x92, 0x47, 0x1b, 0x2c, 0x2e, 0x1f, 0x3f, 0x7f, 0xf8, 0x75, - 0x81, 0xe7, 0x97, 0xd6, 0x55, 0xc5, 0x21, 0xf3, 0x61, 0x8c, 0x06, 0xfe, 0x3c, 0x88, 0x96, 0x62, - 0x68, 0x66, 0x5b, 0x7f, 0x82, 0xd9, 0xc2, 0x95, 0xcd, 0xf7, 0xa7, 0xa6, 0x3d, 0x3a, 0x5d, 0x99, - 0x71, 0xd7, 0xdd, 0x7e, 0x29, 0xf0, 0x5c, 0x54, 0xaf, 0x32, 0x71, 0x2a, 0x6d, 0x0d, 0xd2, 0x16, - 0xcb, 0x68, 0x63, 0x03, 0xee, 0xc5, 0x08, 0x21, 0x51, 0x5f, 0x67, 0x09, 0x63, 0x9e, 0xc9, 0x53, - 0x11, 0x73, 0x5f, 0x84, 0x49, 0x4a, 0x3e, 0x86, 0x5b, 0xe4, 0x9a, 0xff, 0x87, 0x04, 0x5c, 0xa0, - 0x9f, 0xbb, 0x8f, 0x2d, 0x7d, 0xfb, 0x60, 0xd3, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, 0xf3, 0xf1, - 0x63, 0xf5, 0x35, 0xd9, 0x8e, 0xfb, 0x81, 0x23, 0x65, 0xf6, 0x78, 0x5c, 0x68, 0x0d, 0xa6, 0xd9, - 0x44, 0x28, 0x2b, 0x2d, 0x7d, 0x17, 0xcb, 0x8a, 0x73, 0x94, 0x19, 0xa5, 0xc0, 0x78, 0x97, 0x08, - 0xeb, 0x92, 0x83, 0x34, 0xc8, 0x72, 0x61, 0xba, 0xc6, 0x6f, 0x0e, 0xb9, 0xfd, 0xf1, 0x96, 0x66, - 0x32, 0x6c, 0x42, 0x5e, 0x5d, 0x91, 0x32, 0x4c, 0xb2, 0xb7, 0xb4, 0xfe, 0x13, 0x01, 0x2e, 0x8e, - 0x50, 0x74, 0x9c, 0x66, 0x36, 0x07, 0x99, 0x5d, 0xf2, 0x21, 0x9d, 0x6b, 0x3a, 0x23, 0x79, 0xcf, - 0x68, 0x1d, 0x0a, 0xdb, 0x8a, 0xde, 0xc2, 0x9a, 0xcc, 0x2d, 0x71, 0x70, 0x3a, 0x58, 0x74, 0x96, - 0x62, 0x9e, 0xb1, 0xd3, 0x97, 0x76, 0xf9, 0xb7, 0x13, 0x30, 0xb3, 0xa4, 0x69, 0xf5, 0x3a, 0xf7, - 0x60, 0xf1, 0xd9, 0x8b, 0x0b, 0x56, 0x12, 0x3e, 0x58, 0x41, 0x2f, 0x02, 0xd2, 0x74, 0x9b, 0x5d, - 0x5d, 0x60, 0xef, 0x28, 0x9a, 0xb9, 0xe7, 0x6f, 0x7c, 0xcf, 0xb8, 0x6f, 0xea, 0xee, 0x0b, 0xf4, - 0x0e, 0xd0, 0xd0, 0x5a, 0xb6, 0x1d, 0xc5, 0x5b, 0x9f, 0x7f, 0xe1, 0x28, 0xa7, 0x68, 0x58, 0xe8, - 0xed, 0x3d, 0x4a, 0x59, 0x22, 0x8e, 0xfe, 0x8b, 0xae, 0x80, 0xa8, 0x13, 0x0d, 0x38, 0xb2, 0x62, - 0xbb, 0x07, 0x31, 0xd8, 0xdd, 0x09, 0x45, 0x46, 0x5f, 0xb2, 0x83, 0xe7, 0x2b, 0x58, 0x9e, 0xb8, - 0xaf, 0xa7, 0x38, 0x11, 0xd6, 0xef, 0x0a, 0x50, 0xe4, 0x87, 0x47, 0xe2, 0xec, 0x80, 0x5b, 0x90, - 0x77, 0x4f, 0xaa, 0x6c, 0x5b, 0x66, 0xfb, 0x28, 0x43, 0x2d, 0xc7, 0x19, 0x6f, 0x59, 0x66, 0x3b, - 0x74, 0xd6, 0xfc, 0x3e, 0x4c, 0x7b, 0x35, 0x8d, 0x53, 0x05, 0xdf, 0xa3, 0x07, 0x48, 0x99, 0xe0, - 0xb8, 0xb7, 0x9e, 0x3f, 0x09, 0x3d, 0xd0, 0x7d, 0x81, 0x60, 0x75, 0xe3, 0x54, 0xc6, 0x2f, 0x05, - 0x28, 0xd6, 0xbb, 0x0f, 0xd9, 0x35, 0x3a, 0xf1, 0xe9, 0xa1, 0x0a, 0xd9, 0x16, 0xde, 0x76, 0xe4, - 0x63, 0xe5, 0x36, 0x67, 0x08, 0x2b, 0xcd, 0xef, 0xbe, 0x0d, 0x60, 0xd1, 0xd3, 0x4b, 0x54, 0x4e, - 0xf2, 0x88, 0x72, 0xb2, 0x94, 0xd7, 0x8f, 0x75, 0xca, 0x7f, 0x98, 0x80, 0x69, 0xaf, 0xb1, 0x71, - 0x3a, 0xd1, 0xff, 0x11, 0x72, 0x1e, 0xc9, 0x63, 0x38, 0x8f, 0x19, 0xbe, 0xe5, 0x1e, 0xed, 0x40, - 0x16, 0x61, 0x96, 0x06, 0x24, 0xb2, 0xd2, 0xe9, 0xb4, 0x74, 0xac, 0xc9, 0x6c, 0x2f, 0x37, 0x45, - 0xf7, 0x72, 0x67, 0xe8, 0xab, 0x25, 0xf6, 0x66, 0x95, 0xee, 0xeb, 0xde, 0x82, 0xfc, 0xb6, 0x85, - 0xf1, 0x13, 0x2c, 0x53, 0x64, 0x74, 0x94, 0xcc, 0x82, 0x1c, 0x63, 0xac, 0x13, 0x3e, 0x6e, 0x80, - 0xef, 0xc2, 0x0c, 0x55, 0x70, 0xdc, 0x27, 0x27, 0x79, 0xaf, 0xfc, 0xa3, 0x00, 0x28, 0x28, 0xff, - 0x93, 0xeb, 0x98, 0x44, 0xec, 0x1d, 0xf3, 0x02, 0x20, 0x96, 0x64, 0x66, 0xcb, 0x1d, 0x6c, 0xc9, - 0x36, 0x56, 0x4d, 0x7e, 0x03, 0x8c, 0x20, 0x89, 0xfc, 0xcd, 0x26, 0xb6, 0xea, 0x94, 0x5e, 0x7e, - 0x6f, 0x0e, 0xf2, 0x5c, 0x27, 0x5b, 0x86, 0x6e, 0x1a, 0xe8, 0x3a, 0x24, 0x9b, 0x7c, 0xa5, 0x37, - 0x17, 0xb9, 0x12, 0xe4, 0xdf, 0x42, 0x55, 0x9b, 0x90, 0x48, 0x59, 0xc2, 0xd2, 0xe9, 0x3a, 0x11, - 0x41, 0x91, 0x9f, 0x1b, 0x1b, 0x64, 0xe9, 0x74, 0x1d, 0x54, 0x87, 0x69, 0xd5, 0xbf, 0x85, 0x47, - 0x26, 0xec, 0xc9, 0x81, 0x9b, 0x0b, 0x91, 0x77, 0x19, 0xd5, 0x26, 0xa4, 0xa2, 0x1a, 0x7a, 0x81, - 0x2a, 0xc1, 0x6b, 0x5f, 0xd8, 0x74, 0x79, 0x3e, 0xf2, 0x94, 0x65, 0xf8, 0xca, 0x99, 0xda, 0x44, - 0xe0, 0x76, 0x18, 0xf4, 0x1a, 0xa4, 0x35, 0x7a, 0x9d, 0x08, 0xb7, 0xd0, 0x28, 0x23, 0x0a, 0xdd, - 0xe0, 0x52, 0x9b, 0x90, 0x38, 0x07, 0xba, 0x03, 0x79, 0xf6, 0x1f, 0x0b, 0x4e, 0x38, 0x92, 0xbb, - 0x38, 0x58, 0x42, 0xc0, 0xd7, 0xd7, 0x26, 0xa4, 0x9c, 0xe6, 0x53, 0xd1, 0x67, 0x21, 0x65, 0xab, - 0x8a, 0xc1, 0x37, 0x49, 0xce, 0x0e, 0xb8, 0x9e, 0xc0, 0x67, 0xa6, 0xa5, 0xd1, 0xeb, 0xec, 0x36, - 0x39, 0x67, 0x9f, 0x1d, 0xbc, 0x8c, 0xae, 0x7e, 0xe8, 0x00, 0x2d, 0xa9, 0x3e, 0xa6, 0x04, 0x74, - 0x1b, 0x72, 0x0a, 0x89, 0xf2, 0x64, 0x7a, 0x8c, 0xad, 0x04, 0x03, 0x0f, 0x1a, 0xf5, 0x1d, 0x41, - 0xac, 0xd1, 0x53, 0xba, 0x2e, 0xd1, 0x17, 0xd4, 0xc6, 0x56, 0x13, 0x97, 0x72, 0xc3, 0x05, 0x05, - 0xb3, 0x68, 0x3c, 0x41, 0x94, 0x48, 0xa2, 0xbd, 0x1d, 0xf7, 0x88, 0x02, 0x6d, 0x54, 0x7e, 0xe0, - 0xa6, 0x58, 0xc4, 0x11, 0x8b, 0xda, 0x84, 0x94, 0xdf, 0x09, 0x90, 0xd1, 0x22, 0x24, 0x9a, 0x6a, - 0xa9, 0x40, 0x65, 0x3c, 0x3b, 0xec, 0x00, 0x41, 0x6d, 0x42, 0x4a, 0x34, 0x55, 0x82, 0xca, 0x59, - 0xea, 0xf7, 0xbe, 0x51, 0x2a, 0x0e, 0x1c, 0xf1, 0xe1, 0x3c, 0xfa, 0xda, 0x84, 0x44, 0xb3, 0xcd, - 0xc9, 0xf7, 0x36, 0xa1, 0x68, 0xb1, 0x34, 0x24, 0x37, 0xb9, 0x50, 0x1c, 0xb8, 0x51, 0x18, 0x95, - 0x5f, 0x58, 0xa3, 0x51, 0x7f, 0x80, 0x8e, 0xbe, 0x02, 0x27, 0xc2, 0x12, 0xb9, 0xa5, 0xcd, 0xf4, - 0xb9, 0x91, 0x68, 0xb9, 0x61, 0x83, 0x43, 0x56, 0xdf, 0x4b, 0xf4, 0x0a, 0x4c, 0xb2, 0x5e, 0x43, - 0x54, 0x64, 0xd4, 0x0e, 0x78, 0x4f, 0x87, 0xb1, 0xf2, 0xc4, 0xf8, 0x1d, 0x9e, 0x7f, 0x23, 0xb7, - 0xcc, 0x66, 0x69, 0x76, 0xa0, 0xf1, 0xf7, 0xe7, 0x13, 0x11, 0xe3, 0x77, 0x7c, 0x2a, 0xe9, 0x77, - 0x8b, 0xbd, 0xe1, 0xe9, 0x1a, 0x27, 0x06, 0xf6, 0x7b, 0x44, 0x5a, 0x4e, 0x8d, 0xe6, 0x4e, 0xfb, - 0x64, 0x52, 0x35, 0x8b, 0x5d, 0xcf, 0x21, 0xd3, 0x31, 0x75, 0x72, 0x60, 0xd5, 0xfa, 0xaf, 0x1c, - 0xa9, 0xd1, 0x30, 0xc8, 0xa3, 0xa2, 0xfb, 0x20, 0xf2, 0x83, 0xf3, 0xfe, 0xb2, 0xf5, 0x29, 0x2a, - 0xef, 0xf9, 0x48, 0xd7, 0x15, 0x95, 0xdf, 0x50, 0x9b, 0x90, 0xa6, 0xd5, 0xf0, 0x1b, 0xf4, 0x00, - 0x66, 0xa8, 0x3c, 0x59, 0xf5, 0x6f, 0x3c, 0x28, 0x95, 0xfa, 0x12, 0x05, 0x07, 0x5f, 0x8e, 0xe0, - 0x4a, 0x16, 0xd5, 0x9e, 0x57, 0xc4, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x97, 0x9d, 0x1b, 0x68, 0xc6, - 0xe1, 0x5b, 0xcb, 0x88, 0x19, 0xeb, 0x8c, 0x42, 0xcc, 0xd8, 0xe1, 0xb9, 0x3c, 0xbc, 0x3b, 0x9e, - 0x1d, 0x68, 0xc6, 0x51, 0x49, 0x3f, 0xc4, 0x8c, 0x9d, 0x20, 0x9d, 0x98, 0x31, 0x73, 0x10, 0x3d, - 0x72, 0x9f, 0x1b, 0x68, 0xc6, 0x03, 0x0f, 0x92, 0x12, 0x33, 0x56, 0xfa, 0x5e, 0xa2, 0x15, 0x00, - 0x16, 0x9e, 0xe8, 0xc6, 0xb6, 0x59, 0x3a, 0x3b, 0x70, 0x32, 0xe8, 0xcd, 0xe6, 0x21, 0x93, 0x41, - 0xcb, 0xa5, 0x11, 0x47, 0x46, 0xb1, 0x91, 0x4c, 0x77, 0xc9, 0x4a, 0xf3, 0x03, 0x1d, 0x59, 0xdf, - 0x66, 0x19, 0x71, 0x64, 0x7b, 0x1e, 0x91, 0xcc, 0x2a, 0x6c, 0x8d, 0xb5, 0xb4, 0x30, 0xd8, 0x2d, - 0x07, 0x77, 0x17, 0xa8, 0x5b, 0xa6, 0x04, 0xb4, 0x04, 0x59, 0x32, 0x6d, 0x1f, 0x50, 0x37, 0x74, - 0x6e, 0x60, 0xc0, 0xd9, 0x73, 0x38, 0xa0, 0x36, 0x21, 0x65, 0x1e, 0x73, 0x12, 0xf9, 0x3c, 0x5b, - 0xb5, 0x2a, 0x95, 0x07, 0x7e, 0x3e, 0xb4, 0x52, 0x49, 0x3e, 0xcf, 0x38, 0x90, 0x0a, 0x27, 0x59, - 0x5f, 0xf1, 0x73, 0x9c, 0x16, 0x3f, 0x74, 0x58, 0x3a, 0x4f, 0x45, 0x0d, 0x5c, 0x03, 0x8a, 0x3c, - 0x5e, 0x5a, 0x9b, 0x90, 0x66, 0x95, 0xfe, 0xb7, 0x64, 0xc0, 0xf3, 0xa9, 0x87, 0xad, 0x1c, 0x95, - 0x2e, 0x0c, 0x1c, 0xf0, 0x11, 0x6b, 0x6d, 0x64, 0xc0, 0x2b, 0x01, 0x32, 0x9b, 0x80, 0x34, 0xd9, - 0xb6, 0xd9, 0x9e, 0xea, 0xc5, 0x21, 0x13, 0x50, 0x0f, 0xf6, 0x67, 0x13, 0x90, 0x56, 0x67, 0x9c, - 0x44, 0x90, 0xda, 0xc2, 0x8a, 0xc5, 0xdd, 0xec, 0xa5, 0x81, 0x82, 0xfa, 0x6e, 0x02, 0x23, 0x82, - 0x54, 0x8f, 0x48, 0x02, 0x1e, 0xcb, 0xbd, 0x71, 0x83, 0x87, 0x7e, 0x97, 0x07, 0x06, 0x3c, 0x91, - 0x17, 0x83, 0x90, 0x80, 0xc7, 0x0a, 0xbd, 0x40, 0x9f, 0x87, 0x29, 0x8e, 0xd0, 0x4a, 0x57, 0x86, - 0x04, 0xa4, 0x41, 0x68, 0x4d, 0xc6, 0x35, 0xe7, 0x61, 0x5e, 0x96, 0x21, 0x43, 0xd6, 0xbc, 0xe7, - 0x87, 0x78, 0xd9, 0x3e, 0x70, 0xca, 0xbc, 0xac, 0x4f, 0x26, 0x5e, 0x96, 0xd9, 0x29, 0x9f, 0xeb, - 0xae, 0x0e, 0xf4, 0xb2, 0xfd, 0x27, 0x13, 0x88, 0x97, 0x7d, 0xec, 0x53, 0x49, 0xcb, 0x6c, 0x86, - 0x8a, 0x4a, 0x9f, 0x19, 0xd8, 0xb2, 0x30, 0x48, 0x24, 0x2d, 0xe3, 0x3c, 0xa4, 0xdb, 0x58, 0x12, - 0x29, 0xd3, 0xf4, 0x0b, 0x83, 0x8f, 0x4c, 0xf7, 0x82, 0x88, 0x9a, 0x7b, 0x4b, 0x2c, 0xd3, 0xb0, - 0xe7, 0xa8, 0x2c, 0x7e, 0x40, 0x94, 0x6b, 0xea, 0xc5, 0xe1, 0x8e, 0x2a, 0xea, 0xec, 0xab, 0xe7, - 0xa8, 0x42, 0x2f, 0x69, 0x55, 0xd9, 0x39, 0x1f, 0x3a, 0xbe, 0x17, 0x87, 0x9c, 0xee, 0xee, 0x39, - 0x7a, 0x45, 0xab, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x65, 0xd7, 0x10, 0x94, 0xae, 0x0d, 0x1f, 0x42, - 0xe1, 0xeb, 0x10, 0xbc, 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x46, 0x18, 0x2f, 0x0d, 0x9f, 0x33, - 0x7b, 0x43, 0x0b, 0x36, 0x67, 0xf2, 0x98, 0xe2, 0xff, 0x08, 0xb0, 0xc0, 0xea, 0x46, 0xd7, 0xf1, - 0x0e, 0x64, 0x6f, 0x4d, 0x34, 0x90, 0x6a, 0x7e, 0x9d, 0x7e, 0xe0, 0x95, 0x41, 0xd5, 0x1d, 0xb1, - 0xc6, 0x5b, 0x9b, 0x90, 0x9e, 0x53, 0x86, 0x95, 0x5b, 0x9e, 0xe2, 0x3b, 0xa7, 0xde, 0x89, 0xbb, - 0x69, 0x51, 0xbc, 0x93, 0xca, 0x9c, 0x16, 0x4b, 0x77, 0x52, 0x99, 0x33, 0xe2, 0xdc, 0x9d, 0x54, - 0xe6, 0x19, 0xf1, 0xd9, 0xf2, 0x3f, 0x9d, 0x81, 0x82, 0x8b, 0xe1, 0x18, 0x22, 0xba, 0x11, 0x44, - 0x44, 0x67, 0x07, 0x21, 0x22, 0x8e, 0xfa, 0x38, 0x24, 0xba, 0x11, 0x84, 0x44, 0x67, 0x07, 0x41, - 0x22, 0x9f, 0x87, 0x60, 0xa2, 0xc6, 0x20, 0x4c, 0xf4, 0xfc, 0x18, 0x98, 0xc8, 0x13, 0xd5, 0x0b, - 0x8a, 0x56, 0xfa, 0x41, 0xd1, 0x85, 0xe1, 0xa0, 0xc8, 0x13, 0x15, 0x40, 0x45, 0xaf, 0xf7, 0xa0, - 0xa2, 0x73, 0x43, 0x50, 0x91, 0xc7, 0xef, 0xc2, 0xa2, 0xb5, 0x48, 0x58, 0x74, 0x69, 0x14, 0x2c, - 0xf2, 0xe4, 0x84, 0x70, 0xd1, 0xcb, 0x21, 0x5c, 0x34, 0x3f, 0x10, 0x17, 0x79, 0xdc, 0x0c, 0x18, - 0xbd, 0xd1, 0x0b, 0x8c, 0xce, 0x0d, 0x01, 0x46, 0x7e, 0x0b, 0x38, 0x32, 0xaa, 0x45, 0x21, 0xa3, - 0x8b, 0x23, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xe2, 0x08, 0x68, 0xd4, - 0x23, 0x89, 0x61, 0xa3, 0x8d, 0x68, 0x6c, 0x74, 0x79, 0x24, 0x36, 0xf2, 0xa4, 0x85, 0xc1, 0xd1, - 0xb5, 0x00, 0x38, 0x7a, 0x6e, 0x00, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0x42, 0x1f, 0x3a, 0x2a, - 0x0f, 0x43, 0x47, 0x1e, 0xaf, 0x07, 0x8f, 0xde, 0x1a, 0x00, 0x8f, 0xae, 0x8c, 0x86, 0x47, 0x9e, - 0xb0, 0x1e, 0x7c, 0xa4, 0x0c, 0xc5, 0x47, 0x2f, 0x8e, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, - 0xd5, 0x30, 0x40, 0x5a, 0x18, 0x0c, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0xad, 0x45, 0x22, 0xa4, 0x4b, - 0xa3, 0x10, 0x92, 0x3f, 0x0e, 0x82, 0x10, 0x69, 0x23, 0x1a, 0x22, 0x5d, 0x1e, 0x09, 0x91, 0xfc, - 0xee, 0x0f, 0x61, 0xa4, 0xb5, 0x48, 0x8c, 0x74, 0x69, 0x14, 0x46, 0xf2, 0x2b, 0x17, 0x04, 0x49, - 0x6f, 0x0f, 0x04, 0x49, 0x57, 0xc7, 0x01, 0x49, 0x9e, 0xd0, 0x3e, 0x94, 0xf4, 0xce, 0x60, 0x94, - 0xf4, 0x99, 0x23, 0x5c, 0x21, 0x17, 0x09, 0x93, 0xbe, 0xd0, 0x07, 0x93, 0xca, 0xc3, 0x60, 0x92, - 0x6f, 0xcf, 0x2e, 0x4e, 0x52, 0x86, 0xa2, 0x9a, 0x17, 0xc7, 0x44, 0x35, 0xbe, 0xf1, 0x45, 0xc0, - 0x9a, 0x6a, 0x04, 0xac, 0xb9, 0x30, 0x1c, 0xd6, 0xf8, 0xee, 0xdc, 0xc7, 0x35, 0xb5, 0x28, 0x5c, - 0x73, 0x71, 0x04, 0xae, 0xf1, 0xbd, 0x50, 0x00, 0xd8, 0xbc, 0xde, 0x03, 0x6c, 0xce, 0x8d, 0xcc, - 0xb2, 0x09, 0x20, 0x9b, 0xe5, 0x7e, 0x64, 0x73, 0x7e, 0x28, 0xb2, 0xf1, 0x24, 0xf8, 0xd0, 0xe6, - 0xf5, 0x1e, 0x68, 0x73, 0x6e, 0x08, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1c, 0xdb, 0x2c, - 0x8e, 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x23, 0x1a, 0xdc, 0x5c, 0x1e, 0x73, 0xf7, 0xbc, - 0x0f, 0xdd, 0xd4, 0xa2, 0xd0, 0xcd, 0xc5, 0x11, 0xe8, 0x26, 0x38, 0x87, 0x78, 0xf0, 0xa6, 0x16, - 0x05, 0x6f, 0x2e, 0x8e, 0x80, 0x37, 0xbe, 0xa4, 0x00, 0xbe, 0x69, 0x0c, 0xc2, 0x37, 0xcf, 0x8f, - 0x81, 0x6f, 0xfc, 0xe0, 0xa5, 0x07, 0xe0, 0xbc, 0xd9, 0x0b, 0x70, 0xca, 0xc3, 0x00, 0x8e, 0x3f, - 0x22, 0x5d, 0x84, 0xb3, 0x11, 0x8d, 0x70, 0x2e, 0x8f, 0x44, 0x38, 0x41, 0x27, 0x19, 0x80, 0x38, - 0x6b, 0x91, 0x10, 0xe7, 0xd2, 0x28, 0x88, 0xe3, 0x3b, 0xc9, 0x20, 0xc6, 0x79, 0xb3, 0x17, 0xe3, - 0x94, 0x87, 0x61, 0x1c, 0xbf, 0x71, 0x2e, 0xc8, 0xa9, 0x45, 0x81, 0x9c, 0x8b, 0x23, 0x40, 0x8e, - 0xdf, 0x79, 0x01, 0x94, 0xa3, 0x0c, 0x45, 0x39, 0x2f, 0x8e, 0x89, 0x72, 0x7a, 0x1c, 0x57, 0x18, - 0xe6, 0xd4, 0xa2, 0x60, 0xce, 0xc5, 0x11, 0x30, 0x27, 0x50, 0x59, 0x1f, 0xe7, 0x6c, 0x44, 0xe3, - 0x9c, 0xcb, 0x23, 0x71, 0x4e, 0xcf, 0x68, 0x72, 0x81, 0xce, 0x5a, 0x24, 0xd0, 0xb9, 0x34, 0x0a, - 0xe8, 0xf4, 0x4c, 0x7c, 0x3c, 0x38, 0xf8, 0xbf, 0xe3, 0x23, 0x9d, 0x57, 0x8f, 0x8e, 0x74, 0xbc, - 0x6f, 0xc6, 0x02, 0x75, 0xee, 0xa4, 0x32, 0xcf, 0x8a, 0xcf, 0x95, 0x7f, 0x30, 0x09, 0xe9, 0x9a, - 0x97, 0xe3, 0xe2, 0xd7, 0x52, 0x38, 0xce, 0xed, 0x35, 0x68, 0x85, 0x8c, 0x58, 0xea, 0xf7, 0x46, - 0x5f, 0x54, 0xd6, 0x7f, 0x89, 0x16, 0x67, 0x3d, 0xc6, 0x21, 0x50, 0xf4, 0x32, 0x14, 0xba, 0x36, - 0xb6, 0xe4, 0x8e, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x74, 0x7d, 0x61, 0x59, 0xfc, 0xe8, 0x70, 0x3e, - 0xbf, 0x65, 0x63, 0x6b, 0x93, 0xd3, 0xa5, 0x7c, 0x37, 0xf0, 0xe4, 0xfe, 0xa4, 0xcd, 0xe4, 0xf8, - 0x3f, 0x69, 0xf3, 0x16, 0x88, 0x16, 0x56, 0xb4, 0x50, 0x04, 0xc2, 0xae, 0x85, 0x89, 0xb6, 0x19, - 0x7a, 0x56, 0xc5, 0x2d, 0x49, 0xaf, 0x87, 0x99, 0xb6, 0xc2, 0x44, 0x74, 0x1d, 0x4e, 0xb6, 0x95, - 0x7d, 0x9a, 0x83, 0x28, 0xbb, 0x41, 0x1d, 0xcd, 0x2b, 0xcc, 0xd0, 0x7c, 0x59, 0xd4, 0x56, 0xf6, - 0xe9, 0xef, 0xe3, 0xb0, 0x57, 0xf4, 0x66, 0xfc, 0x8b, 0x50, 0xd4, 0x74, 0xdb, 0xd1, 0x0d, 0xd5, - 0xbd, 0xff, 0x93, 0x5d, 0xb4, 0x59, 0x70, 0xa9, 0xec, 0x1e, 0xce, 0xab, 0x30, 0xc3, 0xf3, 0xb1, - 0xfd, 0x5f, 0xcc, 0xa1, 0xf0, 0x25, 0x43, 0x6a, 0x41, 0x5e, 0xf8, 0x3f, 0x95, 0x54, 0x81, 0xe9, - 0xa6, 0xe2, 0xe0, 0x3d, 0xe5, 0x40, 0x76, 0x8f, 0xcb, 0xe4, 0xe8, 0xb5, 0x7a, 0xcf, 0x3c, 0x3d, - 0x9c, 0x2f, 0xdc, 0x66, 0xaf, 0xfa, 0x4e, 0xcd, 0x14, 0x9a, 0x81, 0x17, 0x1a, 0xba, 0x0c, 0xd3, - 0x8a, 0x7d, 0x60, 0xa8, 0x54, 0x3d, 0xd8, 0xb0, 0xbb, 0x36, 0x85, 0x14, 0x19, 0xa9, 0x48, 0xc9, - 0x15, 0x97, 0x8a, 0x5e, 0x85, 0x33, 0x1a, 0x26, 0x31, 0x12, 0x0b, 0x44, 0x1c, 0xd3, 0x94, 0xcd, - 0x96, 0x26, 0xd3, 0xeb, 0x1f, 0x28, 0x9c, 0xc8, 0x48, 0x27, 0x69, 0x01, 0x1a, 0x82, 0x34, 0x4c, - 0xf3, 0x5e, 0x4b, 0xab, 0x92, 0x97, 0xec, 0xaa, 0xcf, 0x3b, 0xa9, 0x4c, 0x5e, 0x2c, 0x94, 0x7f, - 0x4d, 0x80, 0x7c, 0xe8, 0x8c, 0xc2, 0xeb, 0x3d, 0x5b, 0xb4, 0x67, 0xa2, 0xe1, 0xd0, 0xa0, 0x1c, - 0xaf, 0x0c, 0x57, 0xbf, 0x9b, 0xe1, 0x36, 0x3f, 0x38, 0x9c, 0xa6, 0x8b, 0x03, 0x6e, 0x7a, 0x80, - 0xcb, 0xf6, 0x5a, 0xea, 0x37, 0xde, 0x9f, 0x9f, 0x28, 0xff, 0x22, 0x09, 0x85, 0xf0, 0x59, 0x84, - 0xd5, 0x9e, 0x7a, 0x45, 0xb9, 0xab, 0x10, 0xc7, 0xe2, 0x90, 0xfb, 0xcf, 0xb2, 0xfe, 0x7d, 0xdc, - 0xac, 0x9a, 0x0b, 0x43, 0x36, 0xa2, 0x83, 0xf5, 0xf4, 0x19, 0xe7, 0x7e, 0x98, 0xf0, 0x86, 0xfd, - 0x22, 0x4c, 0x32, 0xb5, 0x0b, 0x03, 0xcf, 0x86, 0x52, 0xcd, 0x4b, 0xac, 0x18, 0x71, 0x13, 0x8d, - 0x63, 0x5d, 0x72, 0xe5, 0x5f, 0x0f, 0x70, 0xf4, 0x5f, 0x92, 0xe2, 0x57, 0x9d, 0x4d, 0x1e, 0xed, - 0xaa, 0x33, 0xb6, 0xd1, 0xdc, 0x6a, 0x31, 0x17, 0xcc, 0x06, 0x4a, 0xba, 0xef, 0x00, 0x26, 0x15, - 0xc1, 0x7f, 0x68, 0x6c, 0x51, 0xe2, 0x3f, 0x34, 0x16, 0x48, 0x3a, 0x2c, 0x7a, 0x22, 0xe8, 0xa8, - 0x62, 0xa9, 0xa9, 0xbc, 0xab, 0xbf, 0x23, 0x80, 0x48, 0xc7, 0xd0, 0x2d, 0x8c, 0xb5, 0x58, 0xac, - 0xd0, 0xcd, 0x87, 0x4c, 0x8c, 0x9f, 0x26, 0x1e, 0xba, 0x1f, 0x3d, 0x19, 0xbe, 0x1f, 0xbd, 0xfc, - 0xbe, 0x00, 0x45, 0xaf, 0x86, 0xec, 0x67, 0x75, 0x86, 0xdc, 0x60, 0x76, 0xbc, 0x5f, 0x9a, 0x71, - 0x0f, 0x57, 0x8f, 0xf5, 0xfb, 0x3e, 0xc1, 0xc3, 0xd5, 0xec, 0x57, 0x51, 0xbe, 0x2d, 0xc0, 0xac, - 0x57, 0xc5, 0x8a, 0x7f, 0x70, 0xf6, 0x18, 0x19, 0xf3, 0x12, 0xfd, 0xc5, 0x31, 0x02, 0xda, 0xe9, - 0xa9, 0xf6, 0xb1, 0xcc, 0x13, 0xf1, 0x84, 0x0a, 0xe0, 0x8b, 0x01, 0x5a, 0xa3, 0x4e, 0x7f, 0x8b, - 0x8c, 0xfd, 0x6f, 0x97, 0x6f, 0x05, 0x14, 0x48, 0x47, 0x02, 0xd1, 0xd2, 0x58, 0x43, 0xc6, 0xd5, - 0x12, 0x2d, 0x5c, 0xfe, 0x71, 0xb0, 0x27, 0xaa, 0xbb, 0x24, 0x08, 0xbc, 0x09, 0xc9, 0x5d, 0xa5, - 0x35, 0x2c, 0x9f, 0x24, 0xd4, 0x73, 0x12, 0x29, 0x8d, 0x6e, 0x85, 0xce, 0x1b, 0x27, 0x06, 0x07, - 0x2c, 0xfd, 0x2a, 0x0d, 0x9d, 0x4b, 0x7e, 0xc5, 0x6d, 0x45, 0x72, 0xf4, 0xe7, 0x83, 0x1e, 0xe0, - 0xb5, 0xd4, 0x07, 0xef, 0xcf, 0x0b, 0x57, 0xeb, 0x30, 0x1b, 0x31, 0xbd, 0xa1, 0x22, 0x40, 0xe0, - 0xd6, 0x74, 0xfe, 0x83, 0x67, 0x4b, 0x2b, 0xf2, 0xd6, 0x46, 0xe5, 0xde, 0xfa, 0xfa, 0x6a, 0xa3, - 0x51, 0x5d, 0x11, 0x05, 0x24, 0x42, 0x3e, 0x74, 0xe7, 0x3a, 0xff, 0x9d, 0xb3, 0xab, 0xff, 0x0d, - 0xc0, 0xff, 0xdd, 0x05, 0x22, 0x6b, 0xad, 0xfa, 0x40, 0xbe, 0xbf, 0x74, 0x77, 0xab, 0x5a, 0x17, - 0x27, 0x10, 0x82, 0xe2, 0xf2, 0x52, 0xa3, 0x52, 0x93, 0xa5, 0x6a, 0x7d, 0xf3, 0xde, 0x46, 0xbd, - 0x2a, 0x0a, 0x9c, 0x6f, 0x05, 0xf2, 0xc1, 0x93, 0xd9, 0x68, 0x16, 0xa6, 0x2b, 0xb5, 0x6a, 0x65, - 0x4d, 0xbe, 0xbf, 0xba, 0x24, 0xbf, 0xb5, 0x55, 0xdd, 0xaa, 0x8a, 0x13, 0xb4, 0x6a, 0x94, 0x78, - 0x6b, 0xeb, 0xee, 0x5d, 0x51, 0x40, 0xd3, 0x90, 0x63, 0xcf, 0xf4, 0x7e, 0x76, 0x31, 0x71, 0x75, - 0x1d, 0x72, 0x81, 0x8b, 0xdc, 0xc8, 0xe7, 0x36, 0xb7, 0xea, 0x35, 0xb9, 0xb1, 0xba, 0x5e, 0xad, - 0x37, 0x96, 0xd6, 0x37, 0x99, 0x0c, 0x4a, 0x5b, 0x5a, 0xbe, 0x27, 0x35, 0x44, 0xc1, 0x7b, 0x6e, - 0xdc, 0xdb, 0xaa, 0xd4, 0xbc, 0x9f, 0x6b, 0x4b, 0x65, 0x92, 0x62, 0xf2, 0xea, 0x63, 0x38, 0x3d, - 0xe0, 0x78, 0x32, 0xca, 0xc1, 0xd4, 0x96, 0x41, 0xaf, 0xae, 0x12, 0x27, 0x50, 0x21, 0x70, 0x42, - 0x59, 0x14, 0x50, 0x86, 0x9d, 0x3d, 0x15, 0x13, 0x28, 0x0d, 0x89, 0xfa, 0x4d, 0x31, 0x49, 0x2a, - 0x1a, 0x38, 0xe0, 0x2b, 0xa6, 0x50, 0x96, 0x9f, 0x7e, 0x14, 0x27, 0x51, 0xde, 0x3f, 0x7e, 0x28, - 0xa6, 0xaf, 0x9e, 0x83, 0xc0, 0xd9, 0x2c, 0x04, 0x90, 0xbe, 0xab, 0x38, 0xd8, 0x76, 0xc4, 0x09, - 0x34, 0x05, 0xc9, 0xa5, 0x56, 0x4b, 0x14, 0x6e, 0xfc, 0x81, 0x00, 0x19, 0xf7, 0xaa, 0x71, 0x74, - 0x17, 0x26, 0x19, 0x92, 0x9f, 0x1f, 0x3c, 0x23, 0x51, 0xa7, 0x36, 0xb7, 0x30, 0x6a, 0xca, 0x2a, - 0x4f, 0xa0, 0xb7, 0xf9, 0x6f, 0x2f, 0x12, 0x8b, 0x41, 0xe7, 0x87, 0xd9, 0x93, 0x2b, 0x75, 0xb8, - 0xd1, 0x91, 0x31, 0x52, 0x9e, 0x78, 0x49, 0x58, 0x7e, 0xfe, 0x83, 0x9f, 0x9d, 0x9d, 0xf8, 0xe0, - 0xe9, 0x59, 0xe1, 0xc3, 0xa7, 0x67, 0x85, 0x9f, 0x3e, 0x3d, 0x2b, 0xfc, 0xfd, 0xd3, 0xb3, 0xc2, - 0xaf, 0xfc, 0xfc, 0xec, 0xc4, 0x87, 0x3f, 0x3f, 0x3b, 0xf1, 0xd3, 0x9f, 0x9f, 0x9d, 0x78, 0x67, - 0x8a, 0x73, 0x3f, 0x4c, 0xd3, 0x9f, 0x81, 0xbc, 0xf9, 0x6f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x95, - 0xb2, 0x7d, 0xda, 0x0b, 0x73, 0x00, 0x00, + 0x31, 0x97, 0xfa, 0x7f, 0xbf, 0x71, 0x76, 0xa2, 0x7c, 0x1f, 0xe0, 0x36, 0x76, 0xb8, 0x59, 0xa1, + 0x65, 0x48, 0xef, 0xd0, 0xda, 0x70, 0xc3, 0x5e, 0x88, 0xac, 0x76, 0xc0, 0x04, 0x97, 0x33, 0x44, + 0x03, 0x1f, 0x1e, 0xce, 0x0b, 0x12, 0xe7, 0x64, 0x5d, 0x5e, 0xfe, 0x81, 0x00, 0x39, 0x2a, 0x98, + 0xb5, 0x11, 0x55, 0x7a, 0x24, 0x9f, 0x1b, 0xa9, 0x90, 0x7e, 0xd1, 0x68, 0x11, 0x26, 0x77, 0x95, + 0x56, 0x77, 0xd8, 0xb8, 0xb9, 0x4f, 0xde, 0x4b, 0xac, 0x18, 0x7a, 0x1d, 0xf2, 0xba, 0xe1, 0x60, + 0xc3, 0x91, 0x19, 0x5b, 0x72, 0x04, 0x5b, 0x8e, 0x95, 0xa6, 0x0f, 0xe5, 0xef, 0x0b, 0x00, 0x9b, + 0xdd, 0x38, 0x55, 0x43, 0xc6, 0xfd, 0x58, 0xf5, 0x77, 0xc7, 0x3d, 0x6b, 0xc5, 0x29, 0x48, 0xeb, + 0x46, 0x4b, 0x37, 0x58, 0xfd, 0x33, 0x12, 0x7f, 0x42, 0x27, 0x60, 0xf2, 0x61, 0x4b, 0x37, 0x34, + 0x6a, 0xfe, 0x19, 0x89, 0x3d, 0x70, 0xf5, 0x4b, 0x90, 0xa3, 0x75, 0x8f, 0x51, 0xfb, 0xe5, 0x6f, + 0x26, 0xe0, 0x64, 0xc5, 0x34, 0x34, 0x9d, 0x8c, 0x43, 0xa5, 0xf5, 0xa9, 0xd0, 0xcd, 0xcb, 0x90, + 0xc5, 0xfb, 0x9d, 0x31, 0xbb, 0x37, 0x83, 0xf7, 0x3b, 0xf4, 0xbf, 0x68, 0xd5, 0xa1, 0xcf, 0xc2, + 0x69, 0xa5, 0xd5, 0x32, 0xf7, 0x64, 0x7d, 0x5b, 0xd6, 0x4c, 0x6c, 0xcb, 0x86, 0xe9, 0xc8, 0x78, + 0x5f, 0xb7, 0x1d, 0xea, 0x2a, 0x32, 0xd2, 0x2c, 0x7d, 0xbd, 0xba, 0xbd, 0x62, 0x62, 0x7b, 0xc3, + 0x74, 0xaa, 0xe4, 0x15, 0x57, 0xf8, 0xbb, 0x70, 0xaa, 0x57, 0x37, 0x71, 0xea, 0xfe, 0xaf, 0x04, + 0x28, 0xae, 0x1a, 0xba, 0xf3, 0xa9, 0x50, 0xba, 0xa7, 0xbd, 0x64, 0x50, 0x7b, 0x57, 0x41, 0xdc, + 0x56, 0xf4, 0xd6, 0x3d, 0xa3, 0x61, 0xb6, 0x1f, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0xab, 0xb7, 0x8f, + 0xce, 0x75, 0x76, 0x1f, 0xa6, 0xbd, 0x36, 0xc5, 0xa9, 0xac, 0x27, 0x20, 0xae, 0x1a, 0xaa, 0x85, + 0xdb, 0xd8, 0x88, 0x55, 0x5b, 0xcf, 0x42, 0x56, 0x77, 0xe5, 0x52, 0x8d, 0x25, 0x25, 0x9f, 0xc0, + 0xdb, 0xd4, 0x85, 0x99, 0xc0, 0xb7, 0xe3, 0x74, 0x7e, 0xcf, 0x40, 0xd6, 0xc0, 0x7b, 0xb2, 0xdf, + 0x5f, 0x49, 0x29, 0x63, 0xe0, 0x3d, 0xe6, 0xac, 0x1e, 0x40, 0x61, 0x05, 0xb7, 0xb0, 0x83, 0xe3, + 0xf7, 0xe4, 0x5b, 0x50, 0x74, 0x45, 0xc7, 0xd9, 0x49, 0xbf, 0x2e, 0x00, 0xe2, 0x72, 0xc9, 0xec, + 0x19, 0x67, 0x3f, 0xcd, 0x93, 0xe8, 0xc0, 0xe9, 0x5a, 0x06, 0x9b, 0xe6, 0x99, 0x95, 0x02, 0x23, + 0xd1, 0x99, 0xde, 0xf7, 0xa8, 0xa9, 0xa0, 0x47, 0xf5, 0xa2, 0x15, 0x12, 0xa7, 0xec, 0xc1, 0x6c, + 0xa8, 0x7a, 0xf1, 0x76, 0x65, 0x8a, 0xd6, 0x2c, 0xb1, 0x90, 0x0c, 0x46, 0x66, 0x94, 0x58, 0x7e, + 0x17, 0x66, 0x2a, 0x2d, 0xac, 0x58, 0x71, 0xab, 0x85, 0x77, 0xe7, 0x03, 0x40, 0x41, 0xf1, 0x71, + 0x76, 0xe9, 0x6f, 0x0a, 0x80, 0x24, 0xbc, 0x8b, 0x2d, 0x27, 0xf6, 0x2e, 0x5d, 0x81, 0x9c, 0xa3, + 0x58, 0x4d, 0xec, 0xc8, 0x24, 0x9c, 0xe7, 0xee, 0xea, 0xb9, 0x80, 0x20, 0x12, 0xd4, 0x2f, 0xee, + 0xb4, 0xd4, 0xc5, 0x86, 0x1b, 0xee, 0xbb, 0xc1, 0x19, 0xe3, 0x23, 0x64, 0xae, 0x81, 0x77, 0x60, + 0x36, 0x54, 0xcb, 0x38, 0x55, 0xf0, 0x97, 0x02, 0xe4, 0xea, 0xaa, 0x62, 0xc4, 0xd9, 0xf6, 0x37, + 0x21, 0x67, 0xab, 0x8a, 0x21, 0x6f, 0x9b, 0x56, 0x5b, 0x71, 0xa8, 0xc9, 0x16, 0x43, 0x6d, 0xf7, + 0x82, 0x5d, 0x55, 0x31, 0x6e, 0xd1, 0x42, 0x12, 0xd8, 0xde, 0xff, 0xe8, 0x2a, 0xcc, 0xd8, 0xb8, + 0x85, 0x55, 0x87, 0x48, 0x90, 0xbb, 0x1d, 0x4d, 0x71, 0x30, 0x9f, 0xd0, 0xa6, 0xd9, 0x8b, 0x5b, + 0xa6, 0xb5, 0x45, 0xc9, 0xc1, 0x11, 0x70, 0x27, 0x95, 0x49, 0x8a, 0xa9, 0xf2, 0x3f, 0x0b, 0x90, + 0x67, 0x2d, 0x8a, 0x73, 0x04, 0xbc, 0x0c, 0x29, 0xcb, 0xdc, 0x63, 0x23, 0x20, 0x77, 0xe3, 0x99, + 0x08, 0x11, 0x6b, 0xf8, 0x20, 0x38, 0xf5, 0xd0, 0xe2, 0x68, 0x19, 0x78, 0x88, 0x26, 0x53, 0xee, + 0xe4, 0xb8, 0xdc, 0xc0, 0xb8, 0x24, 0x22, 0xe3, 0x32, 0x4c, 0x3f, 0x54, 0x1c, 0x75, 0x47, 0xb6, + 0x78, 0x25, 0xc9, 0x34, 0x95, 0xbc, 0x92, 0x97, 0x8a, 0x94, 0xec, 0x56, 0xdd, 0x2e, 0xff, 0xae, + 0x6b, 0xce, 0x36, 0xfe, 0x94, 0x75, 0x69, 0x44, 0x37, 0xfd, 0x8b, 0xc0, 0xad, 0xda, 0xad, 0xec, + 0x7f, 0xb5, 0xde, 0xfa, 0x4e, 0x02, 0x4e, 0x57, 0x76, 0xb0, 0xfa, 0xa8, 0x62, 0x1a, 0xb6, 0x6e, + 0x3b, 0xd8, 0x50, 0x0f, 0xe2, 0xec, 0xb2, 0x67, 0x20, 0xbb, 0xa7, 0x3b, 0x3b, 0xb2, 0xa6, 0x6f, + 0x6f, 0x53, 0xff, 0x93, 0x91, 0x32, 0x84, 0xb0, 0xa2, 0x6f, 0x6f, 0xa3, 0x9b, 0x90, 0x6a, 0x9b, + 0x1a, 0x8b, 0x40, 0x8b, 0x37, 0xe6, 0x23, 0xc4, 0xd3, 0xaa, 0xd9, 0xdd, 0xf6, 0xba, 0xa9, 0x61, + 0x89, 0x16, 0x46, 0x67, 0x01, 0x54, 0x42, 0xed, 0x98, 0xba, 0xe1, 0xf0, 0x99, 0x28, 0x40, 0x41, + 0x35, 0xc8, 0x3a, 0xd8, 0x6a, 0xeb, 0x06, 0x1b, 0xaf, 0x44, 0x79, 0x17, 0x22, 0x2b, 0xde, 0x69, + 0xe9, 0xaa, 0xd2, 0xb7, 0xd4, 0xe0, 0x33, 0x73, 0xbf, 0xf7, 0x5e, 0x0a, 0x4a, 0xfd, 0x1a, 0x8a, + 0xd3, 0x4e, 0x36, 0x21, 0x4d, 0x70, 0x6b, 0xcb, 0xe1, 0x96, 0x72, 0x63, 0x90, 0x22, 0x22, 0x6a, + 0x40, 0xf1, 0x6f, 0xcb, 0xe1, 0x95, 0xe7, 0x72, 0xe6, 0xfe, 0x44, 0x80, 0x34, 0x7b, 0x81, 0xae, + 0x43, 0x86, 0xc3, 0x73, 0x8d, 0xd6, 0x31, 0xb9, 0x7c, 0xea, 0xe9, 0xe1, 0xfc, 0x14, 0x03, 0xe3, + 0x2b, 0x1f, 0xf9, 0xff, 0x4a, 0x53, 0x0c, 0x8f, 0x6b, 0xa4, 0xcf, 0x6c, 0x47, 0xb1, 0x1c, 0xba, + 0x0c, 0x42, 0xfa, 0x2c, 0x2f, 0x65, 0x28, 0x61, 0x0d, 0x1f, 0xa0, 0x3b, 0x90, 0xb6, 0x1d, 0xc5, + 0xe9, 0xda, 0xbc, 0xd7, 0x8e, 0x54, 0xd9, 0x3a, 0xe5, 0x94, 0xb8, 0x04, 0x12, 0x50, 0x68, 0xd8, + 0x51, 0xf4, 0x16, 0xed, 0xc6, 0xac, 0xc4, 0x9f, 0xca, 0xdf, 0x12, 0x20, 0xcd, 0x8a, 0xa2, 0xd3, + 0x30, 0xcb, 0x20, 0xf8, 0xea, 0xc6, 0x4a, 0xb5, 0x51, 0x95, 0xd6, 0x57, 0x37, 0x96, 0x1a, 0x55, + 0x71, 0x02, 0x9d, 0x02, 0xe4, 0xbe, 0xa8, 0xdc, 0xdb, 0xa8, 0xaf, 0xd6, 0x1b, 0xd5, 0x0d, 0x02, + 0xdd, 0x09, 0xa0, 0xa7, 0xf4, 0x00, 0x35, 0x81, 0x2e, 0xc0, 0x42, 0x2f, 0x55, 0xae, 0x37, 0x96, + 0x1a, 0x75, 0xb9, 0x5a, 0x6f, 0xac, 0xae, 0x2f, 0x35, 0xaa, 0x2b, 0x62, 0x72, 0x48, 0x29, 0xf2, + 0x11, 0x49, 0xaa, 0x56, 0x1a, 0x62, 0xaa, 0xfc, 0x04, 0x4e, 0x4a, 0x58, 0x35, 0xdb, 0x9d, 0xae, + 0x83, 0x49, 0x2d, 0xed, 0x38, 0xc7, 0xcb, 0x69, 0x98, 0xd2, 0xac, 0x03, 0xd9, 0xea, 0x1a, 0x7c, + 0xb4, 0xa4, 0x35, 0xeb, 0x40, 0xea, 0x1a, 0xdc, 0x18, 0xff, 0x48, 0x80, 0x53, 0xbd, 0x1f, 0x8f, + 0xd3, 0x14, 0xbf, 0x04, 0x39, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xe5, 0x28, 0x3c, 0x60, 0xb8, + 0x1e, 0x90, 0xc4, 0x97, 0xb0, 0x16, 0xd9, 0xda, 0xd5, 0xa2, 0xb7, 0x84, 0xb5, 0x7e, 0xbf, 0x52, + 0xa1, 0xf5, 0x59, 0x21, 0x8c, 0xae, 0x47, 0xa2, 0xb2, 0x28, 0xa5, 0xfc, 0xed, 0x14, 0x14, 0xaa, + 0x86, 0xd6, 0xd8, 0x8f, 0x75, 0x46, 0x38, 0x05, 0x69, 0xd5, 0x6c, 0xb7, 0x75, 0xc7, 0xd5, 0x16, + 0x7b, 0x42, 0x9f, 0x83, 0x8c, 0x86, 0x15, 0xcd, 0x83, 0xff, 0xa3, 0xa2, 0x1e, 0xc9, 0x2b, 0x8e, + 0xbe, 0x02, 0xa7, 0x89, 0x23, 0xb5, 0x0c, 0xa5, 0x25, 0x33, 0x69, 0xb2, 0x63, 0xe9, 0xcd, 0x26, + 0xb6, 0xf8, 0x82, 0xd9, 0x95, 0x88, 0x7a, 0xae, 0x72, 0x8e, 0x0a, 0x65, 0x68, 0xb0, 0xf2, 0xd2, + 0x49, 0x3d, 0x8a, 0x8c, 0xbe, 0xe8, 0xad, 0xaf, 0xd8, 0x1d, 0xc5, 0xb0, 0xb9, 0x93, 0x1a, 0xb4, + 0x0e, 0xc7, 0x75, 0xc9, 0xe7, 0x04, 0x42, 0xb1, 0xd1, 0x35, 0x12, 0xaa, 0x3f, 0xee, 0xea, 0x16, + 0x96, 0xaf, 0x77, 0xd4, 0x52, 0x9a, 0xb4, 0x7d, 0xb9, 0xf8, 0xf4, 0x70, 0x1e, 0x24, 0x46, 0xbe, + 0xbe, 0x59, 0x21, 0xa1, 0x3b, 0xfb, 0xbf, 0xa3, 0xa2, 0x2b, 0x20, 0x1a, 0xa6, 0x6c, 0xe1, 0x6d, + 0x0b, 0xdb, 0x3b, 0xfc, 0xb3, 0x19, 0xaa, 0xb1, 0xa2, 0x61, 0x4a, 0x8c, 0xcc, 0x44, 0x9f, 0x82, + 0x74, 0xc7, 0xd4, 0x6d, 0xd3, 0x28, 0x65, 0x99, 0x46, 0xd9, 0x13, 0x7a, 0x0b, 0x44, 0xdd, 0x90, + 0xb7, 0x5b, 0x7a, 0x73, 0xc7, 0x91, 0xf7, 0x2c, 0xdd, 0xc1, 0x76, 0x69, 0x86, 0x56, 0x3c, 0xca, + 0xd0, 0xea, 0x7c, 0x25, 0x53, 0x7b, 0x9b, 0x94, 0xe4, 0x4d, 0x28, 0xea, 0xc6, 0x2d, 0xca, 0x4f, + 0x89, 0xb6, 0x37, 0x1d, 0x4f, 0x89, 0x99, 0xf2, 0xdf, 0x09, 0x50, 0x74, 0xcd, 0x23, 0x4e, 0x83, + 0xbe, 0x02, 0xa2, 0x69, 0x60, 0xb9, 0xb3, 0xa3, 0xd8, 0x98, 0x77, 0x27, 0x9f, 0x33, 0x8a, 0xa6, + 0x81, 0x37, 0x09, 0x99, 0x75, 0x0e, 0xda, 0x84, 0x19, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x94, 0xbd, + 0xb5, 0x6f, 0x1a, 0xef, 0x8d, 0x19, 0x31, 0x8b, 0x9c, 0xdb, 0xa3, 0x87, 0x02, 0x8d, 0x1f, 0x0b, + 0x30, 0xb3, 0xa4, 0xb5, 0x75, 0xa3, 0xde, 0x69, 0xe9, 0xb1, 0xc2, 0xeb, 0x0b, 0x90, 0xb5, 0x89, + 0x4c, 0xdf, 0x5b, 0xfb, 0xd0, 0x28, 0x43, 0xdf, 0x10, 0xb7, 0x7d, 0x17, 0xa6, 0xf1, 0x7e, 0x47, + 0xb7, 0x14, 0x47, 0x37, 0x0d, 0x86, 0x06, 0x52, 0xe3, 0xb7, 0xad, 0xe8, 0xf3, 0xfa, 0x88, 0x80, + 0xb7, 0xec, 0x01, 0xa0, 0x60, 0xc3, 0xe2, 0x84, 0x05, 0x32, 0xcc, 0x52, 0xd1, 0x5b, 0x86, 0x1d, + 0xb3, 0xd6, 0xb8, 0x3b, 0xfd, 0x32, 0x9c, 0x08, 0x7f, 0x20, 0xce, 0xda, 0xbf, 0xcb, 0x7b, 0x7c, + 0x1d, 0x5b, 0x9f, 0x10, 0x22, 0x0d, 0x8a, 0x8f, 0xb3, 0xe6, 0x5f, 0x17, 0xe0, 0x0c, 0x95, 0x4d, + 0x77, 0x10, 0xb6, 0xb1, 0x45, 0xb7, 0x55, 0xe2, 0x34, 0xda, 0xf3, 0x90, 0x66, 0x00, 0x93, 0x5a, + 0xec, 0xe4, 0x72, 0x8e, 0x04, 0x22, 0x75, 0xc7, 0xb4, 0x48, 0x20, 0xc2, 0x5f, 0xf1, 0x76, 0x2a, + 0x30, 0x17, 0x55, 0x97, 0x98, 0x11, 0xf8, 0x0c, 0x8f, 0x07, 0x89, 0x89, 0x57, 0x76, 0x48, 0x20, + 0x84, 0xaa, 0x90, 0x53, 0xe9, 0x7f, 0xb2, 0x73, 0xd0, 0xc1, 0x54, 0x7e, 0x71, 0x58, 0x28, 0xc9, + 0xd8, 0x1a, 0x07, 0x1d, 0x4c, 0xe2, 0x51, 0xf7, 0x7f, 0xa2, 0xae, 0x40, 0x53, 0x87, 0x06, 0xa3, + 0x74, 0x7c, 0xd1, 0xb2, 0x6e, 0x3c, 0x17, 0xd2, 0xc4, 0x1f, 0x27, 0xb9, 0x2a, 0xd8, 0x97, 0x38, + 0x53, 0xac, 0xe1, 0xc7, 0x3b, 0x70, 0x4a, 0xc3, 0x1d, 0x0b, 0xab, 0x8a, 0x83, 0x35, 0x39, 0xd8, + 0xfc, 0xc4, 0x11, 0x9a, 0x7f, 0xc2, 0x97, 0xe1, 0x53, 0xd1, 0x03, 0x40, 0x01, 0xd9, 0xac, 0x65, + 0x2e, 0xbc, 0x39, 0x8a, 0x52, 0x66, 0x7c, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0x64, 0xf0, 0x7e, 0x47, + 0xa6, 0x7b, 0x8b, 0xa9, 0x23, 0xee, 0x2d, 0x4e, 0xe1, 0xfd, 0x0e, 0x21, 0xa2, 0x2d, 0x32, 0xc3, + 0xb9, 0x13, 0x3f, 0xad, 0xb6, 0x3d, 0x1a, 0x3f, 0xf8, 0xf6, 0xc2, 0xc5, 0x4d, 0x7b, 0x73, 0x3e, + 0x13, 0xc1, 0xfb, 0xee, 0x7d, 0x01, 0x9e, 0x89, 0xec, 0xbb, 0x38, 0x27, 0x3b, 0x77, 0x7b, 0x35, + 0x71, 0x9c, 0xed, 0xd5, 0xf2, 0xef, 0xb9, 0xa3, 0x5e, 0xc2, 0x2d, 0x93, 0xa8, 0xf7, 0x13, 0x58, + 0x8e, 0x9a, 0x72, 0xbb, 0x3d, 0x71, 0xe4, 0x6e, 0x77, 0x59, 0x7b, 0xdc, 0x42, 0x4f, 0x65, 0xe3, + 0x74, 0x0b, 0xbf, 0x2a, 0xc0, 0x6c, 0x0d, 0x2b, 0x96, 0xf3, 0x10, 0x2b, 0x4e, 0xcc, 0x81, 0xeb, + 0xcb, 0x90, 0x34, 0xcc, 0xbd, 0xa3, 0xac, 0xc8, 0x91, 0xf2, 0xfe, 0xb4, 0x15, 0xae, 0x57, 0x9c, + 0xad, 0xfe, 0xf3, 0x04, 0x64, 0x6f, 0x57, 0xe2, 0x6c, 0xeb, 0x1b, 0x7c, 0xdd, 0x96, 0x0d, 0xf5, + 0x28, 0xb3, 0xf4, 0xbe, 0xb7, 0x78, 0xbb, 0xb2, 0x86, 0x0f, 0x5c, 0xb3, 0x24, 0x5c, 0x68, 0x09, + 0xb2, 0xce, 0x0e, 0x89, 0x4f, 0xcd, 0x96, 0x76, 0x94, 0x98, 0xc5, 0xe7, 0x9a, 0x7b, 0x04, 0x93, + 0x54, 0xae, 0xbb, 0xf3, 0x2f, 0x44, 0xec, 0xfc, 0x93, 0xcf, 0x78, 0x61, 0x5f, 0xe2, 0x28, 0x9f, + 0x71, 0x09, 0xac, 0x73, 0xbc, 0xd8, 0x68, 0x52, 0x4c, 0x97, 0xdf, 0x02, 0x20, 0x4d, 0x8b, 0xb3, + 0x7b, 0x7e, 0x39, 0x09, 0xc5, 0xcd, 0xae, 0xbd, 0x13, 0xb3, 0x3d, 0x56, 0x00, 0x3a, 0x5d, 0x7b, + 0x07, 0x5b, 0xb2, 0xb3, 0x6f, 0xf0, 0xf6, 0x8f, 0xc8, 0x29, 0x70, 0x15, 0xc0, 0xf8, 0x1a, 0xfb, + 0x06, 0xba, 0xc7, 0x85, 0x60, 0xd9, 0x4f, 0x4c, 0xb8, 0x3a, 0x06, 0x78, 0x6c, 0xec, 0x1b, 0xeb, + 0xd8, 0x43, 0x8d, 0x4c, 0x20, 0x26, 0x02, 0xdf, 0x80, 0x29, 0xf2, 0x20, 0x3b, 0xe6, 0x51, 0x7a, + 0x3e, 0x4d, 0x78, 0x1a, 0x26, 0x7a, 0x1d, 0xb2, 0x8c, 0x9b, 0xcc, 0x5f, 0x69, 0x3a, 0x7f, 0x45, + 0x35, 0x89, 0x6b, 0x93, 0xce, 0x5c, 0x19, 0xca, 0x4a, 0x66, 0xab, 0x13, 0x30, 0xb9, 0x6d, 0x5a, + 0x2a, 0xa6, 0x99, 0x08, 0x19, 0x89, 0x3d, 0x04, 0x3b, 0xf7, 0x4e, 0x2a, 0x93, 0x11, 0xb3, 0x77, + 0x52, 0x99, 0xac, 0x08, 0xe5, 0x6f, 0x09, 0x30, 0xed, 0xf5, 0x4a, 0x9c, 0x2e, 0xbd, 0x12, 0x52, + 0xe9, 0xd1, 0xfb, 0x85, 0xa8, 0xb1, 0xfc, 0xd7, 0x34, 0xbe, 0x51, 0xcd, 0x5d, 0xda, 0x4d, 0x71, + 0x9a, 0xcd, 0x32, 0xcb, 0x41, 0x49, 0x1c, 0xb3, 0xab, 0x69, 0x56, 0xca, 0x75, 0x38, 0xa1, 0xb7, + 0x89, 0xcf, 0xd7, 0x9d, 0xd6, 0x01, 0xc7, 0x68, 0x0e, 0x76, 0xb7, 0x49, 0x67, 0xfd, 0x77, 0x15, + 0xf7, 0x15, 0x77, 0x83, 0x6c, 0xe3, 0xc4, 0x6f, 0x56, 0x9c, 0x7a, 0x5f, 0x85, 0x82, 0xc5, 0x44, + 0x93, 0x58, 0xe5, 0x88, 0xaa, 0xcf, 0x7b, 0xac, 0x44, 0xfb, 0xdf, 0x4b, 0xc0, 0xf4, 0x5b, 0x5d, + 0x6c, 0x1d, 0x7c, 0x0a, 0x75, 0x7f, 0x09, 0xa6, 0xf7, 0x14, 0x3d, 0xb4, 0xc5, 0xc1, 0xd4, 0x5e, + 0x20, 0x64, 0x6f, 0x83, 0x03, 0x61, 0x40, 0x8f, 0x0c, 0x73, 0xcf, 0x90, 0x09, 0x99, 0x42, 0xe4, + 0x7d, 0x83, 0x2f, 0x29, 0x2f, 0xbf, 0xf2, 0xb7, 0x87, 0xf3, 0x37, 0xc7, 0xca, 0x7f, 0xa2, 0x29, + 0x64, 0xdd, 0xae, 0xae, 0x2d, 0x6e, 0x6d, 0xad, 0xae, 0x48, 0x22, 0x15, 0xf9, 0x36, 0x93, 0xd8, + 0xd8, 0x37, 0xdc, 0xa9, 0xfd, 0x23, 0x01, 0x44, 0x5f, 0x61, 0x71, 0xf6, 0x6a, 0x15, 0x72, 0x8f, + 0xbb, 0xd8, 0xd2, 0x8f, 0xd1, 0xa7, 0xc0, 0x19, 0x89, 0x5b, 0x7a, 0x07, 0xf2, 0x21, 0x3d, 0x24, + 0x3f, 0x9e, 0x1e, 0x72, 0x7b, 0xbe, 0x0a, 0xca, 0x3f, 0x12, 0x00, 0xd1, 0xc6, 0xaf, 0xb2, 0xd5, + 0xfc, 0x4f, 0x99, 0xc1, 0x5c, 0x01, 0x91, 0x66, 0x24, 0xca, 0xfa, 0xb6, 0xdc, 0xd6, 0x6d, 0x5b, + 0x37, 0x9a, 0xdc, 0x62, 0x8a, 0x94, 0xbe, 0xba, 0xbd, 0xce, 0xa8, 0xbc, 0x2f, 0xff, 0x27, 0xcc, + 0x86, 0x5a, 0x13, 0x67, 0x6f, 0x9e, 0x83, 0xfc, 0xb6, 0xd9, 0x35, 0x34, 0x99, 0x2d, 0x8d, 0xf1, + 0x25, 0xc0, 0x1c, 0xa5, 0xb1, 0xef, 0x95, 0xff, 0x35, 0x01, 0x27, 0x24, 0x6c, 0x9b, 0xad, 0x5d, + 0x1c, 0xbf, 0x3e, 0xef, 0x01, 0xdf, 0x72, 0x91, 0x3f, 0x8e, 0x5a, 0xb3, 0x4c, 0x06, 0x9b, 0xee, + 0xc2, 0x6b, 0xeb, 0x17, 0x86, 0x5b, 0x66, 0xff, 0x6a, 0x3a, 0x5f, 0xb9, 0x4b, 0x85, 0x56, 0xee, + 0x76, 0x61, 0x5a, 0x6f, 0x1a, 0x26, 0x71, 0x64, 0x36, 0x7e, 0x6c, 0x74, 0xdb, 0x2e, 0xac, 0x79, + 0x79, 0x8c, 0xba, 0xae, 0x32, 0xce, 0x3a, 0x7e, 0xbc, 0xd1, 0x6d, 0xd3, 0x18, 0x7b, 0xf9, 0x14, + 0xa9, 0xf6, 0xd3, 0xc3, 0xf9, 0x62, 0xe8, 0x9d, 0x2d, 0x15, 0x75, 0xef, 0x99, 0x7c, 0x84, 0x1b, + 0xc0, 0x7f, 0x87, 0x93, 0x3d, 0x1d, 0x10, 0x67, 0x34, 0xf4, 0x17, 0x49, 0x38, 0x13, 0x16, 0x1f, + 0x37, 0x66, 0xf9, 0x4f, 0xd2, 0xc9, 0x35, 0x28, 0xb4, 0x75, 0xe3, 0x78, 0x2b, 0x97, 0xf9, 0xb6, + 0x6e, 0x78, 0xb4, 0x28, 0x73, 0x49, 0xff, 0xfb, 0x99, 0x8b, 0x02, 0x73, 0x51, 0xfd, 0x19, 0xa7, + 0xcd, 0xbc, 0x27, 0x40, 0x3e, 0xee, 0x35, 0xb9, 0xe3, 0xa5, 0x84, 0xf1, 0x36, 0x37, 0xa0, 0xf0, + 0x09, 0x2c, 0xe2, 0xfd, 0xb6, 0x00, 0xa8, 0x61, 0x75, 0x0d, 0x02, 0x8e, 0xef, 0x9a, 0xcd, 0x38, + 0x1b, 0x7b, 0x02, 0x26, 0x75, 0x43, 0xc3, 0xfb, 0xb4, 0xb1, 0x29, 0x89, 0x3d, 0x84, 0x76, 0x1a, + 0x93, 0x63, 0xed, 0x34, 0xfa, 0x99, 0x25, 0xa1, 0x8a, 0xc6, 0xa9, 0x85, 0xef, 0x25, 0x60, 0x96, + 0x37, 0x27, 0xf6, 0x45, 0xcc, 0x63, 0xe5, 0xa3, 0xa3, 0xcf, 0x03, 0x74, 0x2c, 0xbc, 0x2b, 0x33, + 0xd6, 0xe4, 0x58, 0xac, 0x59, 0xc2, 0x41, 0x09, 0xe8, 0x4b, 0x30, 0x4d, 0x06, 0x7a, 0xc7, 0x32, + 0x3b, 0xa6, 0x4d, 0xe2, 0x18, 0x7b, 0x3c, 0x68, 0x34, 0xf3, 0xf4, 0x70, 0xbe, 0xb0, 0xae, 0x1b, + 0x9b, 0x9c, 0xb1, 0x51, 0x97, 0x88, 0xc7, 0xf0, 0x1e, 0xdd, 0x01, 0xf8, 0x37, 0x02, 0x9c, 0xf8, + 0xc4, 0x96, 0x7d, 0xff, 0x23, 0x34, 0xe6, 0xcd, 0x43, 0x22, 0x7d, 0x5c, 0x35, 0xb6, 0xcd, 0xf8, + 0x17, 0xe3, 0xdf, 0x13, 0x60, 0x26, 0x20, 0x3e, 0xce, 0x28, 0xe7, 0x78, 0xa7, 0x1e, 0xbe, 0x4c, + 0xe2, 0x9e, 0xa0, 0xd9, 0xc7, 0x39, 0xa8, 0xfe, 0x34, 0x01, 0xa7, 0x2a, 0x6c, 0x0f, 0xda, 0x4d, + 0xd0, 0x88, 0xd3, 0x4a, 0x4a, 0x30, 0xb5, 0x8b, 0x2d, 0x5b, 0x37, 0xd9, 0x7c, 0x5b, 0x90, 0xdc, + 0x47, 0x34, 0x07, 0x19, 0xdb, 0x50, 0x3a, 0xf6, 0x8e, 0xe9, 0xee, 0xe2, 0x79, 0xcf, 0x5e, 0x32, + 0xc9, 0xe4, 0xf1, 0x93, 0x49, 0xd2, 0xc3, 0x93, 0x49, 0xa6, 0x3e, 0x76, 0x32, 0x09, 0xdf, 0x32, + 0xfb, 0xa1, 0x00, 0xa7, 0xfb, 0xf4, 0x17, 0xa7, 0xcd, 0x7c, 0x15, 0x72, 0x2a, 0x17, 0x4c, 0xbc, + 0x31, 0xdb, 0x15, 0x5c, 0x25, 0xc5, 0x8e, 0x89, 0x51, 0x9e, 0x1e, 0xce, 0x83, 0x5b, 0xd5, 0xd5, + 0x15, 0xae, 0x22, 0xf2, 0xbf, 0x56, 0xfe, 0xff, 0x39, 0x98, 0xae, 0xee, 0xb3, 0x35, 0xef, 0x3a, + 0x0b, 0x0b, 0xd0, 0x2d, 0xc8, 0x74, 0x2c, 0x73, 0x57, 0x77, 0x9b, 0x51, 0x0c, 0x85, 0x4c, 0x6e, + 0x33, 0x7a, 0xb8, 0x36, 0x39, 0x87, 0xe4, 0xf1, 0xa2, 0x06, 0x64, 0xef, 0x9a, 0xaa, 0xd2, 0xba, + 0xa5, 0xb7, 0x5c, 0xfb, 0x7f, 0x69, 0xb4, 0xa0, 0x45, 0x8f, 0x67, 0x53, 0x71, 0x76, 0xdc, 0xae, + 0xf0, 0x88, 0x68, 0x15, 0x32, 0x35, 0xc7, 0xe9, 0x90, 0x97, 0xdc, 0x9b, 0x5c, 0x1e, 0x43, 0x28, + 0x61, 0xe1, 0xb2, 0x3c, 0x76, 0xd4, 0x80, 0x99, 0xdb, 0xa6, 0xd9, 0x6c, 0xe1, 0x4a, 0xcb, 0xec, + 0x6a, 0x15, 0xd3, 0xd8, 0xd6, 0x9b, 0xdc, 0x1f, 0x5f, 0x1a, 0x43, 0xe6, 0xed, 0x4a, 0x5d, 0xea, + 0x17, 0x80, 0x96, 0x20, 0x53, 0xbf, 0xc9, 0x85, 0xb1, 0x38, 0xee, 0xe2, 0x18, 0xc2, 0xea, 0x37, + 0x25, 0x8f, 0x0d, 0xdd, 0x81, 0xdc, 0xd2, 0x93, 0xae, 0x85, 0xb9, 0x94, 0xf4, 0xc0, 0xcc, 0x85, + 0x5e, 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x3a, 0x14, 0xdf, 0x36, 0xad, 0x47, 0x2d, 0x53, 0x71, + 0x5b, 0x38, 0x45, 0xc5, 0x7d, 0x66, 0x0c, 0x71, 0x2e, 0xa3, 0xd4, 0x23, 0x62, 0xee, 0x4b, 0x50, + 0x08, 0x75, 0x13, 0x42, 0x90, 0xea, 0x90, 0x1e, 0x11, 0x68, 0x2a, 0x10, 0xfd, 0x1f, 0xbd, 0x08, + 0x53, 0x86, 0xa9, 0x61, 0xd7, 0x86, 0x0b, 0xcb, 0x27, 0x9e, 0x1e, 0xce, 0xa7, 0x37, 0x4c, 0x8d, + 0x05, 0x14, 0xfc, 0x3f, 0x29, 0x4d, 0x0a, 0xb9, 0xe1, 0xc4, 0xdc, 0x25, 0x48, 0x91, 0xfe, 0x21, + 0x6e, 0xe4, 0xa1, 0x62, 0xe3, 0x2d, 0x4b, 0xe7, 0x32, 0xdd, 0x47, 0x5e, 0xee, 0x27, 0x02, 0x24, + 0xea, 0x37, 0x49, 0x44, 0xfd, 0xb0, 0xab, 0x3e, 0xc2, 0x0e, 0x2f, 0xc5, 0x9f, 0x68, 0xa4, 0x6d, + 0xe1, 0x6d, 0x9d, 0x45, 0x39, 0x59, 0x89, 0x3f, 0xa1, 0xe7, 0x00, 0x14, 0x55, 0xc5, 0xb6, 0x2d, + 0xbb, 0x87, 0xc8, 0xb2, 0x52, 0x96, 0x51, 0xd6, 0xf0, 0x01, 0x61, 0xb3, 0xb1, 0x6a, 0x61, 0xc7, + 0xcd, 0x69, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xed, 0x8e, 0xec, 0x98, 0x8f, 0xb0, 0x41, 0x7b, 0x35, + 0x4b, 0xdc, 0x43, 0xbb, 0xd3, 0x20, 0x04, 0xe2, 0xd9, 0xb0, 0xa1, 0xf9, 0x6e, 0x28, 0x2b, 0x79, + 0xcf, 0x44, 0xa4, 0x85, 0x9b, 0x3a, 0x3f, 0x15, 0x95, 0x95, 0xf8, 0x13, 0xd1, 0x98, 0xd2, 0x75, + 0x76, 0x68, 0x22, 0x47, 0x56, 0xa2, 0xff, 0xf3, 0xa6, 0x7d, 0x53, 0x80, 0xe4, 0xed, 0x4a, 0xfd, + 0xc8, 0x6d, 0x73, 0x25, 0x26, 0x7d, 0x89, 0x34, 0x95, 0x50, 0x6f, 0xb5, 0x74, 0xa3, 0x49, 0x82, + 0x8e, 0xaf, 0x62, 0xd5, 0x6d, 0x59, 0x91, 0x93, 0x37, 0x19, 0x15, 0x2d, 0x40, 0x4e, 0xb5, 0xb0, + 0x86, 0x0d, 0x47, 0x57, 0x5a, 0x36, 0x6f, 0x62, 0x90, 0xc4, 0x2b, 0xf7, 0x35, 0x01, 0x26, 0xa9, + 0x79, 0xa1, 0x67, 0x21, 0xab, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0xf7, 0x13, 0x59, 0xc9, 0x27, 0x0c, + 0xac, 0xe4, 0x39, 0xc8, 0x2b, 0xaa, 0x6a, 0x76, 0x0d, 0x47, 0x36, 0x94, 0x36, 0xe6, 0x95, 0xcd, + 0x71, 0xda, 0x86, 0xd2, 0xc6, 0x68, 0x1e, 0xdc, 0x47, 0xef, 0x28, 0x5f, 0x56, 0x02, 0x4e, 0x5a, + 0xc3, 0x07, 0xbc, 0x26, 0x3f, 0x14, 0x20, 0xe3, 0x9a, 0x25, 0xa9, 0x4c, 0x13, 0x1b, 0xd8, 0x52, + 0x1c, 0xd3, 0xab, 0x8c, 0x47, 0xe8, 0x9d, 0x93, 0xb2, 0xfe, 0x9c, 0x74, 0x02, 0x26, 0x1d, 0xe5, + 0x61, 0xcb, 0xad, 0x07, 0x7b, 0xa0, 0xcb, 0xc7, 0x2d, 0xa5, 0xc9, 0xd6, 0xc8, 0xb2, 0x12, 0x7b, + 0x20, 0x4d, 0xe2, 0xb9, 0xab, 0x4c, 0x3b, 0xfc, 0x89, 0xd4, 0x97, 0xa5, 0x6b, 0x3e, 0xc4, 0x4d, + 0xdd, 0xa0, 0x06, 0x90, 0x94, 0x80, 0x92, 0x96, 0x09, 0x05, 0x3d, 0x03, 0x59, 0x56, 0x00, 0x1b, + 0x1a, 0xb5, 0x82, 0xa4, 0x94, 0xa1, 0x84, 0xaa, 0x7b, 0x76, 0x89, 0x07, 0x22, 0xdf, 0x15, 0x60, + 0x86, 0xe5, 0xdc, 0xb0, 0x3c, 0xcf, 0xf8, 0x66, 0xe5, 0xd7, 0x20, 0xab, 0x29, 0x8e, 0xc2, 0x8e, + 0x0e, 0x26, 0x86, 0x1e, 0x1d, 0x74, 0xdd, 0x24, 0x29, 0x4f, 0x8f, 0x0f, 0x22, 0x48, 0x91, 0xff, + 0xd9, 0x91, 0x4b, 0x89, 0xfe, 0xef, 0x67, 0x31, 0x04, 0xab, 0x1b, 0x67, 0x94, 0xf2, 0xe1, 0x24, + 0x14, 0xaa, 0xfb, 0x1d, 0xd3, 0x8a, 0x79, 0x11, 0x6d, 0x8a, 0x83, 0xdf, 0x21, 0xdb, 0xac, 0x3d, + 0x5e, 0xd0, 0xdd, 0xc1, 0xe4, 0x8c, 0x68, 0x19, 0x80, 0x25, 0x58, 0xd2, 0x3c, 0x9c, 0xe4, 0x11, + 0x36, 0x9b, 0x28, 0x1b, 0xa1, 0xa2, 0x0d, 0xc8, 0xb5, 0x77, 0x55, 0x55, 0xde, 0xd6, 0x5b, 0x0e, + 0x4f, 0x4d, 0x8b, 0xce, 0x85, 0x5e, 0xbf, 0x5f, 0xa9, 0xdc, 0xa2, 0x85, 0x58, 0x86, 0x98, 0xff, + 0x2c, 0x01, 0x91, 0xc0, 0xfe, 0x47, 0x2f, 0x00, 0x3f, 0xea, 0x21, 0xdb, 0xee, 0xc1, 0xad, 0xe5, + 0xc2, 0xd3, 0xc3, 0xf9, 0xac, 0x44, 0xa9, 0xf5, 0x7a, 0x43, 0xca, 0xb2, 0x02, 0x75, 0xdb, 0x41, + 0xe7, 0xa1, 0x60, 0xb6, 0x75, 0x47, 0x76, 0xe3, 0x00, 0x1e, 0x3a, 0xe5, 0x09, 0xd1, 0x8d, 0x13, + 0x50, 0x03, 0x2e, 0x63, 0x83, 0x8c, 0x06, 0xda, 0x4e, 0xf9, 0x21, 0x5b, 0xab, 0x73, 0xd8, 0x88, + 0x92, 0xcd, 0x8e, 0xa3, 0xb7, 0xf5, 0x27, 0x74, 0xa3, 0x97, 0x6f, 0xb2, 0x9c, 0x67, 0xc5, 0x49, + 0xfb, 0x96, 0xe9, 0x22, 0x1e, 0x2f, 0x7b, 0x2f, 0x50, 0x14, 0x7d, 0x4d, 0x80, 0x53, 0x5c, 0x91, + 0xf2, 0xc3, 0x03, 0xb9, 0x45, 0x26, 0x11, 0xdd, 0x39, 0x90, 0x1f, 0xed, 0x96, 0x32, 0x34, 0x40, + 0xfb, 0x5c, 0x64, 0x87, 0x04, 0xec, 0x60, 0xd1, 0xed, 0x96, 0x83, 0xbb, 0x9c, 0x79, 0x6d, 0xb7, + 0x6a, 0x38, 0xd6, 0xc1, 0xf2, 0xe9, 0xa7, 0x87, 0xf3, 0xb3, 0xfd, 0x6f, 0xef, 0x4b, 0xb3, 0x76, + 0x3f, 0xcb, 0xdc, 0x57, 0xa1, 0x34, 0x48, 0x12, 0x12, 0xfd, 0x9d, 0xc6, 0x2c, 0xdb, 0x60, 0x7c, + 0x35, 0xbc, 0x42, 0x30, 0x86, 0xd9, 0xb8, 0xab, 0x04, 0x89, 0x57, 0xdd, 0xd1, 0xfd, 0x0d, 0x01, + 0x0a, 0xcb, 0xdd, 0xd6, 0xa3, 0x7b, 0x9d, 0x7a, 0xb7, 0xdd, 0x56, 0xac, 0x03, 0xe2, 0x18, 0xd8, + 0xa8, 0xd4, 0x9f, 0xb0, 0x14, 0x95, 0x24, 0x1f, 0x76, 0xfa, 0x13, 0x4c, 0x86, 0x1d, 0x4f, 0x40, + 0x27, 0x74, 0x96, 0x5d, 0x7e, 0x1e, 0x0a, 0x14, 0xae, 0xcb, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0x55, + 0xa8, 0xa4, 0x94, 0xa7, 0xc4, 0x2a, 0xa3, 0xa1, 0x8b, 0x50, 0xb4, 0x0f, 0x6c, 0x07, 0xb7, 0x65, + 0x76, 0xd0, 0x9b, 0x61, 0xcc, 0xa4, 0x54, 0x60, 0x54, 0x89, 0x11, 0xcb, 0x3f, 0x4d, 0x42, 0xd1, + 0xd5, 0x6c, 0x9c, 0xe1, 0xeb, 0x32, 0x4c, 0x6e, 0xeb, 0x2d, 0xec, 0x66, 0x09, 0x5c, 0x1a, 0xd2, + 0xa1, 0x3c, 0xb1, 0x98, 0x44, 0x11, 0x2e, 0x00, 0xa2, 0xac, 0x71, 0x8c, 0xb1, 0xb9, 0xff, 0x9d, + 0x80, 0x14, 0x8d, 0x18, 0xaf, 0x43, 0x8a, 0xba, 0x3d, 0x61, 0x1c, 0xb7, 0x47, 0x8b, 0x7a, 0xe1, + 0x4c, 0x22, 0x10, 0xce, 0x90, 0xd8, 0x60, 0x47, 0x79, 0xf9, 0xfa, 0x0d, 0x3a, 0xbe, 0xf2, 0x12, + 0x7f, 0x42, 0xcb, 0x34, 0x7d, 0xc5, 0xb4, 0x1c, 0xac, 0xf1, 0x48, 0x2d, 0xca, 0x33, 0x85, 0x3a, + 0xde, 0x75, 0xb1, 0x2e, 0x1f, 0x3a, 0x03, 0x49, 0x32, 0x70, 0xa7, 0xd8, 0xd6, 0xf6, 0xd3, 0xc3, + 0xf9, 0x24, 0x19, 0xb2, 0x84, 0x86, 0xae, 0x41, 0x2e, 0x3c, 0x4a, 0x84, 0x2b, 0x59, 0xe6, 0x0b, + 0x02, 0x16, 0x0e, 0x2d, 0xcf, 0x82, 0x19, 0x4a, 0xb9, 0x93, 0xca, 0xa4, 0xc4, 0xc9, 0xf2, 0x1f, + 0xa6, 0xa0, 0xb0, 0xda, 0x8e, 0xdb, 0x8b, 0x2e, 0x85, 0x7b, 0x38, 0x2a, 0xbc, 0x0d, 0x7d, 0x34, + 0xa2, 0x83, 0x43, 0xf3, 0x51, 0xf2, 0x68, 0xf3, 0xd1, 0x2a, 0x89, 0xa8, 0xf8, 0x61, 0xf6, 0xe4, + 0x80, 0x48, 0x36, 0xfc, 0xfd, 0x06, 0x71, 0x4c, 0x12, 0xe1, 0xf1, 0x53, 0xed, 0x69, 0x7a, 0xc2, + 0x9b, 0x34, 0x70, 0x63, 0x56, 0x96, 0x1e, 0xdf, 0xca, 0xa6, 0xb0, 0xa1, 0x51, 0x1b, 0x7b, 0xc2, + 0x4d, 0xec, 0x35, 0x48, 0x6a, 0xba, 0x35, 0xe4, 0x66, 0x84, 0xe8, 0x39, 0x85, 0x30, 0x8d, 0xb0, + 0xb5, 0x54, 0xd0, 0xd6, 0x82, 0x38, 0x74, 0xee, 0x1e, 0x80, 0xdf, 0x2e, 0xb4, 0x00, 0x69, 0xb3, + 0xa5, 0xb9, 0xe7, 0x04, 0x0a, 0xcb, 0xd9, 0xa7, 0x87, 0xf3, 0x93, 0xf7, 0x5a, 0xda, 0xea, 0x8a, + 0x34, 0x69, 0xb6, 0xb4, 0x55, 0x8d, 0xde, 0x02, 0x80, 0xf7, 0x64, 0x2f, 0xc7, 0x28, 0x2f, 0x4d, + 0x19, 0x78, 0x8f, 0xa0, 0xde, 0x9e, 0xdc, 0x07, 0x62, 0x38, 0xdf, 0x11, 0xa0, 0xe8, 0xea, 0x30, + 0x5e, 0xe7, 0x90, 0xd1, 0xdb, 0x7c, 0xb0, 0x24, 0x8f, 0x36, 0x58, 0x5c, 0x3e, 0x7e, 0x56, 0xf1, + 0xeb, 0x02, 0xcf, 0x2f, 0xad, 0xab, 0x8a, 0x43, 0xe6, 0xc3, 0x18, 0x0d, 0xfc, 0x79, 0x10, 0x2d, + 0xc5, 0xd0, 0xcc, 0xb6, 0xfe, 0x04, 0xb3, 0x85, 0x2b, 0x9b, 0xef, 0x4f, 0x4d, 0x7b, 0x74, 0xba, + 0x32, 0xe3, 0xae, 0xbb, 0xfd, 0x42, 0xe0, 0xb9, 0xa8, 0x5e, 0x65, 0xe2, 0x54, 0xda, 0x1a, 0xa4, + 0x2d, 0x96, 0xd1, 0xc6, 0x06, 0xdc, 0x8b, 0x11, 0x42, 0xa2, 0xbe, 0xce, 0x12, 0xc6, 0x3c, 0x93, + 0xa7, 0x22, 0xe6, 0xbe, 0x08, 0x93, 0x94, 0x7c, 0x0c, 0xb7, 0xc8, 0x35, 0xff, 0x0f, 0x09, 0xb8, + 0x40, 0x3f, 0x77, 0x1f, 0x5b, 0xfa, 0xf6, 0xc1, 0xa6, 0x65, 0x3a, 0x58, 0x75, 0xb0, 0xe6, 0xe7, + 0xe3, 0xc7, 0xea, 0x6b, 0xb2, 0x1d, 0xf7, 0x03, 0x47, 0xca, 0xec, 0xf1, 0xb8, 0xd0, 0x1a, 0x4c, + 0xb3, 0x89, 0x50, 0x56, 0x5a, 0xfa, 0x2e, 0x96, 0x15, 0xe7, 0x28, 0x33, 0x4a, 0x81, 0xf1, 0x2e, + 0x11, 0xd6, 0x25, 0x07, 0x69, 0x90, 0xe5, 0xc2, 0x74, 0x8d, 0xdf, 0x32, 0x72, 0xfb, 0xe3, 0x2d, + 0xcd, 0x64, 0xd8, 0x84, 0xbc, 0xba, 0x22, 0x65, 0x98, 0x64, 0x6f, 0x69, 0xfd, 0xc7, 0x02, 0x5c, + 0x1c, 0xa1, 0xe8, 0x38, 0xcd, 0x6c, 0x0e, 0x32, 0xbb, 0xe4, 0x43, 0x3a, 0xd7, 0x74, 0x46, 0xf2, + 0x9e, 0xd1, 0x3a, 0x14, 0xb6, 0x15, 0xbd, 0x85, 0x35, 0x99, 0x5b, 0xe2, 0xe0, 0x74, 0xb0, 0xe8, + 0x2c, 0xc5, 0x3c, 0x63, 0xa7, 0x2f, 0xed, 0xf2, 0x6f, 0x25, 0x60, 0x66, 0x49, 0xd3, 0xea, 0x75, + 0xee, 0xc1, 0xe2, 0xb3, 0x17, 0x17, 0xac, 0x24, 0x7c, 0xb0, 0x82, 0x5e, 0x04, 0xa4, 0xe9, 0x36, + 0xbb, 0xe6, 0xc0, 0xde, 0x51, 0x34, 0x73, 0xcf, 0xdf, 0xf8, 0x9e, 0x71, 0xdf, 0xd4, 0xdd, 0x17, + 0xe8, 0x1d, 0xa0, 0xa1, 0xb5, 0x6c, 0x3b, 0x8a, 0xb7, 0x3e, 0xff, 0xc2, 0x51, 0x4e, 0xd1, 0xb0, + 0xd0, 0xdb, 0x7b, 0x94, 0xb2, 0x44, 0x1c, 0xfd, 0x17, 0x5d, 0x01, 0x51, 0x27, 0x1a, 0x70, 0x64, + 0xc5, 0x76, 0x0f, 0x62, 0xb0, 0x63, 0xa9, 0x45, 0x46, 0x5f, 0xb2, 0x83, 0xe7, 0x2b, 0x58, 0x9e, + 0xb8, 0xaf, 0xa7, 0x38, 0x11, 0xd6, 0xef, 0x08, 0x50, 0xe4, 0x87, 0x47, 0xe2, 0xec, 0x80, 0x5b, + 0x90, 0x77, 0x4f, 0xaa, 0x6c, 0x5b, 0x66, 0xfb, 0x28, 0x43, 0x2d, 0xc7, 0x19, 0x6f, 0x59, 0x66, + 0x3b, 0x74, 0x2e, 0xfd, 0x3e, 0x4c, 0x7b, 0x35, 0x8d, 0x53, 0x05, 0xdf, 0xa5, 0x07, 0x48, 0x99, + 0xe0, 0xb8, 0xb7, 0x9e, 0x3f, 0x09, 0x3d, 0xd0, 0x7d, 0x81, 0x60, 0x75, 0xe3, 0x54, 0xc6, 0x2f, + 0x04, 0x28, 0xd6, 0xbb, 0x0f, 0xd9, 0x95, 0x3b, 0xf1, 0xe9, 0xa1, 0x0a, 0xd9, 0x16, 0xde, 0x76, + 0xe4, 0x63, 0xe5, 0x36, 0x67, 0x08, 0x2b, 0xcd, 0xef, 0xbe, 0x0d, 0x60, 0xd1, 0xd3, 0x4b, 0x54, + 0x4e, 0xf2, 0x88, 0x72, 0xb2, 0x94, 0xd7, 0x8f, 0x75, 0xca, 0x7f, 0x90, 0x80, 0x69, 0xaf, 0xb1, + 0x71, 0x3a, 0xd1, 0xff, 0x11, 0x72, 0x1e, 0xc9, 0x63, 0x38, 0x8f, 0x19, 0xbe, 0xe5, 0x1e, 0xed, + 0x40, 0x16, 0x61, 0x96, 0x06, 0x24, 0xb2, 0xd2, 0xe9, 0xb4, 0x74, 0xac, 0xc9, 0x6c, 0x2f, 0x37, + 0x45, 0xf7, 0x72, 0x67, 0xe8, 0xab, 0x25, 0xf6, 0x66, 0x95, 0xee, 0xeb, 0xde, 0x82, 0xfc, 0xb6, + 0x85, 0xf1, 0x13, 0x2c, 0x53, 0x64, 0x74, 0x94, 0xcc, 0x82, 0x1c, 0x63, 0xac, 0x13, 0x3e, 0x6e, + 0x80, 0xef, 0xc2, 0x0c, 0x55, 0x70, 0xdc, 0x27, 0x27, 0x79, 0xaf, 0xfc, 0xa3, 0x00, 0x28, 0x28, + 0xff, 0x93, 0xeb, 0x98, 0x44, 0xec, 0x1d, 0xf3, 0x02, 0x20, 0x96, 0x64, 0x66, 0xcb, 0x1d, 0x6c, + 0xc9, 0x36, 0x56, 0x4d, 0x7e, 0x5b, 0x8c, 0x20, 0x89, 0xfc, 0xcd, 0x26, 0xb6, 0xea, 0x94, 0x5e, + 0x7e, 0x6f, 0x0e, 0xf2, 0x5c, 0x27, 0x5b, 0x86, 0x6e, 0x1a, 0xe8, 0x3a, 0x24, 0x9b, 0x7c, 0xa5, + 0x37, 0x17, 0xb9, 0x12, 0xe4, 0xdf, 0x58, 0x55, 0x9b, 0x90, 0x48, 0x59, 0xc2, 0xd2, 0xe9, 0x3a, + 0x11, 0x41, 0x91, 0x9f, 0x1b, 0x1b, 0x64, 0xe9, 0x74, 0x1d, 0x54, 0x87, 0x69, 0xd5, 0xbf, 0xb1, + 0x47, 0x26, 0xec, 0xc9, 0x81, 0x9b, 0x0b, 0x91, 0xf7, 0x1e, 0xd5, 0x26, 0xa4, 0xa2, 0x1a, 0x7a, + 0x81, 0x2a, 0xc1, 0x2b, 0x62, 0xd8, 0x74, 0x79, 0x3e, 0xf2, 0x94, 0x65, 0xf8, 0x7a, 0x9a, 0xda, + 0x44, 0xe0, 0x26, 0x19, 0xf4, 0x1a, 0xa4, 0x35, 0x7a, 0xf5, 0x08, 0xb7, 0xd0, 0x28, 0x23, 0x0a, + 0xdd, 0xf6, 0x52, 0x9b, 0x90, 0x38, 0x07, 0xba, 0x03, 0x79, 0xf6, 0x1f, 0x0b, 0x4e, 0x38, 0x92, + 0xbb, 0x38, 0x58, 0x42, 0xc0, 0xd7, 0xd7, 0x26, 0xa4, 0x9c, 0xe6, 0x53, 0xd1, 0x67, 0x21, 0x65, + 0xab, 0x8a, 0xc1, 0x37, 0x49, 0xce, 0x0e, 0xb8, 0x9e, 0xc0, 0x67, 0xa6, 0xa5, 0xd1, 0xeb, 0xec, + 0xe6, 0x39, 0x67, 0x9f, 0x1d, 0xbc, 0x8c, 0xae, 0x7e, 0xe8, 0x00, 0x2d, 0xa9, 0x3e, 0xa6, 0x04, + 0x74, 0x1b, 0x72, 0x0a, 0x89, 0xf2, 0x64, 0x7a, 0x8c, 0xad, 0x04, 0x03, 0x0f, 0x1a, 0xf5, 0x1d, + 0x41, 0xac, 0xd1, 0x53, 0xba, 0x2e, 0xd1, 0x17, 0xd4, 0xc6, 0x56, 0x13, 0x97, 0x72, 0xc3, 0x05, + 0x05, 0xb3, 0x68, 0x3c, 0x41, 0x94, 0x48, 0xa2, 0xbd, 0x1d, 0xf7, 0x88, 0x02, 0x6d, 0x54, 0x7e, + 0xe0, 0xa6, 0x58, 0xc4, 0x11, 0x8b, 0xda, 0x84, 0x94, 0xdf, 0x09, 0x90, 0xd1, 0x22, 0x24, 0x9a, + 0x6a, 0xa9, 0x40, 0x65, 0x3c, 0x3b, 0xec, 0x00, 0x41, 0x6d, 0x42, 0x4a, 0x34, 0x55, 0x82, 0xca, + 0x59, 0xea, 0xf7, 0xbe, 0x51, 0x2a, 0x0e, 0x1c, 0xf1, 0xe1, 0x3c, 0xfa, 0xda, 0x84, 0x44, 0xb3, + 0xcd, 0xc9, 0xf7, 0x36, 0xa1, 0x68, 0xb1, 0x34, 0x24, 0x37, 0xb9, 0x50, 0x1c, 0xb8, 0x51, 0x18, + 0x95, 0x5f, 0x58, 0xa3, 0x51, 0x7f, 0x80, 0x8e, 0xbe, 0x02, 0x27, 0xc2, 0x12, 0xb9, 0xa5, 0xcd, + 0xf4, 0xb9, 0x91, 0x68, 0xb9, 0x61, 0x83, 0x43, 0x56, 0xdf, 0x4b, 0xf4, 0x0a, 0x4c, 0xb2, 0x5e, + 0x43, 0x54, 0x64, 0xd4, 0x0e, 0x78, 0x4f, 0x87, 0xb1, 0xf2, 0xc4, 0xf8, 0x1d, 0x9e, 0x7f, 0x23, + 0xb7, 0xcc, 0x66, 0x69, 0x76, 0xa0, 0xf1, 0xf7, 0xe7, 0x13, 0x11, 0xe3, 0x77, 0x7c, 0x2a, 0xe9, + 0x77, 0x8b, 0xbd, 0xe1, 0xe9, 0x1a, 0x27, 0x06, 0xf6, 0x7b, 0x44, 0x5a, 0x4e, 0x8d, 0xe6, 0x4e, + 0xfb, 0x64, 0x52, 0x35, 0x8b, 0x5d, 0xcf, 0x21, 0xd3, 0x31, 0x75, 0x72, 0x60, 0xd5, 0xfa, 0xaf, + 0x1c, 0xa9, 0xd1, 0x30, 0xc8, 0xa3, 0xa2, 0xfb, 0x20, 0xf2, 0x83, 0xf3, 0xfe, 0xb2, 0xf5, 0x29, + 0x2a, 0xef, 0xf9, 0x48, 0xd7, 0x15, 0x95, 0xdf, 0x50, 0x9b, 0x90, 0xa6, 0xd5, 0xf0, 0x1b, 0xf4, + 0x00, 0x66, 0xa8, 0x3c, 0x59, 0xf5, 0x6f, 0x3c, 0x28, 0x95, 0xfa, 0x12, 0x05, 0x07, 0x5f, 0x8e, + 0xe0, 0x4a, 0x16, 0xd5, 0x9e, 0x57, 0xc4, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x97, 0x9d, 0x1b, 0x68, + 0xc6, 0xe1, 0x1b, 0xce, 0x88, 0x19, 0xeb, 0x8c, 0x42, 0xcc, 0xd8, 0xe1, 0xb9, 0x3c, 0xbc, 0x3b, + 0x9e, 0x1d, 0x68, 0xc6, 0x51, 0x49, 0x3f, 0xc4, 0x8c, 0x9d, 0x20, 0x9d, 0x98, 0x31, 0x73, 0x10, + 0x3d, 0x72, 0x9f, 0x1b, 0x68, 0xc6, 0x03, 0x0f, 0x92, 0x12, 0x33, 0x56, 0xfa, 0x5e, 0xa2, 0x15, + 0x00, 0x16, 0x9e, 0xe8, 0xc6, 0xb6, 0x59, 0x3a, 0x3b, 0x70, 0x32, 0xe8, 0xcd, 0xe6, 0x21, 0x93, + 0x41, 0xcb, 0xa5, 0x11, 0x47, 0x46, 0xb1, 0x91, 0x4c, 0x77, 0xc9, 0x4a, 0xf3, 0x03, 0x1d, 0x59, + 0xdf, 0x66, 0x19, 0x71, 0x64, 0x7b, 0x1e, 0x91, 0xcc, 0x2a, 0x6c, 0x8d, 0xb5, 0xb4, 0x30, 0xd8, + 0x2d, 0x07, 0x77, 0x17, 0xa8, 0x5b, 0xa6, 0x04, 0xb4, 0x04, 0x59, 0x32, 0x6d, 0x1f, 0x50, 0x37, + 0x74, 0x6e, 0x60, 0xc0, 0xd9, 0x73, 0x38, 0xa0, 0x36, 0x21, 0x65, 0x1e, 0x73, 0x12, 0xf9, 0x3c, + 0x5b, 0xb5, 0x2a, 0x95, 0x07, 0x7e, 0x3e, 0xb4, 0x52, 0x49, 0x3e, 0xcf, 0x38, 0x90, 0x0a, 0x27, + 0x59, 0x5f, 0xf1, 0x73, 0x9c, 0x16, 0x3f, 0x74, 0x58, 0x3a, 0x4f, 0x45, 0x0d, 0x5c, 0x03, 0x8a, + 0x3c, 0x5e, 0x5a, 0x9b, 0x90, 0x66, 0x95, 0xfe, 0xb7, 0x64, 0xc0, 0xf3, 0xa9, 0x87, 0xad, 0x1c, + 0x95, 0x2e, 0x0c, 0x1c, 0xf0, 0x11, 0x6b, 0x6d, 0x64, 0xc0, 0x2b, 0x01, 0x32, 0x9b, 0x80, 0x34, + 0xd9, 0xb6, 0xd9, 0x9e, 0xea, 0xc5, 0x21, 0x13, 0x50, 0x0f, 0xf6, 0x67, 0x13, 0x90, 0x56, 0x67, + 0x9c, 0x44, 0x90, 0xda, 0xc2, 0x8a, 0xc5, 0xdd, 0xec, 0xa5, 0x81, 0x82, 0xfa, 0x6e, 0x0d, 0x23, + 0x82, 0x54, 0x8f, 0x48, 0x02, 0x1e, 0xcb, 0xbd, 0x71, 0x83, 0x87, 0x7e, 0x97, 0x07, 0x06, 0x3c, + 0x91, 0x17, 0x83, 0x90, 0x80, 0xc7, 0x0a, 0xbd, 0x40, 0x9f, 0x87, 0x29, 0x8e, 0xd0, 0x4a, 0x57, + 0x86, 0x04, 0xa4, 0x41, 0x68, 0x4d, 0xc6, 0x35, 0xe7, 0x61, 0x5e, 0x96, 0x21, 0x43, 0xd6, 0xbc, + 0xe7, 0x87, 0x78, 0xd9, 0x3e, 0x70, 0xca, 0xbc, 0xac, 0x4f, 0x26, 0x5e, 0x96, 0xd9, 0x29, 0x9f, + 0xeb, 0xae, 0x0e, 0xf4, 0xb2, 0xfd, 0x27, 0x13, 0x88, 0x97, 0x7d, 0xec, 0x53, 0x49, 0xcb, 0x6c, + 0x86, 0x8a, 0x4a, 0x9f, 0x19, 0xd8, 0xb2, 0x30, 0x48, 0x24, 0x2d, 0xe3, 0x3c, 0xa4, 0xdb, 0x58, + 0x12, 0x29, 0xd3, 0xf4, 0x0b, 0x83, 0x8f, 0x4c, 0xf7, 0x82, 0x88, 0x9a, 0x7b, 0xa3, 0x2c, 0xd3, + 0xb0, 0xe7, 0xa8, 0x2c, 0x7e, 0x40, 0x94, 0x6b, 0xea, 0xc5, 0xe1, 0x8e, 0x2a, 0xea, 0xec, 0xab, + 0xe7, 0xa8, 0x42, 0x2f, 0x69, 0x55, 0xd9, 0x39, 0x1f, 0x3a, 0xbe, 0x17, 0x87, 0x9c, 0xee, 0xee, + 0x39, 0x7a, 0x45, 0xab, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x65, 0xd7, 0x10, 0x94, 0xae, 0x0d, 0x1f, + 0x42, 0xe1, 0xeb, 0x10, 0xbc, 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x46, 0x18, 0x2f, 0x0d, 0x9f, + 0x33, 0x7b, 0x43, 0x0b, 0x36, 0x67, 0xf2, 0x98, 0xe2, 0xff, 0x08, 0xb0, 0xc0, 0xea, 0x46, 0xd7, + 0xf1, 0x0e, 0x64, 0x6f, 0x4d, 0x34, 0x90, 0x6a, 0x7e, 0x9d, 0x7e, 0xe0, 0x95, 0x41, 0xd5, 0x1d, + 0xb1, 0xc6, 0x5b, 0x9b, 0x90, 0x9e, 0x53, 0x86, 0x95, 0x5b, 0x9e, 0xe2, 0x3b, 0xa7, 0xde, 0x89, + 0xbb, 0x69, 0x51, 0xbc, 0x93, 0xca, 0x9c, 0x16, 0x4b, 0x77, 0x52, 0x99, 0x33, 0xe2, 0xdc, 0x9d, + 0x54, 0xe6, 0x19, 0xf1, 0xd9, 0xf2, 0x3f, 0x9d, 0x81, 0x82, 0x8b, 0xe1, 0x18, 0x22, 0xba, 0x11, + 0x44, 0x44, 0x67, 0x07, 0x21, 0x22, 0x8e, 0xfa, 0x38, 0x24, 0xba, 0x11, 0x84, 0x44, 0x67, 0x07, + 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, 0xc6, 0x20, 0x4c, 0xf4, 0xfc, 0x18, 0x98, 0xc8, 0x13, 0xd5, + 0x0b, 0x8a, 0x56, 0xfa, 0x41, 0xd1, 0x85, 0xe1, 0xa0, 0xc8, 0x13, 0x15, 0x40, 0x45, 0xaf, 0xf7, + 0xa0, 0xa2, 0x73, 0x43, 0x50, 0x91, 0xc7, 0xef, 0xc2, 0xa2, 0xb5, 0x48, 0x58, 0x74, 0x69, 0x14, + 0x2c, 0xf2, 0xe4, 0x84, 0x70, 0xd1, 0xcb, 0x21, 0x5c, 0x34, 0x3f, 0x10, 0x17, 0x79, 0xdc, 0x0c, + 0x18, 0xbd, 0xd1, 0x0b, 0x8c, 0xce, 0x0d, 0x01, 0x46, 0x7e, 0x0b, 0x38, 0x32, 0xaa, 0x45, 0x21, + 0xa3, 0x8b, 0x23, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xe2, 0x08, 0x68, + 0xd4, 0x23, 0x89, 0x61, 0xa3, 0x8d, 0x68, 0x6c, 0x74, 0x79, 0x24, 0x36, 0xf2, 0xa4, 0x85, 0xc1, + 0xd1, 0xb5, 0x00, 0x38, 0x7a, 0x6e, 0x00, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0x42, 0x1f, 0x3a, + 0x2a, 0x0f, 0x43, 0x47, 0x1e, 0xaf, 0x07, 0x8f, 0xde, 0x1a, 0x00, 0x8f, 0xae, 0x8c, 0x86, 0x47, + 0x9e, 0xb0, 0x1e, 0x7c, 0xa4, 0x0c, 0xc5, 0x47, 0x2f, 0x8e, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, + 0xe9, 0xd5, 0x30, 0x40, 0x5a, 0x18, 0x0c, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0xad, 0x45, 0x22, 0xa4, + 0x4b, 0xa3, 0x10, 0x92, 0x3f, 0x0e, 0x82, 0x10, 0x69, 0x23, 0x1a, 0x22, 0x5d, 0x1e, 0x09, 0x91, + 0xfc, 0xee, 0x0f, 0x61, 0xa4, 0xb5, 0x48, 0x8c, 0x74, 0x69, 0x14, 0x46, 0xf2, 0x2b, 0x17, 0x04, + 0x49, 0x6f, 0x0f, 0x04, 0x49, 0x57, 0xc7, 0x01, 0x49, 0x9e, 0xd0, 0x3e, 0x94, 0xf4, 0xce, 0x60, + 0x94, 0xf4, 0x99, 0x23, 0x5c, 0x21, 0x17, 0x09, 0x93, 0xbe, 0xd0, 0x07, 0x93, 0xca, 0xc3, 0x60, + 0x92, 0x6f, 0xcf, 0x2e, 0x4e, 0x52, 0x86, 0xa2, 0x9a, 0x17, 0xc7, 0x44, 0x35, 0xbe, 0xf1, 0x45, + 0xc0, 0x9a, 0x6a, 0x04, 0xac, 0xb9, 0x30, 0x1c, 0xd6, 0xf8, 0xee, 0xdc, 0xc7, 0x35, 0xb5, 0x28, + 0x5c, 0x73, 0x71, 0x04, 0xae, 0xf1, 0xbd, 0x50, 0x00, 0xd8, 0xbc, 0xde, 0x03, 0x6c, 0xce, 0x8d, + 0xcc, 0xb2, 0x09, 0x20, 0x9b, 0xe5, 0x7e, 0x64, 0x73, 0x7e, 0x28, 0xb2, 0xf1, 0x24, 0xf8, 0xd0, + 0xe6, 0xf5, 0x1e, 0x68, 0x73, 0x6e, 0x08, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1c, 0xdb, + 0x2c, 0x8e, 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x23, 0x1a, 0xdc, 0x5c, 0x1e, 0x73, 0xf7, + 0xbc, 0x0f, 0xdd, 0xd4, 0xa2, 0xd0, 0xcd, 0xc5, 0x11, 0xe8, 0x26, 0x38, 0x87, 0x78, 0xf0, 0xa6, + 0x16, 0x05, 0x6f, 0x2e, 0x8e, 0x80, 0x37, 0xbe, 0xa4, 0x00, 0xbe, 0x69, 0x0c, 0xc2, 0x37, 0xcf, + 0x8f, 0x81, 0x6f, 0xfc, 0xe0, 0xa5, 0x07, 0xe0, 0xbc, 0xd9, 0x0b, 0x70, 0xca, 0xc3, 0x00, 0x8e, + 0x3f, 0x22, 0x5d, 0x84, 0xb3, 0x11, 0x8d, 0x70, 0x2e, 0x8f, 0x44, 0x38, 0x41, 0x27, 0x19, 0x80, + 0x38, 0x6b, 0x91, 0x10, 0xe7, 0xd2, 0x28, 0x88, 0xe3, 0x3b, 0xc9, 0x20, 0xc6, 0x79, 0xb3, 0x17, + 0xe3, 0x94, 0x87, 0x61, 0x1c, 0xbf, 0x71, 0x2e, 0xc8, 0xa9, 0x45, 0x81, 0x9c, 0x8b, 0x23, 0x40, + 0x8e, 0xdf, 0x79, 0x01, 0x94, 0xa3, 0x0c, 0x45, 0x39, 0x2f, 0x8e, 0x89, 0x72, 0x7a, 0x1c, 0x57, + 0x18, 0xe6, 0xd4, 0xa2, 0x60, 0xce, 0xc5, 0x11, 0x30, 0x27, 0x50, 0x59, 0x1f, 0xe7, 0x6c, 0x44, + 0xe3, 0x9c, 0xcb, 0x23, 0x71, 0x4e, 0xcf, 0x68, 0x72, 0x81, 0xce, 0x5a, 0x24, 0xd0, 0xb9, 0x34, + 0x0a, 0xe8, 0xf4, 0x4c, 0x7c, 0x3c, 0x38, 0xf8, 0xbf, 0xe3, 0x23, 0x9d, 0x57, 0x8f, 0x8e, 0x74, + 0xbc, 0x6f, 0xc6, 0x02, 0x75, 0xee, 0xa4, 0x32, 0xcf, 0x8a, 0xcf, 0x95, 0xbf, 0x3f, 0x09, 0xe9, + 0x9a, 0x97, 0xe3, 0xe2, 0xd7, 0x52, 0x38, 0xce, 0xed, 0x35, 0x68, 0x85, 0x8c, 0x58, 0xea, 0xf7, + 0x46, 0x5f, 0x54, 0xd6, 0x7f, 0x89, 0x16, 0x67, 0x3d, 0xc6, 0x21, 0x50, 0xf4, 0x32, 0x14, 0xba, + 0x36, 0xb6, 0xe4, 0x8e, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x74, 0x7d, 0x61, 0x59, 0xfc, 0xe8, 0x70, + 0x3e, 0xbf, 0x65, 0x63, 0x6b, 0x93, 0xd3, 0xa5, 0x7c, 0x37, 0xf0, 0xe4, 0xfe, 0xfc, 0xcd, 0xe4, + 0xf8, 0x3f, 0x7f, 0xf3, 0x16, 0x88, 0x16, 0x56, 0xb4, 0x50, 0x04, 0xc2, 0xae, 0x85, 0x89, 0xb6, + 0x19, 0x7a, 0x56, 0xc5, 0x2d, 0x49, 0xaf, 0x87, 0x99, 0xb6, 0xc2, 0x44, 0x74, 0x1d, 0x4e, 0xb6, + 0x95, 0x7d, 0x9a, 0x83, 0x28, 0xbb, 0x41, 0x1d, 0xcd, 0x2b, 0xcc, 0xd0, 0x7c, 0x59, 0xd4, 0x56, + 0xf6, 0xe9, 0x6f, 0xe9, 0xb0, 0x57, 0xf4, 0x16, 0xfd, 0x8b, 0x50, 0xd4, 0x74, 0xdb, 0xd1, 0x0d, + 0xd5, 0xbd, 0xff, 0x93, 0x5d, 0xb4, 0x59, 0x70, 0xa9, 0xec, 0x1e, 0xce, 0xab, 0x30, 0xc3, 0xf3, + 0xb1, 0xfd, 0x5f, 0xd7, 0xa1, 0xf0, 0x25, 0x43, 0x6a, 0x41, 0x5e, 0xf8, 0x3f, 0xab, 0x54, 0x81, + 0xe9, 0xa6, 0xe2, 0xe0, 0x3d, 0xe5, 0x40, 0x76, 0x8f, 0xcb, 0xe4, 0xe8, 0xb5, 0x7a, 0xcf, 0x3c, + 0x3d, 0x9c, 0x2f, 0xdc, 0x66, 0xaf, 0xfa, 0x4e, 0xcd, 0x14, 0x9a, 0x81, 0x17, 0x1a, 0xba, 0x0c, + 0xd3, 0x8a, 0x7d, 0x60, 0xa8, 0x54, 0x3d, 0xd8, 0xb0, 0xbb, 0x36, 0x85, 0x14, 0x19, 0xa9, 0x48, + 0xc9, 0x15, 0x97, 0x8a, 0x5e, 0x85, 0x33, 0x1a, 0x26, 0x31, 0x12, 0x0b, 0x44, 0x1c, 0xd3, 0x94, + 0xcd, 0x96, 0x26, 0xd3, 0xeb, 0x1f, 0x28, 0x9c, 0xc8, 0x48, 0x27, 0x69, 0x01, 0x1a, 0x82, 0x34, + 0x4c, 0xf3, 0x5e, 0x4b, 0xab, 0x92, 0x97, 0xec, 0xaa, 0xcf, 0x3b, 0xa9, 0x4c, 0x5e, 0x2c, 0x94, + 0x7f, 0x45, 0x80, 0x7c, 0xe8, 0x8c, 0xc2, 0xeb, 0x3d, 0x5b, 0xb4, 0x67, 0xa2, 0xe1, 0xd0, 0xa0, + 0x1c, 0xaf, 0x0c, 0x57, 0xbf, 0x9b, 0xe1, 0x36, 0x3f, 0x38, 0x9c, 0xa6, 0x8b, 0x03, 0x6e, 0x7a, + 0x80, 0xcb, 0xf6, 0x5a, 0xea, 0xd7, 0xde, 0x9f, 0x9f, 0x28, 0xff, 0x3c, 0x09, 0x85, 0xf0, 0x59, + 0x84, 0xd5, 0x9e, 0x7a, 0x45, 0xb9, 0xab, 0x10, 0xc7, 0xe2, 0x90, 0xfb, 0xcf, 0xb2, 0xfe, 0x7d, + 0xdc, 0xac, 0x9a, 0x0b, 0x43, 0x36, 0xa2, 0x83, 0xf5, 0xf4, 0x19, 0xe7, 0x7e, 0x90, 0xf0, 0x86, + 0xfd, 0x22, 0x4c, 0x32, 0xb5, 0x0b, 0x03, 0xcf, 0x86, 0x52, 0xcd, 0x4b, 0xac, 0x18, 0x71, 0x13, + 0x8d, 0x63, 0x5d, 0x72, 0xe5, 0x5f, 0x0f, 0x70, 0xf4, 0x5f, 0x9d, 0xe2, 0x57, 0x9d, 0x4d, 0x1e, + 0xed, 0xaa, 0x33, 0xb6, 0xd1, 0xdc, 0x6a, 0x31, 0x17, 0xcc, 0x06, 0x4a, 0xba, 0xef, 0x00, 0x26, + 0x15, 0xc1, 0x7f, 0x94, 0x6c, 0x51, 0xe2, 0x3f, 0x4a, 0x16, 0x48, 0x3a, 0x2c, 0x7a, 0x22, 0xe8, + 0xa8, 0x62, 0xa9, 0xa9, 0xbc, 0xab, 0xbf, 0x23, 0x80, 0x48, 0xc7, 0xd0, 0x2d, 0x8c, 0xb5, 0x58, + 0xac, 0xd0, 0xcd, 0x87, 0x4c, 0x8c, 0x9f, 0x26, 0x1e, 0xba, 0x1f, 0x3d, 0x19, 0xbe, 0x1f, 0xbd, + 0xfc, 0xbe, 0x00, 0x45, 0xaf, 0x86, 0xec, 0x27, 0x78, 0x86, 0xdc, 0x60, 0x76, 0xbc, 0x5f, 0xa5, + 0x71, 0x0f, 0x57, 0x8f, 0xf5, 0x5b, 0x40, 0xc1, 0xc3, 0xd5, 0xec, 0x17, 0x54, 0xbe, 0x2d, 0xc0, + 0xac, 0x57, 0xc5, 0x8a, 0x7f, 0x70, 0xf6, 0x18, 0x19, 0xf3, 0x12, 0xfd, 0x75, 0x32, 0x02, 0xda, + 0xe9, 0xa9, 0xf6, 0xb1, 0xcc, 0x13, 0xf1, 0x84, 0x0a, 0xe0, 0x8b, 0x01, 0x5a, 0xa3, 0x4e, 0x7f, + 0xb7, 0x8c, 0xfd, 0x6f, 0x97, 0x6f, 0x05, 0x14, 0x48, 0x47, 0x02, 0xd1, 0xd2, 0x58, 0x43, 0xc6, + 0xd5, 0x12, 0x2d, 0x5c, 0xfe, 0x51, 0xb0, 0x27, 0xaa, 0xbb, 0x24, 0x08, 0xbc, 0x09, 0xc9, 0x5d, + 0xa5, 0x35, 0x2c, 0x9f, 0x24, 0xd4, 0x73, 0x12, 0x29, 0x8d, 0x6e, 0x85, 0xce, 0x1b, 0x27, 0x06, + 0x07, 0x2c, 0xfd, 0x2a, 0x0d, 0x9d, 0x4b, 0x7e, 0xc5, 0x6d, 0x45, 0x72, 0xf4, 0xe7, 0x83, 0x1e, + 0xe0, 0xb5, 0xd4, 0x07, 0xef, 0xcf, 0x0b, 0x57, 0xeb, 0x30, 0x1b, 0x31, 0xbd, 0xa1, 0x22, 0x40, + 0xe0, 0xd6, 0x74, 0xfe, 0xe3, 0x68, 0x4b, 0x2b, 0xf2, 0xd6, 0x46, 0xe5, 0xde, 0xfa, 0xfa, 0x6a, + 0xa3, 0x51, 0x5d, 0x11, 0x05, 0x24, 0x42, 0x3e, 0x74, 0xe7, 0x3a, 0xff, 0x4d, 0xb4, 0xab, 0xff, + 0x0d, 0xc0, 0xff, 0xdd, 0x05, 0x22, 0x6b, 0xad, 0xfa, 0x40, 0xbe, 0xbf, 0x74, 0x77, 0xab, 0x5a, + 0x17, 0x27, 0x10, 0x82, 0xe2, 0xf2, 0x52, 0xa3, 0x52, 0x93, 0xa5, 0x6a, 0x7d, 0xf3, 0xde, 0x46, + 0xbd, 0x2a, 0x0a, 0x9c, 0x6f, 0x05, 0xf2, 0xc1, 0x93, 0xd9, 0x68, 0x16, 0xa6, 0x2b, 0xb5, 0x6a, + 0x65, 0x4d, 0xbe, 0xbf, 0xba, 0x24, 0xbf, 0xb5, 0x55, 0xdd, 0xaa, 0x8a, 0x13, 0xb4, 0x6a, 0x94, + 0x78, 0x6b, 0xeb, 0xee, 0x5d, 0x51, 0x40, 0xd3, 0x90, 0x63, 0xcf, 0xf4, 0x7e, 0x76, 0x31, 0x71, + 0x75, 0x1d, 0x72, 0x81, 0x8b, 0xdc, 0xc8, 0xe7, 0x36, 0xb7, 0xea, 0x35, 0xb9, 0xb1, 0xba, 0x5e, + 0xad, 0x37, 0x96, 0xd6, 0x37, 0x99, 0x0c, 0x4a, 0x5b, 0x5a, 0xbe, 0x27, 0x35, 0x44, 0xc1, 0x7b, + 0x6e, 0xdc, 0xdb, 0xaa, 0xd4, 0xbc, 0x9f, 0x76, 0x4b, 0x65, 0x92, 0x62, 0xf2, 0xea, 0x63, 0x38, + 0x3d, 0xe0, 0x78, 0x32, 0xca, 0xc1, 0xd4, 0x96, 0x41, 0xaf, 0xae, 0x12, 0x27, 0x50, 0x21, 0x70, + 0x42, 0x59, 0x14, 0x50, 0x86, 0x9d, 0x3d, 0x15, 0x13, 0x28, 0x0d, 0x89, 0xfa, 0x4d, 0x31, 0x49, + 0x2a, 0x1a, 0x38, 0xe0, 0x2b, 0xa6, 0x50, 0x96, 0x9f, 0x7e, 0x14, 0x27, 0x51, 0xde, 0x3f, 0x7e, + 0x28, 0xa6, 0xaf, 0x9e, 0x83, 0xc0, 0xd9, 0x2c, 0x04, 0x90, 0xbe, 0xab, 0x38, 0xd8, 0x76, 0xc4, + 0x09, 0x34, 0x05, 0xc9, 0xa5, 0x56, 0x4b, 0x14, 0x6e, 0xfc, 0xbe, 0x00, 0x19, 0xf7, 0xaa, 0x71, + 0x74, 0x17, 0x26, 0x19, 0x92, 0x9f, 0x1f, 0x3c, 0x23, 0x51, 0xa7, 0x36, 0xb7, 0x30, 0x6a, 0xca, + 0x2a, 0x4f, 0xa0, 0xb7, 0xf9, 0xef, 0x34, 0x12, 0x8b, 0x41, 0xe7, 0x87, 0xd9, 0x93, 0x2b, 0x75, + 0xb8, 0xd1, 0x91, 0x31, 0x52, 0x9e, 0x78, 0x49, 0x58, 0x7e, 0xfe, 0x83, 0x9f, 0x9e, 0x9d, 0xf8, + 0xe0, 0xe9, 0x59, 0xe1, 0xc3, 0xa7, 0x67, 0x85, 0x9f, 0x3c, 0x3d, 0x2b, 0xfc, 0xfd, 0xd3, 0xb3, + 0xc2, 0x2f, 0xfd, 0xec, 0xec, 0xc4, 0x87, 0x3f, 0x3b, 0x3b, 0xf1, 0x93, 0x9f, 0x9d, 0x9d, 0x78, + 0x67, 0x8a, 0x73, 0x3f, 0x4c, 0xd3, 0x9f, 0x8c, 0xbc, 0xf9, 0x6f, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x5e, 0xd6, 0x84, 0x9e, 0x37, 0x73, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 35fa87c95adc..ed95c4303440 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -348,6 +348,8 @@ message ScanRequest { // will set the batch_responses field in the ScanResponse instead of the rows // field. ScanFormat scan_format = 4; + + bool select_for_update = 5; } // A ScanResponse is the return value from the Scan() method. diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index e7549f4a947d..ff36901c1629 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -99,6 +99,17 @@ func (ba *BatchRequest) IsReadOnly() bool { return len(ba.Requests) > 0 && !ba.hasFlag(isWrite|isAdmin) } +func (ba *BatchRequest) HasSelectForUpdate() bool { + for _, union := range ba.Requests { + if scan, ok := union.GetInner().(*ScanRequest); ok { + if scan.SelectForUpdate { + return true + } + } + } + return false +} + // RequiresLeaseHolder returns true if the request can only be served by the // leaseholders of the ranges it addresses. func (ba *BatchRequest) RequiresLeaseHolder() bool { @@ -359,7 +370,9 @@ func (ba *BatchRequest) IntentSpanIterate(br *BatchResponse, fn func(Span)) { for i, arg := range ba.Requests { req := arg.GetInner() if !IsTransactionWrite(req) { - continue + if scan, ok := req.(*ScanRequest); !ok || !scan.SelectForUpdate { + continue + } } var resp Response if br != nil { diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index adad6712b545..0e7380d384d5 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -413,6 +413,10 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio return dsp.checkSupportForNode(n.source.plan) case *scanNode: + if n.lockingStrength != sqlbase.ScanLockingStrength_FOR_NONE { + return cannotDistribute, newQueryNotSupportedErrorf("can't distribute locking scans") + } + rec := canDistribute if n.softLimit != 0 { // We don't yet recommend distributing plans where soft limits propagate diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index 42a9e1de7bfd..01725c9aee2a 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -246,7 +246,9 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { for i := range f.spans { scans[i].ScanFormat = roachpb.BATCH_RESPONSE scans[i].SetSpan(f.spans[i]) - // TODO(nvanbenschoten): use f.lockStr here. + if f.lockStr != sqlbase.ScanLockingStrength_FOR_NONE { + scans[i].SelectForUpdate = true + } ba.Requests[i].MustSetInner(&scans[i]) } } diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 4e9be72c4efa..85af1ac6e3bb 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -494,8 +494,8 @@ func resolveLocalIntents( ok, err := engine.MVCCResolveWriteIntentUsingIter(ctx, readWriter, iterAndBuf, resolveMS, intent) if ok { resolveAllowance-- - resolvedIntents = append(resolvedIntents, intent) } + resolvedIntents = append(resolvedIntents, intent) return err } // For intent ranges, cut into parts inside and outside our key diff --git a/pkg/storage/batcheval/cmd_scan.go b/pkg/storage/batcheval/cmd_scan.go index 3beb1b018b0c..948d55fd9451 100644 --- a/pkg/storage/batcheval/cmd_scan.go +++ b/pkg/storage/batcheval/cmd_scan.go @@ -14,13 +14,33 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/batcheval/result" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/storage/spanset" ) func init() { - RegisterReadOnlyCommand(roachpb.Scan, DefaultDeclareKeys, Scan) + RegisterReadOnlyCommand(roachpb.Scan, scanDeclareKeys, Scan) +} + +func scanDeclareKeys( + desc *roachpb.RangeDescriptor, header roachpb.Header, req roachpb.Request, spans *spanset.SpanSet, +) { + scan := req.(*roachpb.ScanRequest) + var access spanset.SpanAccess + if scan.SelectForUpdate { + access = spanset.SpanReadWrite + } else { + access = spanset.SpanReadOnly + } + + if keys.IsLocal(scan.Key) { + spans.AddNonMVCC(access, scan.Span()) + } else { + spans.AddMVCC(access, scan.Span(), header.Timestamp) + } } // Scan scans the key range specified by start key through end key @@ -34,9 +54,17 @@ func Scan( h := cArgs.Header reply := resp.(*roachpb.ScanResponse) + var writeTooOldOnWriteInFuture bool + if args.SelectForUpdate { + writeTooOldOnWriteInFuture = true + } else { + writeTooOldOnWriteInFuture = false + } + var err error var intents []roachpb.Intent var resumeSpan *roachpb.Span + var res result.Result switch args.ScanFormat { case roachpb.BATCH_RESPONSE: @@ -45,26 +73,48 @@ func Scan( kvData, numKvs, resumeSpan, intents, err = engine.MVCCScanToBytes( ctx, reader, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - Txn: h.Txn, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + WriteTooOldOnWriteInFuture: writeTooOldOnWriteInFuture, }) if err != nil { return result.Result{}, err } reply.NumKeys = numKvs reply.BatchResponses = kvData + if args.SelectForUpdate && h.Txn != nil { + res.Local.UpdatedIntents = make([]roachpb.Intent, numKvs) + var i int + if err := engine.MVCCScanDecodeKeyValues(kvData, func(key engine.MVCCKey, _ []byte) error { + res.Local.UpdatedIntents[i] = roachpb.Intent{ + Span: roachpb.Span{Key: key.Key}, Txn: h.Txn.TxnMeta, Status: roachpb.PENDING, + } + return nil + }); err != nil { + return result.Result{}, err + } + } case roachpb.KEY_VALUES: var rows []roachpb.KeyValue rows, resumeSpan, intents, err = engine.MVCCScan( ctx, reader, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - Txn: h.Txn, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + WriteTooOldOnWriteInFuture: writeTooOldOnWriteInFuture, }) if err != nil { return result.Result{}, err } reply.NumKeys = int64(len(rows)) reply.Rows = rows + if args.SelectForUpdate && h.Txn != nil { + res.Local.UpdatedIntents = make([]roachpb.Intent, len(rows)) + for i, row := range rows { + res.Local.UpdatedIntents[i] = roachpb.Intent{ + Span: roachpb.Span{Key: row.Key}, Txn: h.Txn.TxnMeta, Status: roachpb.PENDING, + } + } + } default: panic(fmt.Sprintf("Unknown scanFormat %d", args.ScanFormat)) } @@ -77,5 +127,6 @@ func Scan( if h.ReadConsistency == roachpb.READ_UNCOMMITTED { reply.IntentRows, err = CollectIntentRows(ctx, reader, cArgs, intents) } - return result.FromEncounteredIntents(intents), err + res.Local.EncounteredIntents = intents + return res, err } diff --git a/pkg/storage/concurrency/lock_table.go b/pkg/storage/concurrency/lock_table.go index d9fc5069e8a9..1617e80155ed 100644 --- a/pkg/storage/concurrency/lock_table.go +++ b/pkg/storage/concurrency/lock_table.go @@ -81,25 +81,38 @@ func (lt *lockTableImpl) AcquireLock(in roachpb.Intent) { func (lt *lockTableImpl) ReleaseLock(in roachpb.Intent) { lt.mu.Lock() defer lt.mu.Unlock() - lt.tmp1.in = in - wqI := lt.qs.Get(<.tmp1) - if wqI == nil { - return - } - wq := wqI.(*perKeyWaitQueue) - if !wq.held || wq.in.Txn.ID != in.Txn.ID { - return + + var toDelete []*perKeyWaitQueue + maybeRelease := func(i btree.Item) bool { + wq := i.(*perKeyWaitQueue) + if !wq.held || wq.in.Txn.ID != in.Txn.ID { + return true + } + if wq.ll.Len() == 0 { + toDelete = append(toDelete, wq) + return true + } + wq.in.Txn = enginepb.TxnMeta{} + wq.held = false + front := wq.ll.Front().Value.(*perKeyWaitQueueElem) + if !front.closed { + close(front.done) + front.closed = true + } + return true } - if wq.ll.Len() == 0 { - lt.qs.Delete(wq) - return + + lt.tmp1.in.Key = in.Key + if in.EndKey == nil { + if i := lt.qs.Get(<.tmp1); i != nil { + maybeRelease(i) + } + } else { + lt.tmp2.in.Key = in.EndKey + lt.qs.AscendRange(<.tmp1, <.tmp2, maybeRelease) } - wq.in.Txn = enginepb.TxnMeta{} - wq.held = false - front := wq.ll.Front().Value.(*perKeyWaitQueueElem) - if !front.closed { - close(front.done) - front.closed = true + for _, d := range toDelete { + lt.qs.Delete(d) } } diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index f419cd7cda3c..c7b9a51dd1e5 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -2356,20 +2356,15 @@ func mvccScanToKvs( return nil, nil, nil, err } kvs := make([]roachpb.KeyValue, numKVs) - var k MVCCKey - var rawBytes []byte var i int - for _, data := range kvData { - for len(data) > 0 { - k, rawBytes, data, err = MVCCScanDecodeKeyValue(data) - if err != nil { - return nil, nil, nil, err - } - kvs[i].Key = k.Key - kvs[i].Value.RawBytes = rawBytes - kvs[i].Value.Timestamp = k.Timestamp - i++ - } + if err := MVCCScanDecodeKeyValues(kvData, func(key MVCCKey, rawBytes []byte) error { + kvs[i].Key = key.Key + kvs[i].Value.RawBytes = rawBytes + kvs[i].Value.Timestamp = key.Timestamp + i++ + return nil + }); err != nil { + return nil, nil, nil, err } return kvs, resumeSpan, intents, err } @@ -2411,10 +2406,11 @@ type MVCCScanOptions struct { // to return no results. // See the documentation for MVCCScan for information on these parameters. - Inconsistent bool - Tombstones bool - Reverse bool - Txn *roachpb.Transaction + Inconsistent bool + Tombstones bool + Reverse bool + Txn *roachpb.Transaction + WriteTooOldOnWriteInFuture bool } // MVCCScan scans the key range [key, endKey) in the provided reader up to some diff --git a/pkg/storage/engine/rocksdb.go b/pkg/storage/engine/rocksdb.go index 87266e0f1093..8200f41d2a2b 100644 --- a/pkg/storage/engine/rocksdb.go +++ b/pkg/storage/engine/rocksdb.go @@ -2484,6 +2484,7 @@ func (r *rocksDBIterator) MVCCScan( goToCTimestamp(timestamp), C.int64_t(max), goToCTxn(opts.Txn), C.bool(opts.Inconsistent), C.bool(opts.Reverse), C.bool(opts.Tombstones), + C.bool(opts.WriteTooOldOnWriteInFuture), ) if err := statusToError(state.status); err != nil { @@ -2514,6 +2515,9 @@ func (r *rocksDBIterator) MVCCScan( return nil, 0, resumeSpan, nil, &roachpb.WriteIntentError{Intents: intents} } + if err := writeTooOldToError(timestamp, state.write_too_old, opts.Txn); err != nil { + return nil, 0, nil, nil, err + } return kvData, numKVs, resumeSpan, intents, nil } @@ -2759,6 +2763,23 @@ func uncertaintyToError( return nil } +func writeTooOldToError( + readTS hlc.Timestamp, wtoTS C.DBTimestamp, txn *roachpb.Transaction, +) error { + if wtoTS.wall_time != 0 || wtoTS.logical != 0 { + wtoTimestamp := hlc.Timestamp{ + WallTime: int64(wtoTS.wall_time), + Logical: int32(wtoTS.logical), + } + writeTimestamp := txn.WriteTimestamp + writeTimestamp.Forward(wtoTimestamp.Next()) + return &roachpb.WriteTooOldError{ + Timestamp: readTS, ActualTimestamp: writeTimestamp, + } + } + return nil +} + // goMerge takes existing and update byte slices that are expected to // be marshaled roachpb.Values and merges the two values returning a // marshaled roachpb.Value or an error. @@ -3295,6 +3316,28 @@ func MVCCScanDecodeKeyValue(repr []byte) (key MVCCKey, value []byte, orepr []byt return MVCCKey{k, ts}, value, orepr, err } +// MVCCScanDecodeKeyValues decodes all key/value pairs returned in one or more +// MVCCScan "batches" (this is not the RocksDB batch repr format). The provided +// function is called for each key/value pair. +func MVCCScanDecodeKeyValues(repr [][]byte, fn func(key MVCCKey, rawBytes []byte) error) error { + var k MVCCKey + var rawBytes []byte + var err error + for _, data := range repr { + for len(data) > 0 { + k, rawBytes, data, err = MVCCScanDecodeKeyValue(data) + if err != nil { + return err + } + err = fn(k, rawBytes) + if err != nil { + return err + } + } + } + return nil +} + func notFoundErrOrDefault(err error) error { errStr := err.Error() if strings.Contains(errStr, "No such") || diff --git a/pkg/workload/tpcc/new_order.go b/pkg/workload/tpcc/new_order.go index c120f9ecee9c..9c147cf17771 100644 --- a/pkg/workload/tpcc/new_order.go +++ b/pkg/workload/tpcc/new_order.go @@ -213,15 +213,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { err = crdb.ExecuteInTx( ctx, (*workload.PgxTx)(tx), func() error { - // Select the district tax rate and next available order number, bumping it. - var dNextOID int - if err := n.updateDistrict.QueryRowTx( - ctx, tx, d.wID, d.dID, - ).Scan(&d.dTax, &dNextOID); err != nil { - return err - } - d.oID = dNextOID - 1 - // Select the warehouse tax rate. if err := n.selectWarehouseTax.QueryRowTx( ctx, tx, wID, @@ -362,19 +353,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { } rows.Close() - // Insert row into the orders and new orders table. - if _, err := n.insertOrder.ExecTx( - ctx, tx, - d.oID, d.dID, d.wID, d.cID, d.oEntryD.Format("2006-01-02 15:04:05"), d.oOlCnt, allLocal, - ); err != nil { - return err - } - if _, err := n.insertNewOrder.ExecTx( - ctx, tx, d.oID, d.dID, d.wID, - ); err != nil { - return err - } - // Update the stock table for each item. if _, err := tx.ExecEx( ctx, @@ -397,6 +375,28 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { return err } + // Select the district tax rate and next available order number, bumping it. + var dNextOID int + if err := n.updateDistrict.QueryRowTx( + ctx, tx, d.wID, d.dID, + ).Scan(&d.dTax, &dNextOID); err != nil { + return err + } + d.oID = dNextOID - 1 + + // Insert row into the orders and new orders table. + if _, err := n.insertOrder.ExecTx( + ctx, tx, + d.oID, d.dID, d.wID, d.cID, d.oEntryD.Format("2006-01-02 15:04:05"), d.oOlCnt, allLocal, + ); err != nil { + return err + } + if _, err := n.insertNewOrder.ExecTx( + ctx, tx, d.oID, d.dID, d.wID, + ); err != nil { + return err + } + // Insert a new order line for each item in the order. olValsStrings := make([]string, d.oOlCnt) for i := range d.items { diff --git a/pkg/workload/ycsb/ycsb.go b/pkg/workload/ycsb/ycsb.go index f02142c370ff..e5dad0c0eea3 100644 --- a/pkg/workload/ycsb/ycsb.go +++ b/pkg/workload/ycsb/ycsb.go @@ -39,6 +39,7 @@ const ( usertableSchemaRelational = `( ycsb_key VARCHAR(255) PRIMARY KEY NOT NULL, + OTHER BOOL, FIELD0 TEXT, FIELD1 TEXT, FIELD2 TEXT, @@ -52,6 +53,7 @@ const ( )` usertableSchemaRelationalWithFamilies = `( ycsb_key VARCHAR(255) PRIMARY KEY NOT NULL, + OTHER BOOL, FIELD0 TEXT, FIELD1 TEXT, FIELD2 TEXT, @@ -63,6 +65,7 @@ const ( FIELD8 TEXT, FIELD9 TEXT, FAMILY (ycsb_key), + FAMILY (OTHER), FAMILY (FIELD0), FAMILY (FIELD1), FAMILY (FIELD2), @@ -91,6 +94,7 @@ type ycsb struct { json bool families bool splits int + singleRow bool workload string requestDistribution string @@ -121,6 +125,7 @@ var ycsbMeta = workload.Meta{ g.flags.BoolVar(&g.json, `json`, false, `Use JSONB rather than relational data`) g.flags.BoolVar(&g.families, `families`, true, `Place each column in its own column family`) g.flags.IntVar(&g.splits, `splits`, 0, `Number of splits to perform before starting normal operations`) + g.flags.BoolVar(&g.singleRow, `single-row`, false, `Read and write to a single row`) g.flags.StringVar(&g.workload, `workload`, `B`, `Workload type. Choose from A-F.`) g.flags.StringVar(&g.requestDistribution, `request-distribution`, ``, `Distribution for request key generation [zipfian, uniform, latest]. The default for workloads A, B, C, E, and F is zipfian, and the default for workload D is latest.`) g.flags.StringVar(&g.scanLengthDistribution, `scan-length-distribution`, `uniform`, `Distribution for scan length generation [zipfian, uniform]. Primarily used for workload E.`) @@ -170,6 +175,9 @@ func (g *ycsb) Hooks() workload.Hooks { g.readFreq = 0.5 g.readModifyWriteFreq = 0.5 g.requestDistribution = "zipfian" + case "U", "u": + g.updateFreq = 1.0 + g.requestDistribution = "zipfian" default: return errors.Errorf("Unknown workload: %q", g.workload) } @@ -210,7 +218,10 @@ func (g *ycsb) Tables() []workload.Table { if g.json { return []interface{}{key, "{}"} } - return []interface{}{key, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil} + if g.singleRow && rowIdx == 0 { + key = keyNameFromHash(0) + } + return []interface{}{key, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil} } if g.json { usertable.Schema = usertableSchemaJSON @@ -283,7 +294,7 @@ func (g *ycsb) Ops(urls []string, reg *histogram.Registry) (workload.QueryLoad, ))`) } else { insertStmt, err = db.Prepare(`INSERT INTO usertable VALUES ( - $1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11 + $1, FALSE, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11 )`) } if err != nil { @@ -484,6 +495,9 @@ func (yw *ycsbWorker) nextReadKey() string { // distribution, so it might be worthwhile to exactly emulate what they're // doing. rowIndex := yw.requestGen.Uint64() % rowCount + if yw.config.singleRow { + return keyNameFromHash(0) + } return yw.buildKeyName(rowIndex) }