Add segment reference lock for bulk load (#17457)

So we can support DataQueryable and DataIndexed states

issue: #16607
Signed-off-by: Yuchen Gao <yuchen.gao@zilliz.com>
pull/17558/head
Ten Thousand Leaves 2022-06-15 12:20:10 +08:00 committed by GitHub
parent e751663fdc
commit 65ab847815
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 1258 additions and 936 deletions

View File

@ -105,7 +105,7 @@ print-build-info:
milvus: build-cpp print-build-info
@echo "Building Milvus ..."
@echo "if build fails on Mac M1 machines, rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@echo "if build fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="1" && GO111MODULE=on $(GO) build \
-ldflags="-X '$(OBJPREFIX).BuildTags=$(BUILD_TAGS)' -X '$(OBJPREFIX).BuildTime=$(BUILD_TIME)' -X '$(OBJPREFIX).GitCommit=$(GIT_COMMIT)' -X '$(OBJPREFIX).GoVersion=$(GO_VERSION)'" \
${APPLE_SILICON_FLAG} -o $(INSTALL_PATH)/milvus $(PWD)/cmd/main.go 1>/dev/null
@ -146,6 +146,7 @@ build-cpp-with-coverage: pre-proc
# Run the tests.
unittest: test-cpp test-go
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
test-indexnode:
@echo "Running go unittests..."
@ -174,23 +175,28 @@ test-querycoord:
test-go: build-cpp-with-unittest
@echo "Running go unittests..."
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@(env bash $(PWD)/scripts/run_go_unittest.sh)
test-cpp: build-cpp-with-unittest
@echo "Running cpp unittests..."
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@(env bash $(PWD)/scripts/run_cpp_unittest.sh)
# Run code coverage.
codecov: codecov-go codecov-cpp
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
# Run codecov-go
codecov-go: build-cpp-with-coverage
@echo "Running go coverage..."
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@(env bash $(PWD)/scripts/run_go_codecov.sh)
# Run codecov-cpp
codecov-cpp: build-cpp-with-coverage
@echo "Running cpp coverage..."
@echo "if test fails on Mac M1 machines, you probably need to rerun scripts/install_deps.sh and then run: \`export PKG_CONFIG_PATH=\"/opt/homebrew/opt/openssl@3/lib/pkgconfig\"\`"
@(env bash $(PWD)/scripts/run_cpp_codecov.sh)
# Package docker image locally.

View File

@ -100,9 +100,9 @@ rootCoord:
maxPartitionNum: 4096 # Maximum number of partitions in a collection
minSegmentSizeToEnableIndex: 1024 # It's a threshold. When the segment size is less than this value, the segment will not be indexed
# (in seconds) Duration after which an import task will expire (be killed). Default 3600 seconds (1 hour).
# (in seconds) Duration after which an import task will expire (be killed). Default 900 seconds (15 minutes).
# Note: If default value is to be changed, change also the default in: internal/util/paramtable/component_param.go
importTaskExpiration: 3600
importTaskExpiration: 900
# (in seconds) Milvus will keep the record of import tasks for at least `importTaskRetention` seconds. Default 86400
# seconds (24 hours).
# Note: If default value is to be changed, change also the default in: internal/util/paramtable/component_param.go
@ -116,13 +116,13 @@ rootCoord:
# Note: If default value is to be changed, change also the default in: internal/util/paramtable/component_param.go
importSegmentStateWaitLimit: 60
# (in seconds) Check the building status of a task's segments' indices every `importIndexCheckInterval` seconds.
# Default 300 seconds (5 minutes).
# Default 10 seconds.
# Note: If default value is to be changed, change also the default in: internal/util/paramtable/component_param.go
importIndexCheckInterval: 300
importIndexCheckInterval: 10
# (in seconds) Maximum time to wait for indices to be built on a single import task's segments.
# Default 1200 seconds (20 minutes).
# Default 600 seconds (10 minutes).
# Note: If default value is to be changed, change also the default in: internal/util/paramtable/component_param.go
importIndexWaitLimit: 1200
importIndexWaitLimit: 600
# Related configuration of proxy, used to validate client requests and reduce the returned results.
proxy:

View File

@ -439,22 +439,23 @@ const char descriptor_table_protodef_common_2eproto[] PROTOBUF_SECTION_VARIABLE(
"\n\rUndefiedState\020\000\022\r\n\tExecuting\020\001\022\r\n\tComp"
"leted\020\002*X\n\020ConsistencyLevel\022\n\n\006Strong\020\000\022"
"\013\n\007Session\020\001\022\013\n\007Bounded\020\002\022\016\n\nEventually\020"
"\003\022\016\n\nCustomized\020\004*\227\001\n\013ImportState\022\021\n\rImp"
"\003\022\016\n\nCustomized\020\004*\257\001\n\013ImportState\022\021\n\rImp"
"ortPending\020\000\022\020\n\014ImportFailed\020\001\022\021\n\rImport"
"Started\020\002\022\024\n\020ImportDownloaded\020\003\022\020\n\014Impor"
"tParsed\020\004\022\023\n\017ImportPersisted\020\005\022\023\n\017Import"
"Completed\020\006*\036\n\014ResourceType\022\016\n\nCollectio"
"n\020\000*\335\001\n\021ResourcePrivilege\022\020\n\014PrivilegeAl"
"l\020\000\022\023\n\017PrivilegeCreate\020\001\022\021\n\rPrivilegeDro"
"p\020\002\022\022\n\016PrivilegeAlter\020\003\022\021\n\rPrivilegeRead"
"\020\004\022\021\n\rPrivilegeLoad\020\005\022\024\n\020PrivilegeReleas"
"e\020\006\022\024\n\020PrivilegeCompact\020\007\022\023\n\017PrivilegeIn"
"sert\020\010\022\023\n\017PrivilegeDelete\020\t:^\n\021privilege"
"_ext_obj\022\037.google.protobuf.MessageOption"
"s\030\351\007 \001(\0132!.milvus.proto.common.Privilege"
"ExtBW\n\016io.milvus.grpcB\013CommonProtoP\001Z3gi"
"thub.com/milvus-io/milvus/internal/proto"
"/commonpb\240\001\001b\006proto3"
"Completed\020\006\022\026\n\022ImportAllocSegment\020\n*\036\n\014R"
"esourceType\022\016\n\nCollection\020\000*\335\001\n\021Resource"
"Privilege\022\020\n\014PrivilegeAll\020\000\022\023\n\017Privilege"
"Create\020\001\022\021\n\rPrivilegeDrop\020\002\022\022\n\016Privilege"
"Alter\020\003\022\021\n\rPrivilegeRead\020\004\022\021\n\rPrivilegeL"
"oad\020\005\022\024\n\020PrivilegeRelease\020\006\022\024\n\020Privilege"
"Compact\020\007\022\023\n\017PrivilegeInsert\020\010\022\023\n\017Privil"
"egeDelete\020\t:^\n\021privilege_ext_obj\022\037.googl"
"e.protobuf.MessageOptions\030\351\007 \001(\0132!.milvu"
"s.proto.common.PrivilegeExtBW\n\016io.milvus"
".grpcB\013CommonProtoP\001Z3github.com/milvus-"
"io/milvus/internal/proto/commonpb\240\001\001b\006pr"
"oto3"
;
static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_common_2eproto_deps[1] = {
&::descriptor_table_google_2fprotobuf_2fdescriptor_2eproto,
@ -475,7 +476,7 @@ static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_com
static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_common_2eproto_once;
static bool descriptor_table_common_2eproto_initialized = false;
const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_common_2eproto = {
&descriptor_table_common_2eproto_initialized, descriptor_table_protodef_common_2eproto, "common.proto", 4660,
&descriptor_table_common_2eproto_initialized, descriptor_table_protodef_common_2eproto, "common.proto", 4684,
&descriptor_table_common_2eproto_once, descriptor_table_common_2eproto_sccs, descriptor_table_common_2eproto_deps, 11, 1,
schemas, file_default_instances, TableStruct_common_2eproto::offsets,
file_level_metadata_common_2eproto, 11, file_level_enum_descriptors_common_2eproto, file_level_service_descriptors_common_2eproto,
@ -745,6 +746,7 @@ bool ImportState_IsValid(int value) {
case 4:
case 5:
case 6:
case 10:
return true;
default:
return false;

View File

@ -454,12 +454,13 @@ enum ImportState : int {
ImportParsed = 4,
ImportPersisted = 5,
ImportCompleted = 6,
ImportAllocSegment = 10,
ImportState_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::min(),
ImportState_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::max()
};
bool ImportState_IsValid(int value);
constexpr ImportState ImportState_MIN = ImportPending;
constexpr ImportState ImportState_MAX = ImportCompleted;
constexpr ImportState ImportState_MAX = ImportAllocSegment;
constexpr int ImportState_ARRAYSIZE = ImportState_MAX + 1;
const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* ImportState_descriptor();

View File

@ -126,9 +126,6 @@ func (srm *SegmentReferenceManager) ReleaseSegmentsLock(segIDs []UniqueID, nodeI
log.Info("Release reference lock on segments", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID))
locKeys := make([]string, 0)
for _, segID := range segIDs {
if _, ok := srm.segmentsLock[segID]; !ok {
continue
}
for _, segLock := range srm.segmentsLock[segID] {
if segLock.nodeID == nodeID {
locKeys = append(locKeys, segLock.locKey)

View File

@ -131,6 +131,7 @@ type Server struct {
dnEventCh <-chan *sessionutil.SessionEvent
icEventCh <-chan *sessionutil.SessionEvent
qcEventCh <-chan *sessionutil.SessionEvent
rcEventCh <-chan *sessionutil.SessionEvent
dataNodeCreator dataNodeCreatorFunc
rootCoordClientCreator rootCoordCreatorFunc
@ -451,6 +452,16 @@ func (s *Server) initServiceDiscovery() error {
}
s.qcEventCh = s.session.WatchServices(typeutil.QueryCoordRole, qcRevision+1, nil)
rcSessions, rcRevision, err := s.session.GetSessions(typeutil.RootCoordRole)
if err != nil {
log.Error("DataCoord get RootCoord session failed", zap.Error(err))
return err
}
for _, session := range rcSessions {
serverIDs = append(serverIDs, session.ServerID)
}
s.rcEventCh = s.session.WatchServices(typeutil.RootCoordRole, rcRevision+1, nil)
s.segReferManager, err = NewSegmentReferenceManager(s.kvClient, serverIDs)
return err
}
@ -657,6 +668,36 @@ func (s *Server) startWatchService(ctx context.Context) {
go s.watchService(ctx)
}
func (s *Server) stopServiceWatch() {
// ErrCompacted is handled inside SessionWatcher, which means there is some other error occurred, closing server.
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
go s.Stop()
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
p.Signal(syscall.SIGINT)
}
}
}
func (s *Server) processSessionEvent(ctx context.Context, role string, event *sessionutil.SessionEvent) {
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("there is a new service online",
zap.String("server role", role),
zap.Int64("server ID", event.Session.ServerID))
case sessionutil.SessionDelEvent:
log.Warn("there is service offline",
zap.String("server role", role),
zap.Int64("server ID", event.Session.ServerID))
if err := retry.Do(ctx, func() error {
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
}, retry.Attempts(100)); err != nil {
panic(err)
}
}
}
// watchService watches services.
func (s *Server) watchService(ctx context.Context) {
defer logutil.LogPanic()
@ -668,75 +709,35 @@ func (s *Server) watchService(ctx context.Context) {
return
case event, ok := <-s.dnEventCh:
if !ok {
// ErrCompacted in handled inside SessionWatcher
// So there is some other error occurred, closing DataCoord server
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
go s.Stop()
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
p.Signal(syscall.SIGINT)
}
}
s.stopServiceWatch()
return
}
if err := s.handleSessionEvent(ctx, event); err != nil {
go func() {
if err := s.Stop(); err != nil {
log.Warn("datacoord server stop error", zap.Error(err))
log.Warn("DataCoord server stop error", zap.Error(err))
}
}()
return
}
case event, ok := <-s.icEventCh:
if !ok {
// ErrCompacted in handled inside SessionWatcher
// So there is some other error occurred, closing DataCoord server
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
go s.Stop()
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
p.Signal(syscall.SIGINT)
}
}
s.stopServiceWatch()
return
}
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("there is a new IndexCoord online", zap.Int64("serverID", event.Session.ServerID))
case sessionutil.SessionDelEvent:
log.Warn("there is IndexCoord offline", zap.Int64("serverID", event.Session.ServerID))
if err := retry.Do(ctx, func() error {
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
}, retry.Attempts(100)); err != nil {
panic(err)
}
}
s.processSessionEvent(ctx, "IndexCoord", event)
case event, ok := <-s.qcEventCh:
if !ok {
// ErrCompacted in handled inside SessionWatcher
// So there is some other error occurred, closing DataCoord server
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
go s.Stop()
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
p.Signal(syscall.SIGINT)
}
}
s.stopServiceWatch()
return
}
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("there is a new QueryCoord online", zap.Int64("serverID", event.Session.ServerID))
case sessionutil.SessionDelEvent:
log.Warn("there is QueryCoord offline", zap.Int64("serverID", event.Session.ServerID))
if err := retry.Do(ctx, func() error {
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
}, retry.Attempts(100)); err != nil {
panic(err)
}
s.processSessionEvent(ctx, "QueryCoord", event)
case event, ok := <-s.rcEventCh:
if !ok {
s.stopServiceWatch()
return
}
s.processSessionEvent(ctx, "RootCoord", event)
}
}
}

View File

@ -722,10 +722,12 @@ func TestServer_watchCoord(t *testing.T) {
dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh
svr.icEventCh = icCh
svr.qcEventCh = qcCh
svr.rcEventCh = rcCh
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
@ -783,10 +785,12 @@ func TestServer_watchQueryCoord(t *testing.T) {
dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh
svr.icEventCh = icCh
svr.qcEventCh = qcCh
svr.rcEventCh = rcCh
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
@ -828,6 +832,69 @@ func TestServer_watchQueryCoord(t *testing.T) {
assert.True(t, closed)
}
func TestServer_watchRootCoord(t *testing.T) {
Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
assert.NotNil(t, etcdKV)
factory := dependency.NewDefaultFactory(true)
svr := CreateServer(context.TODO(), factory)
svr.session = &sessionutil.Session{
TriggerKill: true,
}
svr.kvClient = etcdKV
dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh
svr.icEventCh = icCh
svr.qcEventCh = qcCh
svr.rcEventCh = rcCh
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
svr.segReferManager = segRefer
sc := make(chan os.Signal, 1)
signal.Notify(sc, syscall.SIGINT)
defer signal.Reset(syscall.SIGINT)
closed := false
sigQuit := make(chan struct{}, 1)
svr.serverLoopWg.Add(1)
go func() {
svr.watchService(context.Background())
}()
go func() {
<-sc
closed = true
sigQuit <- struct{}{}
}()
rcCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionAddEvent,
Session: &sessionutil.Session{
ServerID: 3,
},
}
rcCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionDelEvent,
Session: &sessionutil.Session{
ServerID: 3,
},
}
close(rcCh)
<-sigQuit
svr.serverLoopWg.Wait()
assert.True(t, closed)
}
func TestServer_GetMetrics(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)

View File

@ -1127,6 +1127,26 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root
fieldStats = append(fieldStats, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}})
}
// ReportImport with the new segment so RootCoord can add segment ref lock onto it.
// Fail-open.
status, err := node.rootCoord.ReportImport(context.Background(), &rootcoordpb.ImportResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
TaskId: req.GetImportTask().TaskId,
DatanodeId: Params.DataNodeCfg.GetNodeID(),
State: commonpb.ImportState_ImportAllocSegment,
Segments: []int64{segmentID},
})
if err != nil {
log.Error("failed to report import on new segment", zap.Error(err))
return err
}
if status.GetErrorCode() != commonpb.ErrorCode_Success {
log.Error("failed to report import on new segment", zap.String("reason", status.GetReason()))
return fmt.Errorf("failed to report import on new segment: %s", status.GetReason())
}
log.Info("now adding segment to the correct DataNode flow graph")
// Ask DataCoord to add segment to the corresponding DataNode flow graph.
node.dataCoord.AddSegment(context.Background(), &datapb.AddSegmentRequest{

View File

@ -940,9 +940,10 @@ func (m *RootCoordFactory) GetComponentStates(ctx context.Context) (*internalpb.
}
func (m *RootCoordFactory) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) (*commonpb.Status, error) {
v := ctx.Value(ctxKey{}).(string)
if v == returnError {
return nil, fmt.Errorf("injected error")
if ctx != nil && ctx.Value(ctxKey{}) != nil {
if v := ctx.Value(ctxKey{}).(string); v == returnError {
return nil, fmt.Errorf("injected error")
}
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,

View File

@ -229,6 +229,12 @@ func TestGrpcService(t *testing.T) {
core.CallImportService = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return nil
}
core.CallAddSegRefLock = func(context.Context, []int64) error {
return nil
}
core.CallReleaseSegRefLock = func(context.Context, []int64) error {
return nil
}
err = svr.start()
assert.Nil(t, err)

View File

@ -269,6 +269,7 @@ enum ImportState {
ImportParsed = 4;
ImportPersisted = 5;
ImportCompleted = 6;
ImportAllocSegment = 10;
}
enum ResourceType {

View File

@ -637,33 +637,36 @@ func (ConsistencyLevel) EnumDescriptor() ([]byte, []int) {
type ImportState int32
const (
ImportState_ImportPending ImportState = 0
ImportState_ImportFailed ImportState = 1
ImportState_ImportStarted ImportState = 2
ImportState_ImportDownloaded ImportState = 3
ImportState_ImportParsed ImportState = 4
ImportState_ImportPersisted ImportState = 5
ImportState_ImportCompleted ImportState = 6
ImportState_ImportPending ImportState = 0
ImportState_ImportFailed ImportState = 1
ImportState_ImportStarted ImportState = 2
ImportState_ImportDownloaded ImportState = 3
ImportState_ImportParsed ImportState = 4
ImportState_ImportPersisted ImportState = 5
ImportState_ImportCompleted ImportState = 6
ImportState_ImportAllocSegment ImportState = 10
)
var ImportState_name = map[int32]string{
0: "ImportPending",
1: "ImportFailed",
2: "ImportStarted",
3: "ImportDownloaded",
4: "ImportParsed",
5: "ImportPersisted",
6: "ImportCompleted",
0: "ImportPending",
1: "ImportFailed",
2: "ImportStarted",
3: "ImportDownloaded",
4: "ImportParsed",
5: "ImportPersisted",
6: "ImportCompleted",
10: "ImportAllocSegment",
}
var ImportState_value = map[string]int32{
"ImportPending": 0,
"ImportFailed": 1,
"ImportStarted": 2,
"ImportDownloaded": 3,
"ImportParsed": 4,
"ImportPersisted": 5,
"ImportCompleted": 6,
"ImportPending": 0,
"ImportFailed": 1,
"ImportStarted": 2,
"ImportDownloaded": 3,
"ImportParsed": 4,
"ImportPersisted": 5,
"ImportCompleted": 6,
"ImportAllocSegment": 10,
}
func (x ImportState) String() string {
@ -1311,149 +1314,150 @@ func init() {
func init() { proto.RegisterFile("common.proto", fileDescriptor_555bd8c177793206) }
var fileDescriptor_555bd8c177793206 = []byte{
// 2302 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0x49, 0x73, 0x24, 0x47,
0x15, 0x56, 0xa9, 0x7b, 0xd4, 0xea, 0xec, 0x92, 0x94, 0x93, 0x9a, 0x45, 0x1e, 0x8f, 0x6d, 0xb9,
0xb1, 0x8d, 0x10, 0x58, 0x03, 0x76, 0x04, 0x10, 0x44, 0x98, 0x08, 0xa9, 0x5b, 0xd2, 0x74, 0x58,
0x4b, 0x53, 0x2d, 0xd9, 0x0e, 0x22, 0x40, 0x91, 0xaa, 0x7a, 0x6a, 0x95, 0xa7, 0xaa, 0xb2, 0xc8,
0xcc, 0xd6, 0xa8, 0x39, 0x19, 0xf3, 0x07, 0xc0, 0x1c, 0xb8, 0xf2, 0x03, 0x80, 0x60, 0x35, 0x9c,
0x08, 0x76, 0x6c, 0xb6, 0x33, 0x9b, 0x81, 0x23, 0xdc, 0x59, 0xbd, 0x12, 0x2f, 0xb3, 0x36, 0xc9,
0x32, 0x1c, 0xb8, 0x75, 0x7e, 0x6f, 0x7f, 0xf9, 0x96, 0xca, 0x26, 0xae, 0x2f, 0xe2, 0x58, 0x24,
0x2b, 0xa9, 0x14, 0x5a, 0xb0, 0xf9, 0x38, 0x8c, 0x4e, 0x46, 0xca, 0x9e, 0x56, 0x2c, 0xe9, 0xc6,
0xe2, 0x50, 0x88, 0x61, 0x04, 0xb7, 0x0c, 0x78, 0x38, 0x3a, 0xba, 0x15, 0x80, 0xf2, 0x65, 0x98,
0x6a, 0x21, 0x2d, 0x63, 0xfb, 0x80, 0x4c, 0x0d, 0x34, 0xd7, 0x23, 0xc5, 0x9e, 0x20, 0x04, 0xa4,
0x14, 0xf2, 0xc0, 0x17, 0x01, 0x2c, 0x38, 0x8b, 0xce, 0xd2, 0xec, 0x63, 0xf7, 0xaf, 0x5c, 0xa0,
0x75, 0x65, 0x1d, 0xd9, 0x3a, 0x22, 0x00, 0xaf, 0x09, 0xf9, 0x4f, 0x76, 0x8d, 0x4c, 0x49, 0xe0,
0x4a, 0x24, 0x0b, 0x93, 0x8b, 0xce, 0x52, 0xd3, 0xcb, 0x4e, 0xed, 0x0f, 0x12, 0xf7, 0x49, 0x18,
0x3f, 0xc5, 0xa3, 0x11, 0xf4, 0x79, 0x28, 0x19, 0x25, 0xb5, 0x3b, 0x30, 0x36, 0xfa, 0x9b, 0x1e,
0xfe, 0x64, 0x57, 0xc8, 0xa5, 0x13, 0x24, 0x67, 0x82, 0xf6, 0xd0, 0x7e, 0x9c, 0xb4, 0x9e, 0x84,
0x71, 0x97, 0x6b, 0xfe, 0x0e, 0x62, 0x8c, 0xd4, 0x03, 0xae, 0xb9, 0x91, 0x72, 0x3d, 0xf3, 0xbb,
0x7d, 0x93, 0xd4, 0xd7, 0x22, 0x71, 0x58, 0xaa, 0x74, 0x0c, 0x31, 0x53, 0x79, 0x42, 0x68, 0x3f,
0xe2, 0x3e, 0x1c, 0x8b, 0x28, 0x00, 0x69, 0x5c, 0x42, 0xbd, 0x9a, 0x0f, 0x73, 0xbd, 0x9a, 0x0f,
0xd9, 0x87, 0x49, 0x5d, 0x8f, 0x53, 0xeb, 0xcd, 0xec, 0x63, 0x0f, 0x5d, 0x98, 0x81, 0x8a, 0x9a,
0xbd, 0x71, 0x0a, 0x9e, 0x91, 0xc0, 0x14, 0x18, 0x43, 0x6a, 0xa1, 0xb6, 0x58, 0x5b, 0x72, 0xbd,
0xec, 0xd4, 0xfe, 0xc4, 0x19, 0xbb, 0x9b, 0x52, 0x8c, 0x52, 0xd6, 0x23, 0x6e, 0x5a, 0x62, 0x6a,
0xc1, 0x59, 0xac, 0x2d, 0xb5, 0x1e, 0x7b, 0xf8, 0x7f, 0x59, 0x33, 0x4e, 0x7b, 0x67, 0x44, 0xdb,
0x8f, 0x92, 0xc6, 0x6a, 0x10, 0x48, 0x50, 0x8a, 0xcd, 0x92, 0xc9, 0x30, 0xcd, 0x82, 0x99, 0x0c,
0x53, 0xcc, 0x51, 0x2a, 0xa4, 0x36, 0xb1, 0xd4, 0x3c, 0xf3, 0xbb, 0xfd, 0x82, 0x43, 0x1a, 0xdb,
0x6a, 0xb8, 0xc6, 0x15, 0xb0, 0x0f, 0x91, 0xe9, 0x58, 0x0d, 0x0f, 0x4c, 0xbc, 0xf6, 0xc6, 0x6f,
0x5e, 0xe8, 0xc1, 0xb6, 0x1a, 0x9a, 0x38, 0x1b, 0xb1, 0xfd, 0x81, 0x09, 0x8e, 0xd5, 0xb0, 0xd7,
0xcd, 0x34, 0xdb, 0x03, 0xbb, 0x49, 0x9a, 0x3a, 0x8c, 0x41, 0x69, 0x1e, 0xa7, 0x0b, 0xb5, 0x45,
0x67, 0xa9, 0xee, 0x95, 0x00, 0xbb, 0x41, 0xa6, 0x95, 0x18, 0x49, 0x1f, 0x7a, 0xdd, 0x85, 0xba,
0x11, 0x2b, 0xce, 0xed, 0x27, 0x48, 0x73, 0x5b, 0x0d, 0x6f, 0x03, 0x0f, 0x40, 0xb2, 0xf7, 0x93,
0xfa, 0x21, 0x57, 0xd6, 0xa3, 0xd6, 0x3b, 0x7b, 0x84, 0x11, 0x78, 0x86, 0xb3, 0xfd, 0x49, 0xe2,
0x76, 0xb7, 0xb7, 0xfe, 0x0f, 0x0d, 0xe8, 0xba, 0x3a, 0xe6, 0x32, 0xd8, 0xe1, 0x71, 0x5e, 0x88,
0x25, 0xd0, 0x7e, 0xc5, 0x21, 0x6e, 0x5f, 0x86, 0x27, 0x61, 0x04, 0x43, 0x58, 0x3f, 0xd5, 0x6c,
0x83, 0xcc, 0x48, 0xb0, 0xde, 0x57, 0xb3, 0xf7, 0xe0, 0x85, 0x96, 0xbc, 0x8c, 0xd3, 0xa4, 0xd0,
0x95, 0x95, 0x13, 0xdb, 0x27, 0xac, 0xd0, 0x93, 0xe6, 0x06, 0xb2, 0xd2, 0x7b, 0xe4, 0xbf, 0x2a,
0x2b, 0xdc, 0xf1, 0x2e, 0xcb, 0xf3, 0x10, 0x5b, 0x21, 0xf3, 0x85, 0xda, 0x84, 0xc7, 0x70, 0x10,
0x26, 0x01, 0x9c, 0x9a, 0x2b, 0xb9, 0x54, 0xf2, 0x63, 0x68, 0x3d, 0x24, 0x2c, 0x7f, 0x6f, 0x9a,
0x34, 0x8b, 0xae, 0x66, 0x2d, 0xd2, 0x18, 0x8c, 0x7c, 0x1f, 0x94, 0xa2, 0x13, 0x6c, 0x9e, 0xcc,
0xed, 0x27, 0x70, 0x9a, 0x82, 0xaf, 0x21, 0x30, 0x3c, 0xd4, 0x61, 0x97, 0xc9, 0x4c, 0x47, 0x24,
0x09, 0xf8, 0x7a, 0x83, 0x87, 0x11, 0x04, 0x74, 0x92, 0x5d, 0x21, 0xb4, 0x0f, 0x32, 0x0e, 0x95,
0x0a, 0x45, 0xd2, 0x85, 0x24, 0x84, 0x80, 0xd6, 0xd8, 0x75, 0x32, 0xdf, 0x11, 0x51, 0x04, 0xbe,
0x0e, 0x45, 0xb2, 0x23, 0xf4, 0xfa, 0x69, 0xa8, 0xb4, 0xa2, 0x75, 0x54, 0xdb, 0x8b, 0x22, 0x18,
0xf2, 0x68, 0x55, 0x0e, 0x47, 0x31, 0x24, 0x9a, 0x5e, 0x42, 0x1d, 0x19, 0xd8, 0x0d, 0x63, 0x48,
0x50, 0x13, 0x6d, 0x54, 0x50, 0xe3, 0x2c, 0xe6, 0x8d, 0x4e, 0xb3, 0x7b, 0xc8, 0xd5, 0x0c, 0xad,
0x18, 0xe0, 0x31, 0xd0, 0x26, 0x9b, 0x23, 0xad, 0x8c, 0xb4, 0xb7, 0xdb, 0x7f, 0x92, 0x92, 0x8a,
0x06, 0x4f, 0xdc, 0xf5, 0xc0, 0x17, 0x32, 0xa0, 0xad, 0x8a, 0x0b, 0x4f, 0x81, 0xaf, 0x85, 0xec,
0x75, 0xa9, 0x8b, 0x0e, 0x67, 0xe0, 0x00, 0xb8, 0xf4, 0x8f, 0x3d, 0x50, 0xa3, 0x48, 0xd3, 0x19,
0x46, 0x89, 0xbb, 0x11, 0x46, 0xb0, 0x23, 0xf4, 0x86, 0x18, 0x25, 0x01, 0x9d, 0x65, 0xb3, 0x84,
0x6c, 0x83, 0xe6, 0x59, 0x06, 0xe6, 0xd0, 0x6c, 0x87, 0xfb, 0xc7, 0x90, 0x01, 0x94, 0x5d, 0x23,
0xac, 0xc3, 0x93, 0x44, 0xe8, 0x8e, 0x04, 0xae, 0x61, 0xc3, 0xf4, 0x2b, 0xbd, 0x8c, 0xee, 0x9c,
0xc1, 0xc3, 0x08, 0x28, 0x2b, 0xb9, 0xbb, 0x10, 0x41, 0xc1, 0x3d, 0x5f, 0x72, 0x67, 0x38, 0x72,
0x5f, 0x41, 0xe7, 0xd7, 0x46, 0x61, 0x14, 0x98, 0x94, 0xd8, 0x6b, 0xb9, 0x8a, 0x3e, 0x66, 0xce,
0xef, 0x6c, 0xf5, 0x06, 0x7b, 0xf4, 0x1a, 0xbb, 0x4a, 0x2e, 0x67, 0xc8, 0x36, 0x68, 0x19, 0xfa,
0x26, 0x79, 0xd7, 0xd1, 0xd5, 0xdd, 0x91, 0xde, 0x3d, 0xda, 0x86, 0x58, 0xc8, 0x31, 0x5d, 0xc0,
0x0b, 0x35, 0x9a, 0xf2, 0x2b, 0xa2, 0xf7, 0xa0, 0x85, 0xf5, 0x38, 0xd5, 0xe3, 0x32, 0xbd, 0xf4,
0x06, 0xbb, 0x97, 0x5c, 0xdf, 0x4f, 0x03, 0xae, 0xa1, 0x17, 0xe3, 0x30, 0xd9, 0xe3, 0xea, 0x0e,
0x86, 0x3b, 0x92, 0x40, 0xef, 0x65, 0x37, 0xc8, 0xb5, 0xb3, 0x77, 0x51, 0x24, 0xeb, 0x26, 0x0a,
0xda, 0x68, 0x3b, 0x12, 0x02, 0x48, 0x74, 0xc8, 0xa3, 0x5c, 0xf0, 0xbe, 0x52, 0xeb, 0xdb, 0x89,
0xf7, 0x23, 0xd1, 0x46, 0xfe, 0x76, 0xe2, 0x03, 0x6c, 0x81, 0x5c, 0xd9, 0x04, 0xfd, 0x76, 0xca,
0x22, 0x52, 0xb6, 0x42, 0x65, 0x48, 0xfb, 0x0a, 0xa4, 0xca, 0x29, 0x0f, 0x32, 0x46, 0x66, 0x37,
0x41, 0x23, 0x98, 0x63, 0x6d, 0xcc, 0x93, 0x75, 0xcf, 0x13, 0x11, 0xe4, 0xf0, 0xbb, 0x30, 0x07,
0x5d, 0x29, 0xd2, 0x2a, 0xf8, 0x10, 0x86, 0xb9, 0x9b, 0x82, 0xe4, 0x1a, 0x50, 0x47, 0x95, 0xf6,
0x30, 0xea, 0x19, 0x00, 0x66, 0xa0, 0x0a, 0x3f, 0x52, 0xc2, 0x55, 0xab, 0xef, 0xc6, 0x1a, 0xce,
0xb8, 0xb3, 0x8e, 0xcc, 0x49, 0x4b, 0x18, 0x75, 0x66, 0xa4, 0xe8, 0xea, 0x9c, 0xf8, 0x1e, 0x2c,
0x15, 0x2b, 0xb7, 0x29, 0x79, 0xa2, 0x73, 0x7c, 0x99, 0x3d, 0x48, 0xee, 0xf3, 0xe0, 0x48, 0x82,
0x3a, 0xee, 0x8b, 0x28, 0xf4, 0xc7, 0xbd, 0xe4, 0x48, 0x14, 0x25, 0x89, 0x2c, 0xef, 0x45, 0x4f,
0x30, 0x2d, 0x96, 0x9e, 0xc3, 0xef, 0xc3, 0x9c, 0xec, 0x08, 0x3d, 0xc0, 0x81, 0xb7, 0x65, 0x46,
0x28, 0x7d, 0x14, 0xad, 0xec, 0x08, 0x0f, 0xd2, 0x28, 0xf4, 0xf9, 0xea, 0x09, 0x0f, 0x23, 0x7e,
0x18, 0x01, 0x5d, 0x61, 0x8c, 0xcc, 0x74, 0xbb, 0x1e, 0x7c, 0x6a, 0x04, 0x4a, 0x7b, 0xdc, 0x07,
0xfa, 0x97, 0xc6, 0xf2, 0x33, 0x84, 0x98, 0xfa, 0xc1, 0x6f, 0x09, 0x40, 0x6d, 0xe5, 0x69, 0x47,
0x24, 0x40, 0x27, 0x98, 0x4b, 0xa6, 0xf7, 0x93, 0x50, 0xa9, 0x11, 0x04, 0xd4, 0xc1, 0xde, 0xe9,
0x25, 0x7d, 0x29, 0x86, 0xb8, 0xb6, 0xe8, 0x24, 0x52, 0x37, 0xc2, 0x24, 0x54, 0xc7, 0x66, 0x6a,
0x10, 0x32, 0x95, 0x35, 0x51, 0x7d, 0xf9, 0x79, 0x87, 0xb8, 0x03, 0x18, 0xe2, 0x84, 0xb0, 0xca,
0xaf, 0x10, 0x5a, 0x3d, 0x97, 0xea, 0x8b, 0xda, 0x75, 0x70, 0x82, 0x6d, 0x4a, 0x71, 0x37, 0x4c,
0x86, 0x74, 0x12, 0xb5, 0x0d, 0x80, 0x47, 0x46, 0x73, 0x8b, 0x34, 0x36, 0xa2, 0x91, 0x31, 0x53,
0x37, 0x46, 0xf1, 0x80, 0x6c, 0x97, 0x90, 0x84, 0x77, 0x9d, 0x42, 0x40, 0xa7, 0xd8, 0x0c, 0x69,
0xda, 0x0a, 0x47, 0x5a, 0x63, 0xf9, 0xa3, 0x64, 0xee, 0xdc, 0xca, 0x67, 0xd3, 0xa4, 0x9e, 0x99,
0xa6, 0xc4, 0x5d, 0x0b, 0x13, 0x2e, 0xc7, 0x76, 0x8c, 0xd0, 0x00, 0xdb, 0x6b, 0x23, 0x12, 0x5c,
0x67, 0x00, 0x2c, 0xbf, 0xe8, 0x9a, 0x9d, 0x6b, 0x04, 0x67, 0x48, 0x73, 0x3f, 0x09, 0xe0, 0x28,
0x4c, 0x20, 0xa0, 0x13, 0xa6, 0xbd, 0x6d, 0x63, 0x94, 0x7d, 0x16, 0x60, 0x06, 0xd1, 0x99, 0x0a,
0x06, 0xd8, 0xa3, 0xb7, 0xb9, 0xaa, 0x40, 0x47, 0x78, 0x45, 0x5d, 0xf3, 0x45, 0x77, 0x58, 0x15,
0x1f, 0x62, 0xdd, 0x0e, 0x8e, 0xc5, 0xdd, 0x12, 0x53, 0xf4, 0x18, 0x2d, 0x6d, 0x82, 0x1e, 0x8c,
0x95, 0x86, 0xb8, 0x23, 0x92, 0xa3, 0x70, 0xa8, 0x68, 0x88, 0x96, 0xb6, 0x04, 0x0f, 0x2a, 0xe2,
0xcf, 0x62, 0x91, 0x78, 0x10, 0x01, 0x57, 0x55, 0xad, 0x77, 0xcc, 0x80, 0x33, 0xae, 0xae, 0x46,
0x21, 0x57, 0x34, 0xc2, 0x50, 0xd0, 0x4b, 0x7b, 0x8c, 0xf1, 0x52, 0x57, 0x23, 0x0d, 0xd2, 0x9e,
0x13, 0x76, 0x85, 0xcc, 0x59, 0xfe, 0x3e, 0x97, 0x3a, 0x34, 0x4a, 0x5e, 0x72, 0x4c, 0xf9, 0x48,
0x91, 0x96, 0xd8, 0xcb, 0xb8, 0x4f, 0xdc, 0xdb, 0x5c, 0x95, 0xd0, 0xcf, 0x1d, 0x76, 0x8d, 0x5c,
0xce, 0x43, 0x2b, 0xf1, 0x5f, 0x38, 0x6c, 0x9e, 0xcc, 0x62, 0x68, 0x05, 0xa6, 0xe8, 0x2f, 0x0d,
0x88, 0x41, 0x54, 0xc0, 0x5f, 0x19, 0x0d, 0x59, 0x14, 0x15, 0xfc, 0xd7, 0xc6, 0x18, 0x6a, 0xc8,
0x8a, 0x48, 0xd1, 0x57, 0x1d, 0xf4, 0x34, 0x37, 0x96, 0xc1, 0xf4, 0x35, 0xc3, 0x88, 0x5a, 0x0b,
0xc6, 0xd7, 0x0d, 0x63, 0xa6, 0xb3, 0x40, 0xdf, 0x30, 0xe8, 0x6d, 0x9e, 0x04, 0xe2, 0xe8, 0xa8,
0x40, 0xdf, 0x74, 0xd8, 0x02, 0x99, 0x47, 0xf1, 0x35, 0x1e, 0xf1, 0xc4, 0x2f, 0xf9, 0xdf, 0x72,
0xd8, 0x55, 0x42, 0xcf, 0x99, 0x53, 0xf4, 0xb9, 0x49, 0x46, 0xf3, 0xfc, 0x9a, 0xe6, 0xa1, 0x5f,
0x9e, 0x34, 0xb9, 0xca, 0x18, 0x2d, 0xf6, 0x95, 0x49, 0x36, 0x6b, 0x93, 0x6e, 0xcf, 0x5f, 0x9d,
0x64, 0x2d, 0x32, 0xd5, 0x4b, 0x14, 0x48, 0x4d, 0x3f, 0x87, 0xf5, 0x3d, 0x65, 0x87, 0x25, 0xfd,
0x3c, 0xb6, 0xd1, 0x25, 0x53, 0xdf, 0xf4, 0x05, 0x5c, 0xc4, 0xcc, 0x03, 0x05, 0x49, 0x50, 0xe9,
0x1d, 0x45, 0xbf, 0x60, 0x24, 0xec, 0xa6, 0xa3, 0x7f, 0xab, 0x99, 0xd4, 0x54, 0xd7, 0xde, 0xdf,
0x6b, 0xe8, 0xc2, 0x26, 0xe8, 0xb2, 0x9d, 0xe9, 0x3f, 0x6a, 0xec, 0x06, 0xb9, 0x9a, 0x63, 0x66,
0x09, 0x15, 0x8d, 0xfc, 0xcf, 0x1a, 0xbb, 0x49, 0xae, 0xe3, 0x44, 0x2e, 0xea, 0x06, 0x85, 0x42,
0xa5, 0x43, 0x5f, 0xd1, 0x7f, 0xd5, 0xd8, 0xbd, 0xe4, 0xda, 0x26, 0xe8, 0xe2, 0x3e, 0x2a, 0xc4,
0x7f, 0xd7, 0xd8, 0x0c, 0x99, 0xf6, 0x70, 0x4b, 0xc1, 0x09, 0xd0, 0x57, 0x6b, 0x78, 0xa9, 0xf9,
0x31, 0x73, 0xe7, 0xb5, 0x1a, 0xa6, 0xfa, 0x69, 0xae, 0xfd, 0xe3, 0x6e, 0xdc, 0x39, 0xe6, 0x49,
0x02, 0x91, 0xa2, 0xaf, 0xd7, 0x30, 0xa1, 0x1e, 0xc4, 0xe2, 0x04, 0x2a, 0xf0, 0x1b, 0x26, 0x68,
0xc3, 0xfc, 0xb1, 0x11, 0xc8, 0x71, 0x41, 0x78, 0xb3, 0x86, 0x57, 0x63, 0xf9, 0xcf, 0x52, 0xde,
0xaa, 0xb1, 0xfb, 0xc8, 0x82, 0x1d, 0x16, 0xf9, 0xc5, 0x20, 0x71, 0x08, 0x38, 0x49, 0xe9, 0x73,
0xf5, 0x42, 0x63, 0x17, 0x22, 0xcd, 0x0b, 0xb9, 0xcf, 0xd4, 0xd1, 0x2f, 0x6c, 0xae, 0x72, 0x80,
0x2a, 0xfa, 0x7c, 0x1d, 0x6f, 0x74, 0x13, 0x74, 0x36, 0x43, 0x15, 0xfd, 0xac, 0x41, 0x32, 0xcd,
0x46, 0xe5, 0x6f, 0xea, 0x6c, 0x8e, 0x10, 0xdb, 0x93, 0x06, 0xf8, 0x6d, 0xae, 0x0a, 0x3f, 0x53,
0x4e, 0x40, 0x9a, 0x19, 0x4e, 0x7f, 0x57, 0x18, 0xa8, 0x4c, 0x3e, 0xfa, 0xfb, 0x3a, 0xa6, 0x6c,
0x2f, 0x8c, 0x61, 0x2f, 0xf4, 0xef, 0xd0, 0xaf, 0x35, 0x31, 0x65, 0x26, 0xa2, 0x1d, 0x11, 0x80,
0xbd, 0xe1, 0xaf, 0x37, 0xb1, 0x60, 0xb0, 0x0e, 0x6d, 0xc1, 0x7c, 0xc3, 0x9c, 0xb3, 0xe9, 0xdd,
0xeb, 0xd2, 0x6f, 0xe2, 0xe7, 0x12, 0xc9, 0xce, 0x7b, 0x83, 0x5d, 0xfa, 0xad, 0x26, 0x9a, 0x5a,
0x8d, 0x22, 0xe1, 0x73, 0x5d, 0x74, 0xc3, 0xb7, 0x9b, 0xd8, 0x4e, 0x15, 0xeb, 0xd9, 0xad, 0xbd,
0xd8, 0xc4, 0xdc, 0x67, 0xb8, 0x29, 0xb6, 0x2e, 0x0e, 0xc5, 0xef, 0x18, 0xad, 0xf8, 0x78, 0x43,
0x4f, 0xf6, 0x34, 0xfd, 0xae, 0xe1, 0x3b, 0xff, 0x05, 0x40, 0xff, 0xd0, 0xca, 0xea, 0xab, 0x82,
0xbd, 0xd2, 0xb2, 0xfd, 0x71, 0x76, 0xe5, 0xd3, 0x3f, 0x1a, 0xf8, 0xfc, 0x67, 0x02, 0xfd, 0x53,
0x0b, 0x1d, 0xab, 0x6e, 0x7a, 0xfc, 0xde, 0x55, 0xf4, 0xcf, 0x2d, 0xf4, 0xa0, 0xdc, 0xe9, 0xf4,
0xfb, 0x2e, 0x26, 0x2b, 0xdf, 0xe6, 0xf4, 0x07, 0x2e, 0x86, 0x79, 0x6e, 0x8f, 0xd3, 0x1f, 0xba,
0xe6, 0x3a, 0x8a, 0x0d, 0x4e, 0x7f, 0x54, 0x01, 0x90, 0x8b, 0xfe, 0xd8, 0x35, 0x13, 0xe8, 0xcc,
0xd6, 0xa6, 0x3f, 0x71, 0xd1, 0xb7, 0xf3, 0xfb, 0x9a, 0xfe, 0xd4, 0xb5, 0xd7, 0x5d, 0x6c, 0x6a,
0xfa, 0x33, 0x17, 0x3b, 0xe0, 0xe2, 0x1d, 0x4d, 0x5f, 0x32, 0xb6, 0xca, 0xed, 0x4c, 0x5f, 0x76,
0x97, 0xdb, 0xa4, 0xd1, 0x55, 0x91, 0xd9, 0x1b, 0x0d, 0x52, 0xeb, 0xaa, 0x88, 0x4e, 0xe0, 0x98,
0x5d, 0x13, 0x22, 0x5a, 0x3f, 0x4d, 0xe5, 0x53, 0x1f, 0xa0, 0xce, 0xf2, 0x1a, 0x99, 0xeb, 0x88,
0x38, 0xe5, 0x45, 0xbb, 0x99, 0x55, 0x61, 0x77, 0x0c, 0x04, 0xb6, 0x54, 0x26, 0x70, 0x56, 0xaf,
0x9f, 0x82, 0x3f, 0x32, 0x1b, 0xcd, 0xc1, 0x23, 0x0a, 0x61, 0x92, 0x03, 0x3a, 0xb9, 0xfc, 0x0c,
0xa1, 0x1d, 0x91, 0xa8, 0x50, 0x69, 0x48, 0xfc, 0xf1, 0x16, 0x9c, 0x40, 0x64, 0xf6, 0xa6, 0x96,
0x22, 0x19, 0xd2, 0x09, 0xf3, 0x24, 0x00, 0xf3, 0x69, 0x6f, 0xb7, 0xeb, 0x1a, 0x7e, 0xd6, 0x99,
0xef, 0xfe, 0x59, 0x42, 0xd6, 0x4f, 0x20, 0xd1, 0x23, 0x1e, 0x45, 0x63, 0x5a, 0xc3, 0x73, 0x67,
0xa4, 0xb4, 0x88, 0xc3, 0x4f, 0x9b, 0xfd, 0xfd, 0x45, 0x87, 0xb4, 0xec, 0x2a, 0x2d, 0x5c, 0xb3,
0xc7, 0x3e, 0x24, 0x41, 0x68, 0x94, 0xe3, 0x67, 0xab, 0x81, 0xb2, 0xa5, 0xef, 0x94, 0x4c, 0x03,
0xcd, 0xa5, 0xce, 0xdf, 0x17, 0x16, 0xea, 0x8a, 0xbb, 0x49, 0x24, 0x78, 0x60, 0xf6, 0x79, 0x21,
0xda, 0xe7, 0x52, 0x99, 0xa5, 0x8e, 0x5f, 0xf5, 0x99, 0x7e, 0x69, 0xe2, 0x09, 0xe8, 0xa5, 0x12,
0x2c, 0x63, 0x9e, 0x5a, 0xbe, 0x9f, 0xb8, 0xd5, 0x97, 0x99, 0xf1, 0xbc, 0x5c, 0x77, 0x13, 0xcb,
0xaf, 0x38, 0xb8, 0x40, 0xce, 0x3f, 0xad, 0x68, 0xe5, 0x25, 0xb8, 0x1a, 0x45, 0xf6, 0x85, 0x54,
0x20, 0xb6, 0xe0, 0x6c, 0x04, 0x05, 0x88, 0x45, 0x47, 0x71, 0x98, 0xcf, 0x56, 0x24, 0x35, 0x48,
0x5a, 0x3b, 0xc3, 0xe6, 0x01, 0xc7, 0x00, 0xaa, 0x10, 0xf6, 0xad, 0x7d, 0x17, 0x55, 0xb8, 0xcc,
0xfa, 0xa1, 0x53, 0x67, 0xd0, 0xac, 0x00, 0x68, 0xe3, 0x8c, 0x37, 0xd9, 0x5e, 0x98, 0x3e, 0x03,
0x66, 0xfb, 0xa1, 0xf9, 0x11, 0x41, 0x2e, 0x17, 0x4f, 0xce, 0x03, 0x38, 0xd5, 0x07, 0xe2, 0xf0,
0x59, 0xf6, 0xc0, 0x8a, 0xfd, 0xcb, 0x68, 0x25, 0xff, 0xcb, 0x68, 0x65, 0x1b, 0x94, 0xe2, 0x43,
0xd8, 0x4d, 0xcd, 0x02, 0x5d, 0xf8, 0x6b, 0xc3, 0xbc, 0xa9, 0x2f, 0x7e, 0xe9, 0x56, 0xdf, 0xc8,
0xde, 0x5c, 0x5a, 0x39, 0xed, 0x1e, 0x3e, 0xbb, 0xf6, 0x34, 0x99, 0x0d, 0x45, 0x2e, 0x37, 0x94,
0xa9, 0xbf, 0xd6, 0xea, 0x18, 0xb9, 0x3e, 0xea, 0xe8, 0x3b, 0x1f, 0x7f, 0x7c, 0x18, 0xea, 0xe3,
0xd1, 0x21, 0x6a, 0xbb, 0x65, 0xd9, 0x1e, 0x0d, 0x45, 0xf6, 0xeb, 0x56, 0x98, 0x68, 0x6c, 0xee,
0xc8, 0xfe, 0x99, 0x75, 0xcb, 0x5a, 0x4c, 0x0f, 0xbf, 0xe4, 0x38, 0x87, 0x53, 0x06, 0x7a, 0xfc,
0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x1f, 0x15, 0x60, 0x12, 0x13, 0x00, 0x00,
// 2309 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0xd9, 0x73, 0x23, 0x47,
0x19, 0xf7, 0x58, 0x5a, 0xcb, 0x6a, 0x8d, 0xed, 0xde, 0xf6, 0x1e, 0xce, 0x66, 0x93, 0x38, 0x22,
0x09, 0xc6, 0x10, 0x2f, 0x24, 0x55, 0x40, 0x51, 0x15, 0xaa, 0x6c, 0xc9, 0xf6, 0xaa, 0xe2, 0x43,
0x8c, 0xec, 0x24, 0x45, 0x15, 0xb8, 0xda, 0x33, 0x9f, 0xe5, 0xc9, 0xce, 0x4c, 0x0f, 0xdd, 0x2d,
0xaf, 0xc5, 0x53, 0x08, 0xff, 0x00, 0x84, 0x7f, 0x80, 0x3f, 0x80, 0xfb, 0x08, 0x3c, 0x51, 0xdc,
0x24, 0x5c, 0xcf, 0x5c, 0x01, 0x1e, 0xe1, 0x9d, 0x33, 0x27, 0xf5, 0x75, 0xcf, 0x65, 0xc7, 0x81,
0x07, 0xde, 0xd4, 0xbf, 0xef, 0xfe, 0xfa, 0x3b, 0xa6, 0x45, 0x5c, 0x5f, 0xc4, 0xb1, 0x48, 0x56,
0x52, 0x29, 0xb4, 0x60, 0xf3, 0x71, 0x18, 0x9d, 0x8c, 0x94, 0x3d, 0xad, 0x58, 0xd2, 0x8d, 0xc5,
0xa1, 0x10, 0xc3, 0x08, 0x6e, 0x19, 0xf0, 0x70, 0x74, 0x74, 0x2b, 0x00, 0xe5, 0xcb, 0x30, 0xd5,
0x42, 0x5a, 0xc6, 0xf6, 0x01, 0x99, 0x1a, 0x68, 0xae, 0x47, 0x8a, 0x3d, 0x41, 0x08, 0x48, 0x29,
0xe4, 0x81, 0x2f, 0x02, 0x58, 0x70, 0x16, 0x9d, 0xa5, 0xd9, 0xc7, 0xee, 0x5f, 0xb9, 0x40, 0xeb,
0xca, 0x3a, 0xb2, 0x75, 0x44, 0x00, 0x5e, 0x13, 0xf2, 0x9f, 0xec, 0x1a, 0x99, 0x92, 0xc0, 0x95,
0x48, 0x16, 0x26, 0x17, 0x9d, 0xa5, 0xa6, 0x97, 0x9d, 0xda, 0x1f, 0x24, 0xee, 0x93, 0x30, 0x7e,
0x8a, 0x47, 0x23, 0xe8, 0xf3, 0x50, 0x32, 0x4a, 0x6a, 0x77, 0x60, 0x6c, 0xf4, 0x37, 0x3d, 0xfc,
0xc9, 0xae, 0x90, 0x4b, 0x27, 0x48, 0xce, 0x04, 0xed, 0xa1, 0xfd, 0x38, 0x69, 0x3d, 0x09, 0xe3,
0x2e, 0xd7, 0xfc, 0x1d, 0xc4, 0x18, 0xa9, 0x07, 0x5c, 0x73, 0x23, 0xe5, 0x7a, 0xe6, 0x77, 0xfb,
0x26, 0xa9, 0xaf, 0x45, 0xe2, 0xb0, 0x54, 0xe9, 0x18, 0x62, 0xa6, 0xf2, 0x84, 0xd0, 0x7e, 0xc4,
0x7d, 0x38, 0x16, 0x51, 0x00, 0xd2, 0xb8, 0x84, 0x7a, 0x35, 0x1f, 0xe6, 0x7a, 0x35, 0x1f, 0xb2,
0x0f, 0x93, 0xba, 0x1e, 0xa7, 0xd6, 0x9b, 0xd9, 0xc7, 0x1e, 0xba, 0x30, 0x03, 0x15, 0x35, 0x7b,
0xe3, 0x14, 0x3c, 0x23, 0x81, 0x29, 0x30, 0x86, 0xd4, 0x42, 0x6d, 0xb1, 0xb6, 0xe4, 0x7a, 0xd9,
0xa9, 0xfd, 0x89, 0x33, 0x76, 0x37, 0xa5, 0x18, 0xa5, 0xac, 0x47, 0xdc, 0xb4, 0xc4, 0xd4, 0x82,
0xb3, 0x58, 0x5b, 0x6a, 0x3d, 0xf6, 0xf0, 0xff, 0xb2, 0x66, 0x9c, 0xf6, 0xce, 0x88, 0xb6, 0x1f,
0x25, 0x8d, 0xd5, 0x20, 0x90, 0xa0, 0x14, 0x9b, 0x25, 0x93, 0x61, 0x9a, 0x05, 0x33, 0x19, 0xa6,
0x98, 0xa3, 0x54, 0x48, 0x6d, 0x62, 0xa9, 0x79, 0xe6, 0x77, 0xfb, 0x05, 0x87, 0x34, 0xb6, 0xd5,
0x70, 0x8d, 0x2b, 0x60, 0x1f, 0x22, 0xd3, 0xb1, 0x1a, 0x1e, 0x98, 0x78, 0xed, 0x8d, 0xdf, 0xbc,
0xd0, 0x83, 0x6d, 0x35, 0x34, 0x71, 0x36, 0x62, 0xfb, 0x03, 0x13, 0x1c, 0xab, 0x61, 0xaf, 0x9b,
0x69, 0xb6, 0x07, 0x76, 0x93, 0x34, 0x75, 0x18, 0x83, 0xd2, 0x3c, 0x4e, 0x17, 0x6a, 0x8b, 0xce,
0x52, 0xdd, 0x2b, 0x01, 0x76, 0x83, 0x4c, 0x2b, 0x31, 0x92, 0x3e, 0xf4, 0xba, 0x0b, 0x75, 0x23,
0x56, 0x9c, 0xdb, 0x4f, 0x90, 0xe6, 0xb6, 0x1a, 0xde, 0x06, 0x1e, 0x80, 0x64, 0xef, 0x27, 0xf5,
0x43, 0xae, 0xac, 0x47, 0xad, 0x77, 0xf6, 0x08, 0x23, 0xf0, 0x0c, 0x67, 0xfb, 0x93, 0xc4, 0xed,
0x6e, 0x6f, 0xfd, 0x1f, 0x1a, 0xd0, 0x75, 0x75, 0xcc, 0x65, 0xb0, 0xc3, 0xe3, 0xbc, 0x10, 0x4b,
0xa0, 0xfd, 0x8a, 0x43, 0xdc, 0xbe, 0x0c, 0x4f, 0xc2, 0x08, 0x86, 0xb0, 0x7e, 0xaa, 0xd9, 0x06,
0x99, 0x91, 0x60, 0xbd, 0xaf, 0x66, 0xef, 0xc1, 0x0b, 0x2d, 0x79, 0x19, 0xa7, 0x49, 0xa1, 0x2b,
0x2b, 0x27, 0xb6, 0x4f, 0x58, 0xa1, 0x27, 0xcd, 0x0d, 0x64, 0xa5, 0xf7, 0xc8, 0x7f, 0x55, 0x56,
0xb8, 0xe3, 0x5d, 0x96, 0xe7, 0x21, 0xb6, 0x42, 0xe6, 0x0b, 0xb5, 0x09, 0x8f, 0xe1, 0x20, 0x4c,
0x02, 0x38, 0x35, 0x57, 0x72, 0xa9, 0xe4, 0xc7, 0xd0, 0x7a, 0x48, 0x58, 0xfe, 0xde, 0x34, 0x69,
0x16, 0x5d, 0xcd, 0x5a, 0xa4, 0x31, 0x18, 0xf9, 0x3e, 0x28, 0x45, 0x27, 0xd8, 0x3c, 0x99, 0xdb,
0x4f, 0xe0, 0x34, 0x05, 0x5f, 0x43, 0x60, 0x78, 0xa8, 0xc3, 0x2e, 0x93, 0x99, 0x8e, 0x48, 0x12,
0xf0, 0xf5, 0x06, 0x0f, 0x23, 0x08, 0xe8, 0x24, 0xbb, 0x42, 0x68, 0x1f, 0x64, 0x1c, 0x2a, 0x15,
0x8a, 0xa4, 0x0b, 0x49, 0x08, 0x01, 0xad, 0xb1, 0xeb, 0x64, 0xbe, 0x23, 0xa2, 0x08, 0x7c, 0x1d,
0x8a, 0x64, 0x47, 0xe8, 0xf5, 0xd3, 0x50, 0x69, 0x45, 0xeb, 0xa8, 0xb6, 0x17, 0x45, 0x30, 0xe4,
0xd1, 0xaa, 0x1c, 0x8e, 0x62, 0x48, 0x34, 0xbd, 0x84, 0x3a, 0x32, 0xb0, 0x1b, 0xc6, 0x90, 0xa0,
0x26, 0xda, 0xa8, 0xa0, 0xc6, 0x59, 0xcc, 0x1b, 0x9d, 0x66, 0xf7, 0x90, 0xab, 0x19, 0x5a, 0x31,
0xc0, 0x63, 0xa0, 0x4d, 0x36, 0x47, 0x5a, 0x19, 0x69, 0x6f, 0xb7, 0xff, 0x24, 0x25, 0x15, 0x0d,
0x9e, 0xb8, 0xeb, 0x81, 0x2f, 0x64, 0x40, 0x5b, 0x15, 0x17, 0x9e, 0x02, 0x5f, 0x0b, 0xd9, 0xeb,
0x52, 0x17, 0x1d, 0xce, 0xc0, 0x01, 0x70, 0xe9, 0x1f, 0x7b, 0xa0, 0x46, 0x91, 0xa6, 0x33, 0x8c,
0x12, 0x77, 0x23, 0x8c, 0x60, 0x47, 0xe8, 0x0d, 0x31, 0x4a, 0x02, 0x3a, 0xcb, 0x66, 0x09, 0xd9,
0x06, 0xcd, 0xb3, 0x0c, 0xcc, 0xa1, 0xd9, 0x0e, 0xf7, 0x8f, 0x21, 0x03, 0x28, 0xbb, 0x46, 0x58,
0x87, 0x27, 0x89, 0xd0, 0x1d, 0x09, 0x5c, 0xc3, 0x86, 0xe9, 0x57, 0x7a, 0x19, 0xdd, 0x39, 0x83,
0x87, 0x11, 0x50, 0x56, 0x72, 0x77, 0x21, 0x82, 0x82, 0x7b, 0xbe, 0xe4, 0xce, 0x70, 0xe4, 0xbe,
0x82, 0xce, 0xaf, 0x8d, 0xc2, 0x28, 0x30, 0x29, 0xb1, 0xd7, 0x72, 0x15, 0x7d, 0xcc, 0x9c, 0xdf,
0xd9, 0xea, 0x0d, 0xf6, 0xe8, 0x35, 0x76, 0x95, 0x5c, 0xce, 0x90, 0x6d, 0xd0, 0x32, 0xf4, 0x4d,
0xf2, 0xae, 0xa3, 0xab, 0xbb, 0x23, 0xbd, 0x7b, 0xb4, 0x0d, 0xb1, 0x90, 0x63, 0xba, 0x80, 0x17,
0x6a, 0x34, 0xe5, 0x57, 0x44, 0xef, 0x41, 0x0b, 0xeb, 0x71, 0xaa, 0xc7, 0x65, 0x7a, 0xe9, 0x0d,
0x76, 0x2f, 0xb9, 0xbe, 0x9f, 0x06, 0x5c, 0x43, 0x2f, 0xc6, 0x61, 0xb2, 0xc7, 0xd5, 0x1d, 0x0c,
0x77, 0x24, 0x81, 0xde, 0xcb, 0x6e, 0x90, 0x6b, 0x67, 0xef, 0xa2, 0x48, 0xd6, 0x4d, 0x14, 0xb4,
0xd1, 0x76, 0x24, 0x04, 0x90, 0xe8, 0x90, 0x47, 0xb9, 0xe0, 0x7d, 0xa5, 0xd6, 0xb7, 0x13, 0xef,
0x47, 0xa2, 0x8d, 0xfc, 0xed, 0xc4, 0x07, 0xd8, 0x02, 0xb9, 0xb2, 0x09, 0xfa, 0xed, 0x94, 0x45,
0xa4, 0x6c, 0x85, 0xca, 0x90, 0xf6, 0x15, 0x48, 0x95, 0x53, 0x1e, 0x64, 0x8c, 0xcc, 0x6e, 0x82,
0x46, 0x30, 0xc7, 0xda, 0x98, 0x27, 0xeb, 0x9e, 0x27, 0x22, 0xc8, 0xe1, 0x77, 0x61, 0x0e, 0xba,
0x52, 0xa4, 0x55, 0xf0, 0x21, 0x0c, 0x73, 0x37, 0x05, 0xc9, 0x35, 0xa0, 0x8e, 0x2a, 0xed, 0x61,
0xd4, 0x33, 0x00, 0xcc, 0x40, 0x15, 0x7e, 0xa4, 0x84, 0xab, 0x56, 0xdf, 0x8d, 0x35, 0x9c, 0x71,
0x67, 0x1d, 0x99, 0x93, 0x96, 0x30, 0xea, 0xcc, 0x48, 0xd1, 0xd5, 0x39, 0xf1, 0x3d, 0x58, 0x2a,
0x56, 0x6e, 0x53, 0xf2, 0x44, 0xe7, 0xf8, 0x32, 0x7b, 0x90, 0xdc, 0xe7, 0xc1, 0x91, 0x04, 0x75,
0xdc, 0x17, 0x51, 0xe8, 0x8f, 0x7b, 0xc9, 0x91, 0x28, 0x4a, 0x12, 0x59, 0xde, 0x8b, 0x9e, 0x60,
0x5a, 0x2c, 0x3d, 0x87, 0xdf, 0x87, 0x39, 0xd9, 0x11, 0x7a, 0x80, 0x03, 0x6f, 0xcb, 0x8c, 0x50,
0xfa, 0x28, 0x5a, 0xd9, 0x11, 0x1e, 0xa4, 0x51, 0xe8, 0xf3, 0xd5, 0x13, 0x1e, 0x46, 0xfc, 0x30,
0x02, 0xba, 0xc2, 0x18, 0x99, 0xe9, 0x76, 0x3d, 0xf8, 0xd4, 0x08, 0x94, 0xf6, 0xb8, 0x0f, 0xf4,
0x2f, 0x8d, 0xe5, 0x67, 0x08, 0x31, 0xf5, 0x83, 0xdf, 0x12, 0x80, 0xda, 0xca, 0xd3, 0x8e, 0x48,
0x80, 0x4e, 0x30, 0x97, 0x4c, 0xef, 0x27, 0xa1, 0x52, 0x23, 0x08, 0xa8, 0x83, 0xbd, 0xd3, 0x4b,
0xfa, 0x52, 0x0c, 0x71, 0x6d, 0xd1, 0x49, 0xa4, 0x6e, 0x84, 0x49, 0xa8, 0x8e, 0xcd, 0xd4, 0x20,
0x64, 0x2a, 0x6b, 0xa2, 0xfa, 0xf2, 0xf3, 0x0e, 0x71, 0x07, 0x30, 0xc4, 0x09, 0x61, 0x95, 0x5f,
0x21, 0xb4, 0x7a, 0x2e, 0xd5, 0x17, 0xb5, 0xeb, 0xe0, 0x04, 0xdb, 0x94, 0xe2, 0x6e, 0x98, 0x0c,
0xe9, 0x24, 0x6a, 0x1b, 0x00, 0x8f, 0x8c, 0xe6, 0x16, 0x69, 0x6c, 0x44, 0x23, 0x63, 0xa6, 0x6e,
0x8c, 0xe2, 0x01, 0xd9, 0x2e, 0x21, 0x09, 0xef, 0x3a, 0x85, 0x80, 0x4e, 0xb1, 0x19, 0xd2, 0xb4,
0x15, 0x8e, 0xb4, 0xc6, 0xf2, 0x47, 0xc9, 0xdc, 0xb9, 0x95, 0xcf, 0xa6, 0x49, 0x3d, 0x33, 0x4d,
0x89, 0xbb, 0x16, 0x26, 0x5c, 0x8e, 0xed, 0x18, 0xa1, 0x01, 0xb6, 0xd7, 0x46, 0x24, 0xb8, 0xce,
0x00, 0x58, 0x7e, 0xd1, 0x35, 0x3b, 0xd7, 0x08, 0xce, 0x90, 0xe6, 0x7e, 0x12, 0xc0, 0x51, 0x98,
0x40, 0x40, 0x27, 0x4c, 0x7b, 0xdb, 0xc6, 0x28, 0xfb, 0x2c, 0xc0, 0x0c, 0xa2, 0x33, 0x15, 0x0c,
0xb0, 0x47, 0x6f, 0x73, 0x55, 0x81, 0x8e, 0xf0, 0x8a, 0xba, 0xe6, 0x8b, 0xee, 0xb0, 0x2a, 0x3e,
0xc4, 0xba, 0x1d, 0x1c, 0x8b, 0xbb, 0x25, 0xa6, 0xe8, 0x31, 0x5a, 0xda, 0x04, 0x3d, 0x18, 0x2b,
0x0d, 0x71, 0x47, 0x24, 0x47, 0xe1, 0x50, 0xd1, 0x10, 0x2d, 0x6d, 0x09, 0x1e, 0x54, 0xc4, 0x9f,
0xc5, 0x22, 0xf1, 0x20, 0x02, 0xae, 0xaa, 0x5a, 0xef, 0x98, 0x01, 0x67, 0x5c, 0x5d, 0x8d, 0x42,
0xae, 0x68, 0x84, 0xa1, 0xa0, 0x97, 0xf6, 0x18, 0xe3, 0xa5, 0xae, 0x46, 0x1a, 0xa4, 0x3d, 0x27,
0xec, 0x0a, 0x99, 0xb3, 0xfc, 0x7d, 0x2e, 0x75, 0x68, 0x94, 0xbc, 0xe4, 0x98, 0xf2, 0x91, 0x22,
0x2d, 0xb1, 0x97, 0x71, 0x9f, 0xb8, 0xb7, 0xb9, 0x2a, 0xa1, 0x9f, 0x3b, 0xec, 0x1a, 0xb9, 0x9c,
0x87, 0x56, 0xe2, 0xbf, 0x70, 0xd8, 0x3c, 0x99, 0xc5, 0xd0, 0x0a, 0x4c, 0xd1, 0x5f, 0x1a, 0x10,
0x83, 0xa8, 0x80, 0xbf, 0x32, 0x1a, 0xb2, 0x28, 0x2a, 0xf8, 0xaf, 0x8d, 0x31, 0xd4, 0x90, 0x15,
0x91, 0xa2, 0xaf, 0x3a, 0xe8, 0x69, 0x6e, 0x2c, 0x83, 0xe9, 0x6b, 0x86, 0x11, 0xb5, 0x16, 0x8c,
0xaf, 0x1b, 0xc6, 0x4c, 0x67, 0x81, 0xbe, 0x61, 0xd0, 0xdb, 0x3c, 0x09, 0xc4, 0xd1, 0x51, 0x81,
0xbe, 0xe9, 0xb0, 0x05, 0x32, 0x8f, 0xe2, 0x6b, 0x3c, 0xe2, 0x89, 0x5f, 0xf2, 0xbf, 0xe5, 0xb0,
0xab, 0x84, 0x9e, 0x33, 0xa7, 0xe8, 0x73, 0x93, 0x8c, 0xe6, 0xf9, 0x35, 0xcd, 0x43, 0xbf, 0x34,
0x69, 0x72, 0x95, 0x31, 0x5a, 0xec, 0xcb, 0x93, 0x6c, 0xd6, 0x26, 0xdd, 0x9e, 0xbf, 0x32, 0xc9,
0x5a, 0x64, 0xaa, 0x97, 0x28, 0x90, 0x9a, 0x7e, 0x0e, 0xeb, 0x7b, 0xca, 0x0e, 0x4b, 0xfa, 0x79,
0x6c, 0xa3, 0x4b, 0xa6, 0xbe, 0xe9, 0x0b, 0xb8, 0x88, 0x99, 0x07, 0x0a, 0x92, 0xa0, 0xd2, 0x3b,
0x8a, 0x7e, 0xc1, 0x48, 0xd8, 0x4d, 0x47, 0xff, 0x56, 0x33, 0xa9, 0xa9, 0xae, 0xbd, 0xbf, 0xd7,
0xd0, 0x85, 0x4d, 0xd0, 0x65, 0x3b, 0xd3, 0x7f, 0xd4, 0xd8, 0x0d, 0x72, 0x35, 0xc7, 0xcc, 0x12,
0x2a, 0x1a, 0xf9, 0x9f, 0x35, 0x76, 0x93, 0x5c, 0xc7, 0x89, 0x5c, 0xd4, 0x0d, 0x0a, 0x85, 0x4a,
0x87, 0xbe, 0xa2, 0xff, 0xaa, 0xb1, 0x7b, 0xc9, 0xb5, 0x4d, 0xd0, 0xc5, 0x7d, 0x54, 0x88, 0xff,
0xae, 0xb1, 0x19, 0x32, 0xed, 0xe1, 0x96, 0x82, 0x13, 0xa0, 0xaf, 0xd6, 0xf0, 0x52, 0xf3, 0x63,
0xe6, 0xce, 0x6b, 0x35, 0x4c, 0xf5, 0xd3, 0x5c, 0xfb, 0xc7, 0xdd, 0xb8, 0x73, 0xcc, 0x93, 0x04,
0x22, 0x45, 0x5f, 0xaf, 0x61, 0x42, 0x3d, 0x88, 0xc5, 0x09, 0x54, 0xe0, 0x37, 0x4c, 0xd0, 0x86,
0xf9, 0x63, 0x23, 0x90, 0xe3, 0x82, 0xf0, 0x66, 0x0d, 0xaf, 0xc6, 0xf2, 0x9f, 0xa5, 0xbc, 0x55,
0x63, 0xf7, 0x91, 0x05, 0x3b, 0x2c, 0xf2, 0x8b, 0x41, 0xe2, 0x10, 0x70, 0x92, 0xd2, 0xe7, 0xea,
0x85, 0xc6, 0x2e, 0x44, 0x9a, 0x17, 0x72, 0x9f, 0xa9, 0xa3, 0x5f, 0xd8, 0x5c, 0xe5, 0x00, 0x55,
0xf4, 0xf9, 0x3a, 0xde, 0xe8, 0x26, 0xe8, 0x6c, 0x86, 0x2a, 0xfa, 0x59, 0x83, 0x64, 0x9a, 0x8d,
0xca, 0xdf, 0xd4, 0xd9, 0x1c, 0x21, 0xb6, 0x27, 0x0d, 0xf0, 0xdb, 0x5c, 0x15, 0x7e, 0xa6, 0x9c,
0x80, 0x34, 0x33, 0x9c, 0xfe, 0xae, 0x30, 0x50, 0x99, 0x7c, 0xf4, 0xf7, 0x75, 0x4c, 0xd9, 0x5e,
0x18, 0xc3, 0x5e, 0xe8, 0xdf, 0xa1, 0x5f, 0x6b, 0x62, 0xca, 0x4c, 0x44, 0x3b, 0x22, 0x00, 0x7b,
0xc3, 0x5f, 0x6f, 0x62, 0xc1, 0x60, 0x1d, 0xda, 0x82, 0xf9, 0x86, 0x39, 0x67, 0xd3, 0xbb, 0xd7,
0xa5, 0xdf, 0xc4, 0xcf, 0x25, 0x92, 0x9d, 0xf7, 0x06, 0xbb, 0xf4, 0x5b, 0x4d, 0x34, 0xb5, 0x1a,
0x45, 0xc2, 0xe7, 0xba, 0xe8, 0x86, 0x6f, 0x37, 0xb1, 0x9d, 0x2a, 0xd6, 0xb3, 0x5b, 0x7b, 0xb1,
0x89, 0xb9, 0xcf, 0x70, 0x53, 0x6c, 0x5d, 0x1c, 0x8a, 0xdf, 0x31, 0x5a, 0xf1, 0xf1, 0x86, 0x9e,
0xec, 0x69, 0xfa, 0x5d, 0xc3, 0x77, 0xfe, 0x0b, 0x80, 0xfe, 0xa1, 0x95, 0xd5, 0x57, 0x05, 0x7b,
0xa5, 0x65, 0xfb, 0xe3, 0xec, 0xca, 0xa7, 0x7f, 0x34, 0xf0, 0xf9, 0xcf, 0x04, 0xfa, 0xa7, 0x16,
0x3a, 0x56, 0xdd, 0xf4, 0xf8, 0xbd, 0xab, 0xe8, 0x9f, 0x5b, 0xe8, 0x41, 0xb9, 0xd3, 0xe9, 0xf7,
0x5d, 0x4c, 0x56, 0xbe, 0xcd, 0xe9, 0x0f, 0x5c, 0x0c, 0xf3, 0xdc, 0x1e, 0xa7, 0x3f, 0x74, 0xcd,
0x75, 0x14, 0x1b, 0x9c, 0xfe, 0xa8, 0x02, 0x20, 0x17, 0xfd, 0xb1, 0x6b, 0x26, 0xd0, 0x99, 0xad,
0x4d, 0x7f, 0xe2, 0xa2, 0x6f, 0xe7, 0xf7, 0x35, 0xfd, 0xa9, 0x6b, 0xaf, 0xbb, 0xd8, 0xd4, 0xf4,
0x67, 0x2e, 0x76, 0xc0, 0xc5, 0x3b, 0x9a, 0xbe, 0x64, 0x6c, 0x95, 0xdb, 0x99, 0xbe, 0xec, 0x2e,
0xb7, 0x49, 0xa3, 0xab, 0x22, 0xb3, 0x37, 0x1a, 0xa4, 0xd6, 0x55, 0x11, 0x9d, 0xc0, 0x31, 0xbb,
0x26, 0x44, 0xb4, 0x7e, 0x9a, 0xca, 0xa7, 0x3e, 0x40, 0x9d, 0xe5, 0x35, 0x32, 0xd7, 0x11, 0x71,
0xca, 0x8b, 0x76, 0x33, 0xab, 0xc2, 0xee, 0x18, 0x08, 0x6c, 0xa9, 0x4c, 0xe0, 0xac, 0x5e, 0x3f,
0x05, 0x7f, 0x64, 0x36, 0x9a, 0x83, 0x47, 0x14, 0xc2, 0x24, 0x07, 0x74, 0x72, 0xf9, 0x19, 0x42,
0x3b, 0x22, 0x51, 0xa1, 0xd2, 0x90, 0xf8, 0xe3, 0x2d, 0x38, 0x81, 0xc8, 0xec, 0x4d, 0x2d, 0x45,
0x32, 0xa4, 0x13, 0xe6, 0x49, 0x00, 0xe6, 0xd3, 0xde, 0x6e, 0xd7, 0x35, 0xfc, 0xac, 0x33, 0xdf,
0xfd, 0xb3, 0x84, 0xac, 0x9f, 0x40, 0xa2, 0x47, 0x3c, 0x8a, 0xc6, 0xb4, 0x86, 0xe7, 0xce, 0x48,
0x69, 0x11, 0x87, 0x9f, 0x36, 0xfb, 0xfb, 0xab, 0x0e, 0x69, 0xd9, 0x55, 0x5a, 0xb8, 0x66, 0x8f,
0x7d, 0x48, 0x82, 0xd0, 0x28, 0xc7, 0xcf, 0x56, 0x03, 0x65, 0x4b, 0xdf, 0x29, 0x99, 0x06, 0x9a,
0x4b, 0x9d, 0xbf, 0x2f, 0x2c, 0xd4, 0x15, 0x77, 0x93, 0x48, 0xf0, 0xc0, 0xec, 0xf3, 0x42, 0xb4,
0xcf, 0xa5, 0x32, 0x4b, 0x1d, 0xbf, 0xea, 0x33, 0xfd, 0xd2, 0xc4, 0x13, 0xd0, 0x4b, 0x25, 0x58,
0xc6, 0x3c, 0x85, 0xcb, 0xd3, 0x82, 0xa6, 0xd8, 0xf3, 0x4a, 0x27, 0xcb, 0xf7, 0x13, 0xb7, 0xfa,
0x62, 0x33, 0x11, 0x95, 0x6b, 0x70, 0x62, 0xf9, 0x15, 0x07, 0x17, 0xcb, 0xf9, 0x27, 0x17, 0xad,
0xbc, 0x10, 0x57, 0xa3, 0xc8, 0xbe, 0x9c, 0x0a, 0xc4, 0x16, 0xa2, 0x8d, 0xac, 0x00, 0xb1, 0x18,
0x29, 0x0e, 0xf9, 0xd9, 0x8a, 0xa4, 0x06, 0x49, 0x6b, 0x67, 0xd8, 0x3c, 0xe0, 0x18, 0x58, 0x15,
0xc2, 0x7e, 0xb6, 0xef, 0xa5, 0x0a, 0x97, 0x59, 0x4b, 0x74, 0xea, 0x0c, 0x9a, 0x15, 0x06, 0x6d,
0x9c, 0xf1, 0x26, 0xdb, 0x17, 0xd3, 0x67, 0xc0, 0x6c, 0x6f, 0x34, 0x3f, 0x22, 0xc8, 0xe5, 0xe2,
0x29, 0x7a, 0x00, 0xa7, 0xfa, 0x40, 0x1c, 0x3e, 0xcb, 0x1e, 0x58, 0xb1, 0x7f, 0x25, 0xad, 0xe4,
0x7f, 0x25, 0xad, 0x6c, 0x83, 0x52, 0x7c, 0x08, 0xbb, 0xa9, 0x59, 0xac, 0x0b, 0x7f, 0x6d, 0x98,
0xb7, 0xf6, 0xc5, 0x2f, 0xe0, 0xea, 0xdb, 0xd9, 0x9b, 0x4b, 0x2b, 0xa7, 0xdd, 0xc3, 0x67, 0xd7,
0x9e, 0x26, 0xb3, 0xa1, 0xc8, 0xe5, 0x86, 0x32, 0xf5, 0xd7, 0x5a, 0x1d, 0x23, 0xd7, 0x47, 0x1d,
0x7d, 0xe7, 0xe3, 0x8f, 0x0f, 0x43, 0x7d, 0x3c, 0x3a, 0x44, 0x6d, 0xb7, 0x2c, 0xdb, 0xa3, 0xa1,
0xc8, 0x7e, 0xdd, 0x0a, 0x13, 0x8d, 0x4d, 0x1f, 0xd9, 0x3f, 0xb9, 0x6e, 0x59, 0x8b, 0xe9, 0xe1,
0x17, 0x1d, 0xe7, 0x70, 0xca, 0x40, 0x8f, 0xff, 0x27, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x59, 0x97,
0x8f, 0x2a, 0x13, 0x00, 0x00,
}

View File

@ -478,19 +478,19 @@ message ImportTaskState {
}
message ImportTaskInfo {
int64 id = 1; // Task ID.
int64 request_id = 2; // Request ID of the import task.
int64 datanode_id = 3; // ID of DataNode that processes the task.
int64 collection_id = 4; // Collection ID for the import task.
int64 partition_id = 5; // Partition ID for the import task.
repeated string channel_names = 6; // Names of channels for the collection.
string bucket = 7; // Bucket for the import task.
bool row_based = 8; // Boolean indicating whether import files are row-based or column-based.
repeated string files = 9; // A list of files to import.
int64 create_ts = 10; // Timestamp when the import task is created.
ImportTaskState state = 11; // State of the import task.
bool heuristic_data_queryable = 12; // A flag indicating (heuristically) whether import data are queryable (i.e. loaded in query nodes)
bool heuristic_data_indexed = 13; // A flag indicating (heuristically) whether import data are indexed.
int64 id = 1; // Task ID.
int64 request_id = 2; // Request ID of the import task.
int64 datanode_id = 3; // ID of DataNode that processes the task.
int64 collection_id = 4; // Collection ID for the import task.
int64 partition_id = 5; // Partition ID for the import task.
repeated string channel_names = 6; // Names of channels for the collection.
string bucket = 7; // Bucket for the import task.
bool row_based = 8; // Boolean indicating whether import files are row-based or column-based.
repeated string files = 9; // A list of files to import.
int64 create_ts = 10; // Timestamp when the import task is created.
ImportTaskState state = 11; // State of the import task.
bool data_queryable = 12; // A flag indicating whether import data are queryable (i.e. loaded in query nodes)
bool data_indexed = 13; // A flag indicating whether import data are indexed.
}
message ImportTaskResponse {

View File

@ -3556,22 +3556,22 @@ func (m *ImportTaskState) GetErrorMessage() string {
}
type ImportTaskInfo struct {
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
RequestId int64 `protobuf:"varint,2,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"`
DatanodeId int64 `protobuf:"varint,3,opt,name=datanode_id,json=datanodeId,proto3" json:"datanode_id,omitempty"`
CollectionId int64 `protobuf:"varint,4,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
PartitionId int64 `protobuf:"varint,5,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
ChannelNames []string `protobuf:"bytes,6,rep,name=channel_names,json=channelNames,proto3" json:"channel_names,omitempty"`
Bucket string `protobuf:"bytes,7,opt,name=bucket,proto3" json:"bucket,omitempty"`
RowBased bool `protobuf:"varint,8,opt,name=row_based,json=rowBased,proto3" json:"row_based,omitempty"`
Files []string `protobuf:"bytes,9,rep,name=files,proto3" json:"files,omitempty"`
CreateTs int64 `protobuf:"varint,10,opt,name=create_ts,json=createTs,proto3" json:"create_ts,omitempty"`
State *ImportTaskState `protobuf:"bytes,11,opt,name=state,proto3" json:"state,omitempty"`
HeuristicDataQueryable bool `protobuf:"varint,12,opt,name=heuristic_data_queryable,json=heuristicDataQueryable,proto3" json:"heuristic_data_queryable,omitempty"`
HeuristicDataIndexed bool `protobuf:"varint,13,opt,name=heuristic_data_indexed,json=heuristicDataIndexed,proto3" json:"heuristic_data_indexed,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
RequestId int64 `protobuf:"varint,2,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"`
DatanodeId int64 `protobuf:"varint,3,opt,name=datanode_id,json=datanodeId,proto3" json:"datanode_id,omitempty"`
CollectionId int64 `protobuf:"varint,4,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
PartitionId int64 `protobuf:"varint,5,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
ChannelNames []string `protobuf:"bytes,6,rep,name=channel_names,json=channelNames,proto3" json:"channel_names,omitempty"`
Bucket string `protobuf:"bytes,7,opt,name=bucket,proto3" json:"bucket,omitempty"`
RowBased bool `protobuf:"varint,8,opt,name=row_based,json=rowBased,proto3" json:"row_based,omitempty"`
Files []string `protobuf:"bytes,9,rep,name=files,proto3" json:"files,omitempty"`
CreateTs int64 `protobuf:"varint,10,opt,name=create_ts,json=createTs,proto3" json:"create_ts,omitempty"`
State *ImportTaskState `protobuf:"bytes,11,opt,name=state,proto3" json:"state,omitempty"`
DataQueryable bool `protobuf:"varint,12,opt,name=data_queryable,json=dataQueryable,proto3" json:"data_queryable,omitempty"`
DataIndexed bool `protobuf:"varint,13,opt,name=data_indexed,json=dataIndexed,proto3" json:"data_indexed,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ImportTaskInfo) Reset() { *m = ImportTaskInfo{} }
@ -3676,16 +3676,16 @@ func (m *ImportTaskInfo) GetState() *ImportTaskState {
return nil
}
func (m *ImportTaskInfo) GetHeuristicDataQueryable() bool {
func (m *ImportTaskInfo) GetDataQueryable() bool {
if m != nil {
return m.HeuristicDataQueryable
return m.DataQueryable
}
return false
}
func (m *ImportTaskInfo) GetHeuristicDataIndexed() bool {
func (m *ImportTaskInfo) GetDataIndexed() bool {
if m != nil {
return m.HeuristicDataIndexed
return m.DataIndexed
}
return false
}
@ -4076,235 +4076,234 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 3640 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3b, 0x4b, 0x6f, 0x1b, 0xd7,
0xd5, 0x1e, 0xbe, 0x44, 0x1e, 0x3e, 0x44, 0x5d, 0x3b, 0x32, 0x4d, 0xdb, 0xb2, 0x3c, 0x8e, 0x1d,
0xc5, 0x71, 0xec, 0x44, 0x4e, 0x10, 0xe3, 0xcb, 0x0b, 0xb6, 0x65, 0x2b, 0xc4, 0x27, 0xf9, 0x93,
0x47, 0x4a, 0xfc, 0xa1, 0x29, 0x4a, 0x8c, 0x38, 0x57, 0xd4, 0x44, 0x9c, 0x19, 0x7a, 0x66, 0x68,
0x59, 0xd9, 0xc4, 0x68, 0xd0, 0x02, 0x29, 0x8a, 0xb6, 0x40, 0x37, 0x2d, 0xd0, 0x45, 0xd1, 0x55,
0x1f, 0x28, 0x50, 0x20, 0xe8, 0xa2, 0x2d, 0xba, 0x0f, 0xda, 0x45, 0x7f, 0x40, 0x17, 0x5d, 0xb6,
0x9b, 0xf6, 0x37, 0x14, 0xf7, 0x31, 0x77, 0xde, 0xe4, 0x48, 0xb4, 0xe3, 0x1d, 0xef, 0xbd, 0xe7,
0x9c, 0x7b, 0xee, 0x3d, 0xef, 0x33, 0x97, 0xd0, 0xd4, 0x54, 0x57, 0xed, 0xf6, 0x2c, 0xcb, 0xd6,
0xae, 0x0e, 0x6d, 0xcb, 0xb5, 0xd0, 0x9c, 0xa1, 0x0f, 0x1e, 0x8d, 0x1c, 0x36, 0xba, 0x4a, 0x96,
0xdb, 0xb5, 0x9e, 0x65, 0x18, 0x96, 0xc9, 0xa6, 0xda, 0x0d, 0xdd, 0x74, 0xb1, 0x6d, 0xaa, 0x03,
0x3e, 0xae, 0x05, 0x11, 0xda, 0x35, 0xa7, 0xb7, 0x8b, 0x0d, 0x95, 0x8d, 0xe4, 0x19, 0x28, 0xde,
0x31, 0x86, 0xee, 0x81, 0xfc, 0x13, 0x09, 0x6a, 0x77, 0x07, 0x23, 0x67, 0x57, 0xc1, 0x0f, 0x47,
0xd8, 0x71, 0xd1, 0x6b, 0x50, 0xd8, 0x56, 0x1d, 0xdc, 0x92, 0x16, 0xa5, 0xa5, 0xea, 0xf2, 0x99,
0xab, 0xa1, 0x5d, 0xf9, 0x7e, 0xeb, 0x4e, 0xff, 0x96, 0xea, 0x60, 0x85, 0x42, 0x22, 0x04, 0x05,
0x6d, 0xbb, 0xb3, 0xd2, 0xca, 0x2d, 0x4a, 0x4b, 0x79, 0x85, 0xfe, 0x46, 0x0b, 0x00, 0x0e, 0xee,
0x1b, 0xd8, 0x74, 0x3b, 0x2b, 0x4e, 0x2b, 0xbf, 0x98, 0x5f, 0xca, 0x2b, 0x81, 0x19, 0x24, 0x43,
0xad, 0x67, 0x0d, 0x06, 0xb8, 0xe7, 0xea, 0x96, 0xd9, 0x59, 0x69, 0x15, 0x28, 0x6e, 0x68, 0x4e,
0xfe, 0x99, 0x04, 0x75, 0xce, 0x9a, 0x33, 0xb4, 0x4c, 0x07, 0xa3, 0xeb, 0x50, 0x72, 0x5c, 0xd5,
0x1d, 0x39, 0x9c, 0xbb, 0xd3, 0x89, 0xdc, 0x6d, 0x52, 0x10, 0x85, 0x83, 0x26, 0xb2, 0x17, 0xdd,
0x3e, 0x1f, 0xdf, 0x3e, 0x72, 0x84, 0x42, 0xf4, 0x08, 0xf2, 0x6f, 0x25, 0x68, 0x6e, 0x7a, 0x43,
0xef, 0xf6, 0x4e, 0x40, 0xb1, 0x67, 0x8d, 0x4c, 0x97, 0x32, 0x58, 0x57, 0xd8, 0x00, 0x9d, 0x87,
0x5a, 0x6f, 0x57, 0x35, 0x4d, 0x3c, 0xe8, 0x9a, 0xaa, 0x81, 0x29, 0x2b, 0x15, 0xa5, 0xca, 0xe7,
0xee, 0xa9, 0x06, 0xce, 0xc4, 0xd1, 0x22, 0x54, 0x87, 0xaa, 0xed, 0xea, 0xa1, 0x3b, 0x0b, 0x4e,
0xa1, 0x36, 0x94, 0x75, 0xa7, 0x63, 0x0c, 0x2d, 0xdb, 0x6d, 0x15, 0x17, 0xa5, 0xa5, 0xb2, 0x22,
0xc6, 0xf2, 0xcf, 0x25, 0x98, 0xbf, 0xe9, 0x38, 0x7a, 0xdf, 0x8c, 0x71, 0x3d, 0x0f, 0x25, 0xd3,
0xd2, 0x70, 0x67, 0x85, 0xb2, 0x9d, 0x57, 0xf8, 0x08, 0x9d, 0x86, 0xca, 0x10, 0x63, 0xbb, 0x6b,
0x5b, 0x03, 0x8f, 0xe9, 0x32, 0x99, 0x50, 0xac, 0x01, 0x46, 0xf7, 0x61, 0xce, 0x89, 0x10, 0x62,
0x92, 0xae, 0x2e, 0x5f, 0xb8, 0x1a, 0xd3, 0xd5, 0xab, 0xd1, 0x4d, 0x95, 0x38, 0xb6, 0xfc, 0x24,
0x07, 0xc7, 0x05, 0x1c, 0xe3, 0x95, 0xfc, 0x26, 0xb7, 0xea, 0xe0, 0xbe, 0x60, 0x8f, 0x0d, 0xb2,
0xdc, 0xaa, 0x10, 0x47, 0x3e, 0x28, 0x8e, 0x0c, 0xca, 0x17, 0xbd, 0xeb, 0x62, 0xfc, 0xae, 0xcf,
0x41, 0x15, 0x3f, 0x1e, 0xea, 0x36, 0xee, 0xba, 0xba, 0x81, 0x5b, 0xa5, 0x45, 0x69, 0xa9, 0xa0,
0x00, 0x9b, 0xda, 0xd2, 0x8d, 0xa0, 0xb6, 0xce, 0x64, 0xd6, 0x56, 0xf9, 0x17, 0x12, 0x9c, 0x8c,
0x49, 0x89, 0xab, 0xbf, 0x02, 0x4d, 0x7a, 0x72, 0xff, 0x66, 0x88, 0x21, 0x90, 0x0b, 0xbf, 0x34,
0xee, 0xc2, 0x7d, 0x70, 0x25, 0x86, 0x1f, 0x60, 0x32, 0x97, 0x9d, 0xc9, 0x3d, 0x38, 0xb9, 0x8a,
0x5d, 0xbe, 0x01, 0x59, 0xc3, 0xce, 0xd1, 0xdd, 0x47, 0xd8, 0xce, 0x72, 0x31, 0x3b, 0xfb, 0x5d,
0x4e, 0xd8, 0x19, 0xdd, 0xaa, 0x63, 0xee, 0x58, 0xe8, 0x0c, 0x54, 0x04, 0x08, 0xd7, 0x0a, 0x7f,
0x02, 0xbd, 0x05, 0x45, 0xc2, 0x29, 0x53, 0x89, 0xc6, 0xf2, 0xf9, 0xe4, 0x33, 0x05, 0x68, 0x2a,
0x0c, 0x1e, 0x75, 0xa0, 0xe1, 0xb8, 0xaa, 0xed, 0x76, 0x87, 0x96, 0x43, 0xe5, 0x4c, 0x15, 0xa7,
0xba, 0x2c, 0x87, 0x29, 0x08, 0x47, 0xbb, 0xee, 0xf4, 0x37, 0x38, 0xa4, 0x52, 0xa7, 0x98, 0xde,
0x10, 0xdd, 0x81, 0x1a, 0x36, 0x35, 0x9f, 0x50, 0x21, 0x33, 0xa1, 0x2a, 0x36, 0x35, 0x41, 0xc6,
0x97, 0x4f, 0x31, 0xbb, 0x7c, 0xbe, 0x2f, 0x41, 0x2b, 0x2e, 0xa0, 0x69, 0x9c, 0xe8, 0xdb, 0x0c,
0x09, 0x33, 0x01, 0x8d, 0xb5, 0x70, 0x21, 0x24, 0x85, 0xa3, 0xc8, 0x3a, 0xbc, 0xe0, 0x73, 0x43,
0x57, 0x9e, 0x99, 0xb2, 0x7c, 0x2e, 0xc1, 0x7c, 0x74, 0xaf, 0x69, 0xce, 0xfd, 0x06, 0x14, 0x75,
0x73, 0xc7, 0xf2, 0x8e, 0xbd, 0x30, 0xc6, 0xce, 0xc8, 0x5e, 0x0c, 0x58, 0x36, 0xe0, 0xf4, 0x2a,
0x76, 0x3b, 0xa6, 0x83, 0x6d, 0xf7, 0x96, 0x6e, 0x0e, 0xac, 0xfe, 0x86, 0xea, 0xee, 0x4e, 0x61,
0x23, 0x21, 0x75, 0xcf, 0x45, 0xd4, 0x5d, 0xfe, 0xa5, 0x04, 0x67, 0x92, 0xf7, 0xe3, 0x47, 0x6f,
0x43, 0x79, 0x47, 0xc7, 0x03, 0x8d, 0xdc, 0x99, 0x44, 0xef, 0x4c, 0x8c, 0x89, 0xad, 0x0c, 0x09,
0x30, 0x3f, 0xe1, 0xf9, 0x14, 0x05, 0xdd, 0x74, 0x6d, 0xdd, 0xec, 0xaf, 0xe9, 0x8e, 0xab, 0x30,
0xf8, 0xc0, 0x7d, 0xe6, 0xb3, 0x6b, 0xe6, 0xf7, 0x24, 0x58, 0x58, 0xc5, 0xee, 0x6d, 0xe1, 0x6a,
0xc9, 0xba, 0xee, 0xb8, 0x7a, 0xcf, 0x79, 0xba, 0x09, 0x48, 0x86, 0x78, 0x2a, 0xff, 0x50, 0x82,
0x73, 0xa9, 0xcc, 0xf0, 0xab, 0xe3, 0xae, 0xc4, 0x73, 0xb4, 0xc9, 0xae, 0xe4, 0x7f, 0xf1, 0xc1,
0x47, 0xea, 0x60, 0x84, 0x37, 0x54, 0xdd, 0x66, 0xae, 0xe4, 0x88, 0x8e, 0xf5, 0x37, 0x12, 0x9c,
0x5d, 0xc5, 0xee, 0x86, 0x17, 0x66, 0x9e, 0xe3, 0xed, 0x4c, 0xce, 0x36, 0xe4, 0x1f, 0x30, 0x61,
0x26, 0x72, 0xfb, 0x5c, 0xae, 0x6f, 0x81, 0xda, 0x41, 0xc0, 0x20, 0x6f, 0xb3, 0x5c, 0x80, 0x5f,
0x9e, 0xfc, 0x1d, 0x09, 0x4e, 0xdd, 0xec, 0x3d, 0x1c, 0xe9, 0x36, 0xe6, 0x40, 0x6b, 0x56, 0x6f,
0xef, 0xe8, 0x57, 0xeb, 0xe7, 0x4d, 0xb9, 0x50, 0xde, 0x34, 0x21, 0xfb, 0xa5, 0x7c, 0x28, 0x78,
0x80, 0x55, 0xe7, 0xf9, 0xf2, 0xf1, 0x24, 0x0f, 0xb5, 0x8f, 0x78, 0xbe, 0x44, 0xc3, 0x6a, 0x54,
0x2f, 0xa4, 0x64, 0xbd, 0x08, 0xa4, 0x58, 0x49, 0x59, 0xd7, 0x2a, 0xd4, 0x1d, 0x8c, 0xf7, 0x8e,
0x12, 0x44, 0x6b, 0x04, 0x51, 0x04, 0xbf, 0x35, 0x98, 0x1b, 0x99, 0x3b, 0xa4, 0x04, 0xc0, 0x1a,
0xbf, 0x28, 0x96, 0x89, 0x4f, 0xf6, 0xc4, 0x71, 0x44, 0xf4, 0x01, 0xcc, 0x46, 0x69, 0x15, 0x33,
0xd1, 0x8a, 0xa2, 0xa1, 0x0e, 0x34, 0x35, 0xdb, 0x1a, 0x0e, 0xb1, 0xd6, 0x75, 0x3c, 0x52, 0xa5,
0x6c, 0xa4, 0x38, 0x9e, 0x47, 0x4a, 0xfe, 0x42, 0x82, 0xf9, 0x07, 0xaa, 0xdb, 0xdb, 0x5d, 0x31,
0xb8, 0xb2, 0x4e, 0x61, 0xea, 0xef, 0x42, 0xe5, 0x11, 0x17, 0x84, 0xe7, 0xcf, 0xcf, 0x25, 0x30,
0x14, 0x14, 0xb9, 0xe2, 0x63, 0xc8, 0x5f, 0x49, 0x70, 0x82, 0x16, 0x5c, 0x1e, 0x77, 0x5f, 0xbf,
0xd3, 0x99, 0x50, 0x74, 0xa1, 0x4b, 0xd0, 0x30, 0x54, 0x7b, 0x6f, 0xd3, 0x87, 0x29, 0x52, 0x98,
0xc8, 0xac, 0xfc, 0x18, 0x80, 0x8f, 0xd6, 0x9d, 0xfe, 0x11, 0xf8, 0xbf, 0x01, 0x33, 0x7c, 0x57,
0xee, 0x7f, 0x26, 0x09, 0xd6, 0x03, 0x97, 0xff, 0x22, 0x41, 0xc3, 0x8f, 0x28, 0xd4, 0xaa, 0x1a,
0x90, 0x13, 0xb6, 0x94, 0xeb, 0xac, 0xa0, 0x77, 0xa1, 0xc4, 0x8a, 0x71, 0x4e, 0xfb, 0x62, 0x98,
0x36, 0x2f, 0xd4, 0x03, 0x61, 0x89, 0x4e, 0x28, 0x1c, 0x89, 0xdc, 0x91, 0xf0, 0xc2, 0xc2, 0xaa,
0xfd, 0x19, 0xd4, 0x81, 0xd9, 0x70, 0x12, 0xeb, 0xd9, 0xcc, 0x62, 0x9a, 0xf7, 0x5d, 0x51, 0x5d,
0x95, 0x3a, 0xdf, 0x46, 0x28, 0x87, 0x75, 0xe4, 0xff, 0x14, 0xa1, 0x1a, 0x38, 0x65, 0xec, 0x24,
0x51, 0x91, 0xe6, 0x26, 0xc7, 0x91, 0x7c, 0xbc, 0x92, 0xba, 0x08, 0x0d, 0x9d, 0xe6, 0x2e, 0x5d,
0xae, 0x8a, 0x34, 0xd8, 0x54, 0x94, 0x3a, 0x9b, 0xe5, 0x76, 0x81, 0x16, 0xa0, 0x6a, 0x8e, 0x8c,
0xae, 0xb5, 0xd3, 0xb5, 0xad, 0x7d, 0x87, 0x97, 0x64, 0x15, 0x73, 0x64, 0xfc, 0xdf, 0x8e, 0x62,
0xed, 0x3b, 0x7e, 0xd6, 0x5f, 0x3a, 0x64, 0xd6, 0xbf, 0x00, 0x55, 0x43, 0x7d, 0x4c, 0xa8, 0x76,
0xcd, 0x91, 0x41, 0xab, 0xb5, 0xbc, 0x52, 0x31, 0xd4, 0xc7, 0x8a, 0xb5, 0x7f, 0x6f, 0x64, 0xa0,
0x25, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0xcb, 0xbd, 0x32, 0x2d, 0xf7, 0x1a, 0x64, 0xfe, 0x8e,
0x5f, 0xf2, 0xc5, 0xeb, 0x87, 0xca, 0x14, 0xf5, 0x83, 0x66, 0x0c, 0x7c, 0x42, 0x90, 0xbd, 0x7e,
0xd0, 0x8c, 0x81, 0x20, 0x73, 0x03, 0x66, 0xb6, 0x69, 0x46, 0xe8, 0xb4, 0xaa, 0xa9, 0x1e, 0xea,
0x2e, 0x49, 0x06, 0x59, 0xe2, 0xa8, 0x78, 0xe0, 0xe8, 0x1d, 0xa8, 0xd0, 0x50, 0x4c, 0x71, 0x6b,
0x99, 0x70, 0x7d, 0x04, 0x82, 0xad, 0xe1, 0x81, 0xab, 0x52, 0xec, 0x7a, 0x36, 0x6c, 0x81, 0x80,
0x5e, 0x83, 0xe3, 0x3d, 0x1b, 0xab, 0x2e, 0xd6, 0x6e, 0x1d, 0xdc, 0xb6, 0x8c, 0xa1, 0x4a, 0x95,
0xa9, 0xd5, 0xa0, 0x2d, 0x8d, 0xa4, 0x25, 0xe2, 0x18, 0x7a, 0x62, 0x74, 0xd7, 0xb6, 0x8c, 0xd6,
0x2c, 0x73, 0x0c, 0xe1, 0x59, 0x74, 0x16, 0xc0, 0x73, 0xdd, 0xaa, 0xdb, 0x6a, 0x52, 0x29, 0x56,
0xf8, 0xcc, 0x4d, 0x57, 0xfe, 0x0c, 0x4e, 0xf8, 0x1a, 0x12, 0x90, 0x46, 0x5c, 0xb0, 0xd2, 0x51,
0x05, 0x3b, 0x3e, 0x97, 0xff, 0x5b, 0x01, 0xe6, 0x37, 0xd5, 0x47, 0xf8, 0xd9, 0x97, 0x0d, 0x99,
0xfc, 0xf1, 0x1a, 0xcc, 0xd1, 0x4a, 0x61, 0x39, 0xc0, 0xcf, 0x98, 0x08, 0x1c, 0x14, 0x67, 0x1c,
0x11, 0xbd, 0x4f, 0x52, 0x07, 0xdc, 0xdb, 0xdb, 0xb0, 0x74, 0x3f, 0xfa, 0x9e, 0x4d, 0xa0, 0x73,
0x5b, 0x40, 0x29, 0x41, 0x0c, 0xb4, 0x11, 0x77, 0x6d, 0x2c, 0xee, 0xbe, 0x34, 0xb6, 0x1e, 0xf5,
0x6f, 0x3f, 0xea, 0xe1, 0x50, 0x0b, 0x66, 0x78, 0x74, 0xa7, 0x76, 0x5f, 0x56, 0xbc, 0x21, 0xda,
0x80, 0xe3, 0xec, 0x04, 0x9b, 0x5c, 0xa9, 0xd9, 0xe1, 0xcb, 0x99, 0x0e, 0x9f, 0x84, 0x1a, 0xb6,
0x89, 0xca, 0x61, 0x6d, 0xa2, 0x05, 0x33, 0x5c, 0x4f, 0xa9, 0x2f, 0x28, 0x2b, 0xde, 0x90, 0x88,
0x59, 0xa7, 0x3d, 0x3e, 0xdd, 0xec, 0xb7, 0xaa, 0x74, 0xcd, 0x9f, 0x20, 0x25, 0x17, 0xf8, 0xf7,
0x39, 0xa1, 0x73, 0xf2, 0x1e, 0x94, 0x85, 0x86, 0xe7, 0x32, 0x6b, 0xb8, 0xc0, 0x89, 0xfa, 0xe8,
0x7c, 0xc4, 0x47, 0xcb, 0x7f, 0x95, 0xa0, 0xb6, 0x42, 0x8e, 0xb4, 0x66, 0xf5, 0x69, 0x44, 0xb9,
0x08, 0x0d, 0x1b, 0xf7, 0x2c, 0x5b, 0xeb, 0x62, 0xd3, 0xb5, 0x75, 0xcc, 0xaa, 0xf3, 0x82, 0x52,
0x67, 0xb3, 0x77, 0xd8, 0x24, 0x01, 0x23, 0x6e, 0xd7, 0x71, 0x55, 0x63, 0xd8, 0xdd, 0x21, 0xe6,
0x9d, 0x63, 0x60, 0x62, 0x96, 0x5a, 0xf7, 0x79, 0xa8, 0xf9, 0x60, 0xae, 0x45, 0xf7, 0x2f, 0x28,
0x55, 0x31, 0xb7, 0x65, 0xa1, 0x17, 0xa1, 0x41, 0xef, 0xb4, 0x3b, 0xb0, 0xfa, 0x5d, 0x52, 0xc9,
0xf2, 0x60, 0x53, 0xd3, 0x38, 0x5b, 0x44, 0x56, 0x61, 0x28, 0x47, 0xff, 0x14, 0xf3, 0x70, 0x23,
0xa0, 0x36, 0xf5, 0x4f, 0x31, 0x89, 0xf5, 0x75, 0x12, 0x3b, 0xef, 0x59, 0x1a, 0xde, 0x3a, 0x62,
0xa6, 0x91, 0xa1, 0x8b, 0x79, 0x06, 0x2a, 0xe2, 0x04, 0xfc, 0x48, 0xfe, 0x04, 0xba, 0x0b, 0x0d,
0x2f, 0x09, 0xed, 0xb2, 0x5a, 0xab, 0x90, 0x9a, 0xf9, 0x05, 0xa2, 0x9f, 0xa3, 0xd4, 0x3d, 0x34,
0x3a, 0x94, 0xef, 0x42, 0x2d, 0xb8, 0x4c, 0x76, 0xdd, 0x8c, 0x2a, 0x8a, 0x98, 0x20, 0xda, 0x78,
0x6f, 0x64, 0x10, 0x99, 0x72, 0xc7, 0xe2, 0x0d, 0xe5, 0xcf, 0x25, 0xa8, 0xf3, 0x90, 0xbd, 0x29,
0x3a, 0xf0, 0xf4, 0x68, 0x12, 0x3d, 0x1a, 0xfd, 0x8d, 0xfe, 0x27, 0xdc, 0xa2, 0x7b, 0x31, 0xd1,
0x09, 0x50, 0x22, 0x34, 0x3b, 0x0e, 0xc5, 0xeb, 0x2c, 0xb5, 0xfd, 0x13, 0xa2, 0x68, 0x5c, 0x34,
0x54, 0xd1, 0x5a, 0x30, 0xa3, 0x6a, 0x9a, 0x8d, 0x1d, 0x87, 0xf3, 0xe1, 0x0d, 0xc9, 0xca, 0x23,
0x6c, 0x3b, 0x9e, 0xca, 0xe7, 0x15, 0x6f, 0x88, 0xde, 0x81, 0xb2, 0x48, 0xa7, 0xf3, 0x49, 0x29,
0x54, 0x90, 0x4f, 0x5e, 0x8b, 0x0a, 0x0c, 0xf9, 0xf7, 0x39, 0x68, 0xf0, 0x0b, 0xbb, 0xc5, 0x63,
0xea, 0x78, 0xe3, 0xbb, 0x05, 0xb5, 0x1d, 0xdf, 0xf6, 0xc7, 0xf5, 0x9c, 0x82, 0x2e, 0x22, 0x84,
0x33, 0xc9, 0x00, 0xc3, 0x51, 0xbd, 0x30, 0x55, 0x54, 0x2f, 0x1e, 0xd6, 0x83, 0xc5, 0xf3, 0xbc,
0x52, 0x42, 0x9e, 0x27, 0x7f, 0x13, 0xaa, 0x01, 0x02, 0xd4, 0x43, 0xb3, 0x66, 0x15, 0xbf, 0x31,
0x6f, 0x88, 0xae, 0xfb, 0xb9, 0x0d, 0xbb, 0xaa, 0x53, 0x09, 0xbc, 0x44, 0xd2, 0x1a, 0xf9, 0x57,
0x12, 0x94, 0x38, 0xe5, 0x73, 0x50, 0xe5, 0x4e, 0x87, 0xe6, 0x7d, 0x8c, 0x3a, 0xf0, 0x29, 0x92,
0xf8, 0x3d, 0x3d, 0xaf, 0x73, 0x0a, 0xca, 0x11, 0x7f, 0x33, 0xc3, 0xc3, 0x82, 0xb7, 0x14, 0x70,
0x32, 0x64, 0x89, 0xfa, 0x97, 0xaf, 0x24, 0xda, 0x68, 0x57, 0x70, 0xcf, 0x7a, 0x84, 0xed, 0x83,
0xe9, 0xdb, 0x99, 0x6f, 0x07, 0x14, 0x3a, 0x63, 0x7d, 0x28, 0x10, 0xd0, 0xdb, 0xfe, 0x75, 0xe7,
0x93, 0xba, 0x39, 0x41, 0x0f, 0xc3, 0xd5, 0xd1, 0xbf, 0xf6, 0x1f, 0xb1, 0xc6, 0x6c, 0xf8, 0x28,
0x47, 0xcd, 0x6b, 0x9e, 0x4a, 0xd9, 0x21, 0xff, 0x58, 0x82, 0x53, 0xab, 0xd8, 0xbd, 0x1b, 0x2e,
0xee, 0x9f, 0x37, 0x57, 0x06, 0xb4, 0x93, 0x98, 0x9a, 0x46, 0xea, 0x6d, 0x28, 0x8b, 0x36, 0x05,
0x6b, 0x99, 0x8b, 0xb1, 0xfc, 0x5d, 0x09, 0x5a, 0x7c, 0x17, 0xba, 0x27, 0x49, 0xa9, 0x07, 0xd8,
0xc5, 0xda, 0xd7, 0x5d, 0x37, 0xff, 0x59, 0x82, 0x66, 0xd0, 0xe3, 0x53, 0xa7, 0xfd, 0x26, 0x14,
0x69, 0x7b, 0x82, 0x73, 0x30, 0x51, 0x59, 0x19, 0x34, 0x71, 0x19, 0x34, 0xcd, 0xdb, 0x12, 0xc1,
0x89, 0x0f, 0xfd, 0xb0, 0x93, 0x3f, 0x7c, 0xd8, 0xe1, 0x61, 0xd8, 0x1a, 0x11, 0xba, 0xac, 0x1d,
0xea, 0x4f, 0xc8, 0x5f, 0xe6, 0xa0, 0xe5, 0xd7, 0x23, 0x5f, 0xbb, 0xdf, 0x4f, 0xc9, 0x56, 0xf3,
0x4f, 0x29, 0x5b, 0x2d, 0x4c, 0xef, 0xeb, 0x8b, 0x49, 0xbe, 0xfe, 0x4f, 0x39, 0x68, 0xf8, 0xb7,
0xb6, 0x31, 0x50, 0x4d, 0x34, 0x0f, 0xa5, 0xe1, 0x40, 0xf5, 0xbb, 0x8f, 0x7c, 0x84, 0x36, 0x45,
0x9e, 0x13, 0xbe, 0xa7, 0x57, 0x92, 0x64, 0x98, 0x22, 0x08, 0x25, 0x42, 0x82, 0x94, 0x83, 0xac,
0xa0, 0xa0, 0x45, 0x3d, 0xcf, 0xad, 0x98, 0xb2, 0x90, 0x7a, 0xfe, 0x0a, 0x20, 0x2e, 0xe1, 0xae,
0x6e, 0x76, 0x1d, 0xdc, 0xb3, 0x4c, 0x8d, 0xc9, 0xbe, 0xa8, 0x34, 0xf9, 0x4a, 0xc7, 0xdc, 0x64,
0xf3, 0xe8, 0x4d, 0x28, 0xb8, 0x07, 0x43, 0xe6, 0xc5, 0x1b, 0x89, 0xde, 0xd1, 0xe7, 0x6b, 0xeb,
0x60, 0x88, 0x15, 0x0a, 0x8e, 0x16, 0x00, 0x08, 0x29, 0xd7, 0x56, 0x1f, 0xf1, 0x90, 0x58, 0x50,
0x02, 0x33, 0x44, 0x9b, 0xbd, 0x3b, 0x9c, 0x61, 0xa1, 0x83, 0x0f, 0xe5, 0x3f, 0xe4, 0xa0, 0xe9,
0x93, 0x54, 0xb0, 0x33, 0x1a, 0xb8, 0xa9, 0xf7, 0x37, 0xbe, 0x18, 0x9c, 0x94, 0x37, 0xbc, 0x0f,
0x55, 0x2e, 0xcf, 0x43, 0xe8, 0x03, 0x30, 0x94, 0xb5, 0x31, 0x0a, 0x5a, 0x7c, 0x4a, 0x0a, 0x5a,
0x3a, 0xa4, 0x82, 0xca, 0x9b, 0x30, 0xef, 0xf9, 0x3d, 0x1f, 0x60, 0x1d, 0xbb, 0xea, 0x98, 0x84,
0xe3, 0x1c, 0x54, 0x59, 0x3c, 0x63, 0x81, 0x9c, 0xa5, 0xea, 0xb0, 0x2d, 0x2a, 0x5c, 0xf9, 0x5b,
0x70, 0x82, 0xfa, 0x8d, 0x68, 0x2b, 0x37, 0x4b, 0x5f, 0x5d, 0x16, 0x85, 0x00, 0x49, 0xfa, 0x99,
0x76, 0x57, 0x94, 0xd0, 0x9c, 0xbc, 0x06, 0x2f, 0x44, 0xe8, 0x4f, 0x11, 0x17, 0x48, 0x2a, 0x34,
0xbf, 0x19, 0xfe, 0x4c, 0x7c, 0xf4, 0xe8, 0x77, 0x56, 0x74, 0x6e, 0xbb, 0xba, 0x16, 0xd5, 0x2f,
0x0d, 0xbd, 0x07, 0x15, 0x13, 0xef, 0x77, 0x83, 0xce, 0x37, 0x43, 0x83, 0xae, 0x6c, 0xe2, 0x7d,
0xfa, 0x4b, 0xbe, 0x07, 0x27, 0x63, 0xac, 0x4e, 0x73, 0xf6, 0x3f, 0x4a, 0x70, 0x6a, 0xc5, 0xb6,
0x86, 0x1f, 0xe9, 0xb6, 0x3b, 0x52, 0x07, 0xe1, 0x0f, 0x45, 0xcf, 0xa6, 0x8c, 0xfb, 0x20, 0x10,
0x86, 0x99, 0x5f, 0xbe, 0x92, 0xa0, 0xae, 0x71, 0xa6, 0xf8, 0xa1, 0x03, 0x41, 0xfb, 0x9f, 0xf9,
0x24, 0xe6, 0x39, 0xdc, 0x84, 0x60, 0x93, 0x25, 0x4b, 0x49, 0xec, 0xfa, 0xe4, 0x8f, 0xda, 0xf5,
0x49, 0xb1, 0xfc, 0xc2, 0x53, 0xb2, 0xfc, 0x43, 0x97, 0x21, 0x1f, 0x40, 0xb8, 0x23, 0x47, 0x5d,
0xee, 0x91, 0x5a, 0x79, 0xb7, 0x00, 0xfc, 0xee, 0x14, 0x7f, 0xe5, 0x93, 0x85, 0x4c, 0x00, 0x8b,
0x48, 0x4b, 0x78, 0x59, 0xda, 0x55, 0x0e, 0xf5, 0x4b, 0xee, 0x43, 0x3b, 0x49, 0x4b, 0xa7, 0xd1,
0xfc, 0x2f, 0x73, 0x00, 0xec, 0x49, 0xd8, 0x96, 0xea, 0xec, 0x1d, 0x2d, 0xa3, 0xbc, 0x00, 0x75,
0x5f, 0x61, 0x7c, 0x7b, 0x0f, 0x6a, 0x91, 0x46, 0x4c, 0x42, 0x24, 0xb6, 0x04, 0x26, 0x96, 0xec,
0x6a, 0x94, 0x4e, 0xc0, 0x6a, 0x98, 0x52, 0x44, 0x9c, 0x1e, 0x3a, 0x0d, 0x15, 0xdb, 0xda, 0xef,
0x12, 0x33, 0xd3, 0xbc, 0x57, 0x6d, 0xb6, 0xb5, 0x4f, 0x8c, 0x4f, 0x43, 0x27, 0x61, 0xc6, 0x55,
0x9d, 0x3d, 0x42, 0xbf, 0xc4, 0xc2, 0x1d, 0x19, 0x76, 0x34, 0x74, 0x02, 0x8a, 0x3b, 0xfa, 0x00,
0x3b, 0xad, 0x19, 0x4a, 0x92, 0x0d, 0xd0, 0x5b, 0xde, 0x7b, 0x8e, 0x72, 0xe6, 0xef, 0xd1, 0xec,
0x49, 0xc7, 0x57, 0x12, 0xcc, 0xfa, 0xb7, 0x46, 0x1d, 0x10, 0xf1, 0x69, 0xd4, 0x9f, 0xdd, 0xb6,
0x34, 0xe6, 0x2a, 0x1a, 0x29, 0x9f, 0x58, 0x18, 0x22, 0xf3, 0x5a, 0x3e, 0xca, 0xb8, 0xbc, 0x9c,
0x9c, 0x8b, 0x1c, 0x5a, 0xd7, 0xbc, 0x2f, 0x3c, 0x25, 0xdb, 0xda, 0xef, 0x68, 0xe2, 0x36, 0xd8,
0xb3, 0x36, 0x96, 0x85, 0x92, 0xdb, 0xb8, 0x4d, 0x5f, 0xb6, 0x5d, 0x80, 0x3a, 0xb6, 0x6d, 0xcb,
0xee, 0x1a, 0xd8, 0x71, 0xd4, 0x3e, 0xe6, 0x49, 0x57, 0x8d, 0x4e, 0xae, 0xb3, 0x39, 0xf9, 0x1f,
0x79, 0x68, 0xf8, 0x47, 0xf1, 0xbe, 0xeb, 0xe8, 0x9a, 0xf7, 0x5d, 0x47, 0xd7, 0x88, 0x33, 0xb7,
0x99, 0x2b, 0x0c, 0x38, 0x73, 0x3e, 0xd3, 0xd1, 0x48, 0x1c, 0x24, 0x06, 0x66, 0x5a, 0x1a, 0xf6,
0x05, 0x0b, 0xde, 0x14, 0x97, 0x6b, 0x48, 0x3f, 0x0a, 0x19, 0xf4, 0xa3, 0x98, 0x41, 0x3f, 0x4a,
0x09, 0xfa, 0x31, 0x0f, 0xa5, 0xed, 0x51, 0x6f, 0x0f, 0xbb, 0x3c, 0x3d, 0xe2, 0xa3, 0xb0, 0xde,
0x94, 0x23, 0x7a, 0x23, 0xd4, 0xa3, 0x12, 0x54, 0x8f, 0xd3, 0x50, 0x61, 0x1f, 0x17, 0xba, 0xae,
0x43, 0xbb, 0xac, 0x79, 0xa5, 0xcc, 0x26, 0xb6, 0x1c, 0x74, 0xc3, 0xab, 0x1d, 0xaa, 0x49, 0x86,
0x4e, 0x3d, 0x4e, 0x44, 0x43, 0xbc, 0xca, 0xe1, 0x06, 0xb4, 0x76, 0xf1, 0xc8, 0xa6, 0x6f, 0x23,
0xba, 0xf4, 0x61, 0xef, 0xc3, 0x11, 0xb6, 0x0f, 0xd4, 0xed, 0x01, 0x6e, 0xd5, 0x28, 0x63, 0xf3,
0x62, 0x7d, 0x45, 0x75, 0xd5, 0xfb, 0xde, 0x2a, 0x7a, 0x03, 0xe6, 0x23, 0x98, 0xba, 0xa9, 0xe1,
0xc7, 0x58, 0x6b, 0xd5, 0x29, 0xde, 0x89, 0x10, 0x5e, 0x87, 0xad, 0xc9, 0x9f, 0x00, 0xf2, 0x39,
0x99, 0xae, 0x76, 0x8c, 0x88, 0x3a, 0x17, 0x15, 0xb5, 0xfc, 0x6b, 0x09, 0xe6, 0x82, 0x9b, 0x1d,
0x35, 0x80, 0xbe, 0x07, 0x55, 0xd6, 0xb3, 0xee, 0x12, 0x03, 0xe6, 0xd5, 0xe3, 0xd9, 0xb1, 0x77,
0xac, 0x80, 0xee, 0xfb, 0xb1, 0x0b, 0x50, 0xdf, 0xb7, 0xec, 0x3d, 0xdd, 0xec, 0x77, 0x09, 0x67,
0x9e, 0xd9, 0xd4, 0xf8, 0xe4, 0x3d, 0x32, 0x27, 0x7f, 0x21, 0xc1, 0xc2, 0x87, 0x43, 0x4d, 0x75,
0x71, 0x20, 0x93, 0x98, 0xf6, 0x81, 0xcd, 0x9b, 0xde, 0x1b, 0x97, 0x5c, 0xb6, 0xbe, 0x2b, 0x83,
0x96, 0xd7, 0xe1, 0x94, 0x82, 0x1d, 0x6c, 0x6a, 0xa1, 0xc5, 0xa3, 0x72, 0x21, 0x0f, 0xa1, 0x9d,
0x44, 0x6e, 0x1a, 0xd9, 0xb3, 0x94, 0xae, 0x6b, 0x13, 0xb2, 0x2e, 0xf7, 0x50, 0x24, 0x93, 0xa0,
0xfb, 0xb8, 0xf2, 0xbf, 0x24, 0x98, 0xbb, 0xa9, 0x79, 0xfb, 0x3d, 0xb3, 0xcc, 0x31, 0x9a, 0x59,
0xe5, 0xe3, 0x99, 0xd5, 0xd3, 0x72, 0x37, 0xdc, 0xe9, 0x9a, 0x23, 0xc3, 0x0b, 0x26, 0x36, 0xfd,
0x02, 0x7c, 0xf9, 0xa7, 0x12, 0xcc, 0xc5, 0xfa, 0x02, 0xa8, 0x01, 0xf0, 0xa1, 0xd9, 0xe3, 0x0d,
0x93, 0xe6, 0x31, 0x54, 0x83, 0xb2, 0xd7, 0x3e, 0x69, 0x4a, 0xa8, 0x0a, 0x33, 0x5b, 0x16, 0x85,
0x6e, 0xe6, 0x50, 0x13, 0x6a, 0x0c, 0x71, 0xd4, 0xeb, 0x61, 0xc7, 0x69, 0xe6, 0xc5, 0xcc, 0x5d,
0x55, 0x1f, 0x8c, 0x6c, 0xdc, 0x2c, 0xa0, 0x3a, 0x54, 0xb6, 0x2c, 0xfe, 0x2c, 0xa8, 0x59, 0x44,
0x08, 0x1a, 0xde, 0x1b, 0x21, 0x8e, 0x54, 0x0a, 0xcc, 0x79, 0x68, 0x33, 0x97, 0x77, 0x82, 0x15,
0x34, 0x29, 0x2b, 0xd1, 0x49, 0x38, 0xfe, 0xa1, 0xa9, 0xe1, 0x1d, 0xdd, 0xc4, 0x9a, 0xbf, 0xd4,
0x3c, 0x86, 0x8e, 0xc3, 0x6c, 0xc7, 0x34, 0xb1, 0x1d, 0x98, 0x94, 0xc8, 0xe4, 0x3a, 0xb6, 0xfb,
0x38, 0x30, 0x99, 0x43, 0x73, 0x50, 0x5f, 0xd7, 0x1f, 0x07, 0xa6, 0xf2, 0xcb, 0x7f, 0x3f, 0x09,
0x15, 0xe2, 0x64, 0x6e, 0x5b, 0x96, 0xad, 0xa1, 0x21, 0x20, 0xfa, 0x74, 0xce, 0x18, 0x5a, 0xa6,
0x78, 0x63, 0x8a, 0x5e, 0x4b, 0x49, 0x7e, 0xe2, 0xa0, 0x5c, 0x5d, 0xda, 0x97, 0x52, 0x30, 0x22,
0xe0, 0xf2, 0x31, 0x64, 0xd0, 0x1d, 0x49, 0x59, 0xbe, 0xa5, 0xf7, 0xf6, 0xbc, 0x47, 0x01, 0x63,
0x76, 0x8c, 0x80, 0x7a, 0x3b, 0x46, 0x9e, 0xae, 0xf2, 0x01, 0x7b, 0xdf, 0xe8, 0xd9, 0x8b, 0x7c,
0x0c, 0x3d, 0x84, 0x13, 0xab, 0x38, 0xe0, 0x1f, 0xbc, 0x0d, 0x97, 0xd3, 0x37, 0x8c, 0x01, 0x1f,
0x72, 0xcb, 0x35, 0x28, 0xd2, 0x1e, 0x1c, 0x4a, 0x72, 0x21, 0xc1, 0x3f, 0x69, 0xb4, 0x17, 0xd3,
0x01, 0x04, 0xb5, 0x4f, 0x60, 0x36, 0xf2, 0x90, 0x1c, 0xbd, 0x9c, 0x80, 0x96, 0xfc, 0x97, 0x80,
0xf6, 0xe5, 0x2c, 0xa0, 0x62, 0xaf, 0x3e, 0x34, 0xc2, 0x0f, 0xef, 0xd0, 0x52, 0x02, 0x7e, 0xe2,
0x23, 0xe0, 0xf6, 0xcb, 0x19, 0x20, 0xc5, 0x46, 0x06, 0x34, 0xa3, 0x0f, 0x9b, 0xd1, 0xe5, 0xb1,
0x04, 0xc2, 0xea, 0xf6, 0x4a, 0x26, 0x58, 0xb1, 0xdd, 0x01, 0x55, 0x82, 0xd8, 0xc3, 0x5a, 0x74,
0x35, 0x99, 0x4c, 0xda, 0x8b, 0xdf, 0xf6, 0xb5, 0xcc, 0xf0, 0x62, 0xeb, 0x6f, 0xb3, 0xde, 0x7f,
0xd2, 0xe3, 0x54, 0xf4, 0x7a, 0x32, 0xb9, 0x31, 0xaf, 0x6a, 0xdb, 0xcb, 0x87, 0x41, 0x11, 0x4c,
0x7c, 0x46, 0x9b, 0xf6, 0x09, 0x0f, 0x3c, 0xa3, 0x76, 0xe7, 0xd1, 0x4b, 0x7f, 0xb9, 0xda, 0x7e,
0xfd, 0x10, 0x18, 0x82, 0x01, 0x2b, 0xfa, 0x74, 0xdc, 0x33, 0xc3, 0x6b, 0x13, 0xb5, 0xe6, 0x68,
0x36, 0xf8, 0x31, 0xcc, 0x46, 0x9e, 0x5f, 0x24, 0x5a, 0x4d, 0xf2, 0x13, 0x8d, 0xf6, 0xb8, 0xb0,
0xca, 0x4c, 0x32, 0xf2, 0x0d, 0x04, 0xa5, 0x68, 0x7f, 0xc2, 0x77, 0x92, 0xf6, 0xe5, 0x2c, 0xa0,
0xe2, 0x20, 0x0e, 0x75, 0x97, 0x91, 0xef, 0x08, 0xe8, 0x4a, 0x32, 0x8d, 0xe4, 0x6f, 0x20, 0xed,
0x57, 0x33, 0x42, 0x8b, 0x4d, 0xbb, 0x00, 0xab, 0xd8, 0x5d, 0xc7, 0xae, 0x4d, 0x74, 0xe4, 0x52,
0xe2, 0x95, 0xfb, 0x00, 0xde, 0x36, 0x2f, 0x4d, 0x84, 0x13, 0x1b, 0xfc, 0x3f, 0x20, 0x2f, 0xc4,
0x06, 0x1e, 0xff, 0x5c, 0x18, 0xdb, 0x6a, 0x65, 0x7d, 0xd1, 0x49, 0xb2, 0x79, 0x08, 0xcd, 0x75,
0xd5, 0x24, 0x45, 0xb6, 0x4f, 0xf7, 0x4a, 0x22, 0x63, 0x51, 0xb0, 0x94, 0xdb, 0x4a, 0x85, 0x16,
0x87, 0xd9, 0x17, 0x31, 0x54, 0x15, 0x26, 0x88, 0xa3, 0xbe, 0xc5, 0xbf, 0x8d, 0x08, 0x60, 0x8a,
0x6f, 0x19, 0x03, 0x2f, 0x36, 0x7e, 0x22, 0xd1, 0x3f, 0x28, 0x44, 0x00, 0x1e, 0xe8, 0xee, 0xee,
0xc6, 0x40, 0x35, 0x9d, 0x2c, 0x2c, 0x50, 0xc0, 0x43, 0xb0, 0xc0, 0xe1, 0x05, 0x0b, 0x1a, 0xd4,
0x43, 0x9d, 0x4c, 0x94, 0xf4, 0x82, 0x27, 0xa9, 0x97, 0xda, 0x5e, 0x9a, 0x0c, 0x28, 0x76, 0xd9,
0x85, 0xba, 0xa7, 0xaf, 0xec, 0x72, 0x5f, 0x4e, 0xe3, 0xd4, 0x87, 0x49, 0x31, 0xb7, 0x64, 0xd0,
0xa0, 0xb9, 0xc5, 0x1b, 0x35, 0x28, 0x5b, 0x83, 0x6f, 0x9c, 0xb9, 0xa5, 0x77, 0x7f, 0x98, 0x3f,
0x89, 0x34, 0x45, 0x93, 0x9d, 0x55, 0x62, 0x8f, 0x37, 0xd1, 0x9f, 0xa4, 0xf4, 0x58, 0xe5, 0x63,
0xe8, 0x01, 0x94, 0x58, 0xe5, 0x85, 0x5e, 0x1c, 0x5f, 0x94, 0x71, 0xea, 0x17, 0x27, 0x40, 0x09,
0xc2, 0x7b, 0x70, 0x32, 0xa5, 0x24, 0x4b, 0x8c, 0x73, 0xe3, 0xcb, 0xb7, 0x49, 0x56, 0xae, 0x02,
0x8a, 0xff, 0x01, 0x20, 0x51, 0x4c, 0xa9, 0xff, 0x13, 0xc8, 0xb0, 0x45, 0xfc, 0x6d, 0x7f, 0xe2,
0x16, 0xa9, 0x7f, 0x01, 0x98, 0xb4, 0xc5, 0x7d, 0x00, 0xbf, 0xf0, 0x4a, 0x94, 0x47, 0xac, 0x2e,
0x9b, 0x40, 0x72, 0xf9, 0xdf, 0x25, 0x28, 0x7b, 0xef, 0x65, 0x9e, 0x43, 0x72, 0xff, 0x1c, 0xb2,
0xed, 0x8f, 0x61, 0x36, 0xf2, 0xf0, 0x3e, 0xd1, 0x78, 0x92, 0x1f, 0xe7, 0x4f, 0x92, 0xd0, 0x03,
0xfe, 0xd7, 0x65, 0x11, 0x78, 0x5f, 0x4a, 0xcb, 0xd8, 0xa3, 0x31, 0x77, 0x02, 0xe1, 0x67, 0x1e,
0x61, 0xef, 0x01, 0x04, 0x22, 0xe0, 0xf8, 0x8f, 0x98, 0xc4, 0xa9, 0x4f, 0x62, 0x78, 0xfd, 0x90,
0x7e, 0x63, 0x02, 0x39, 0x87, 0x58, 0x57, 0xb4, 0xcd, 0x91, 0x62, 0x5d, 0x29, 0xcd, 0x95, 0x44,
0x3f, 0x9b, 0xde, 0x3b, 0x79, 0x26, 0xf6, 0x76, 0xeb, 0xfa, 0x37, 0x5e, 0xef, 0xeb, 0xee, 0xee,
0x68, 0x9b, 0xac, 0x5c, 0x63, 0xa0, 0xaf, 0xea, 0x16, 0xff, 0x75, 0xcd, 0x53, 0xf4, 0x6b, 0x14,
0xfb, 0x1a, 0xd9, 0x63, 0xb8, 0xbd, 0x5d, 0xa2, 0xa3, 0xeb, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff,
0x51, 0xc9, 0x7d, 0xde, 0x1d, 0x40, 0x00, 0x00,
// 3628 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3b, 0x5d, 0x6f, 0x1b, 0xd7,
0x95, 0x1e, 0x7e, 0x89, 0x3c, 0xfc, 0x10, 0x75, 0xed, 0xc8, 0x34, 0x6d, 0xcb, 0xf2, 0x38, 0x76,
0x14, 0xc7, 0xb1, 0x13, 0x79, 0x83, 0x04, 0x9b, 0x2f, 0xd8, 0x96, 0xad, 0x10, 0x2b, 0x79, 0xe5,
0x91, 0x12, 0x2f, 0x36, 0x8b, 0x25, 0x46, 0x9c, 0x2b, 0x6a, 0x22, 0xce, 0x0c, 0x3d, 0x33, 0xb4,
0xac, 0xbc, 0xc4, 0xd8, 0x60, 0x17, 0xc8, 0x62, 0xb1, 0x2d, 0xd0, 0x97, 0x16, 0xe8, 0x43, 0xd1,
0xa7, 0x7e, 0xa0, 0x40, 0x81, 0xa0, 0x0f, 0x69, 0xd1, 0xf7, 0xa0, 0x7d, 0xe8, 0x0f, 0xe8, 0x0f,
0x68, 0x5f, 0xda, 0xdf, 0x50, 0xdc, 0x8f, 0xb9, 0xf3, 0x4d, 0x8e, 0x48, 0x3b, 0x7e, 0xd3, 0x3d,
0x3c, 0xe7, 0xdc, 0x73, 0xcf, 0x3d, 0xdf, 0x73, 0x05, 0x4d, 0x4d, 0x75, 0xd5, 0x6e, 0xcf, 0xb2,
0x6c, 0xed, 0xfa, 0xd0, 0xb6, 0x5c, 0x0b, 0x2d, 0x18, 0xfa, 0xe0, 0xf1, 0xc8, 0x61, 0xab, 0xeb,
0xe4, 0xe7, 0x76, 0xad, 0x67, 0x19, 0x86, 0x65, 0x32, 0x50, 0xbb, 0xa1, 0x9b, 0x2e, 0xb6, 0x4d,
0x75, 0xc0, 0xd7, 0xb5, 0x20, 0x41, 0xbb, 0xe6, 0xf4, 0xf6, 0xb1, 0xa1, 0xb2, 0x95, 0x3c, 0x07,
0xc5, 0xbb, 0xc6, 0xd0, 0x3d, 0x92, 0x7f, 0x28, 0x41, 0xed, 0xde, 0x60, 0xe4, 0xec, 0x2b, 0xf8,
0xd1, 0x08, 0x3b, 0x2e, 0x7a, 0x03, 0x0a, 0xbb, 0xaa, 0x83, 0x5b, 0xd2, 0xb2, 0xb4, 0x52, 0x5d,
0x3d, 0x77, 0x3d, 0xb4, 0x2b, 0xdf, 0x6f, 0xd3, 0xe9, 0xdf, 0x56, 0x1d, 0xac, 0x50, 0x4c, 0x84,
0xa0, 0xa0, 0xed, 0x76, 0xd6, 0x5a, 0xb9, 0x65, 0x69, 0x25, 0xaf, 0xd0, 0xbf, 0xd1, 0x12, 0x80,
0x83, 0xfb, 0x06, 0x36, 0xdd, 0xce, 0x9a, 0xd3, 0xca, 0x2f, 0xe7, 0x57, 0xf2, 0x4a, 0x00, 0x82,
0x64, 0xa8, 0xf5, 0xac, 0xc1, 0x00, 0xf7, 0x5c, 0xdd, 0x32, 0x3b, 0x6b, 0xad, 0x02, 0xa5, 0x0d,
0xc1, 0xe4, 0x1f, 0x4b, 0x50, 0xe7, 0xa2, 0x39, 0x43, 0xcb, 0x74, 0x30, 0xba, 0x09, 0x25, 0xc7,
0x55, 0xdd, 0x91, 0xc3, 0xa5, 0x3b, 0x9b, 0x28, 0xdd, 0x36, 0x45, 0x51, 0x38, 0x6a, 0xa2, 0x78,
0xd1, 0xed, 0xf3, 0xf1, 0xed, 0x23, 0x47, 0x28, 0x44, 0x8f, 0x20, 0xff, 0x4a, 0x82, 0xe6, 0xb6,
0xb7, 0xf4, 0xb4, 0x77, 0x0a, 0x8a, 0x3d, 0x6b, 0x64, 0xba, 0x54, 0xc0, 0xba, 0xc2, 0x16, 0xe8,
0x22, 0xd4, 0x7a, 0xfb, 0xaa, 0x69, 0xe2, 0x41, 0xd7, 0x54, 0x0d, 0x4c, 0x45, 0xa9, 0x28, 0x55,
0x0e, 0xbb, 0xaf, 0x1a, 0x38, 0x93, 0x44, 0xcb, 0x50, 0x1d, 0xaa, 0xb6, 0xab, 0x87, 0x74, 0x16,
0x04, 0xa1, 0x36, 0x94, 0x75, 0xa7, 0x63, 0x0c, 0x2d, 0xdb, 0x6d, 0x15, 0x97, 0xa5, 0x95, 0xb2,
0x22, 0xd6, 0xf2, 0x4f, 0x24, 0x58, 0xbc, 0xe5, 0x38, 0x7a, 0xdf, 0x8c, 0x49, 0xbd, 0x08, 0x25,
0xd3, 0xd2, 0x70, 0x67, 0x8d, 0x8a, 0x9d, 0x57, 0xf8, 0x0a, 0x9d, 0x85, 0xca, 0x10, 0x63, 0xbb,
0x6b, 0x5b, 0x03, 0x4f, 0xe8, 0x32, 0x01, 0x28, 0xd6, 0x00, 0xa3, 0x07, 0xb0, 0xe0, 0x44, 0x18,
0xb1, 0x9b, 0xae, 0xae, 0x5e, 0xba, 0x1e, 0xb3, 0xd5, 0xeb, 0xd1, 0x4d, 0x95, 0x38, 0xb5, 0xfc,
0x34, 0x07, 0x27, 0x05, 0x1e, 0x93, 0x95, 0xfc, 0x4d, 0xb4, 0xea, 0xe0, 0xbe, 0x10, 0x8f, 0x2d,
0xb2, 0x68, 0x55, 0x5c, 0x47, 0x3e, 0x78, 0x1d, 0x19, 0x8c, 0x2f, 0xaa, 0xeb, 0x62, 0x5c, 0xd7,
0x17, 0xa0, 0x8a, 0x9f, 0x0c, 0x75, 0x1b, 0x77, 0x5d, 0xdd, 0xc0, 0xad, 0xd2, 0xb2, 0xb4, 0x52,
0x50, 0x80, 0x81, 0x76, 0x74, 0x23, 0x68, 0xad, 0x73, 0x99, 0xad, 0x55, 0xfe, 0xa9, 0x04, 0xa7,
0x63, 0xb7, 0xc4, 0xcd, 0x5f, 0x81, 0x26, 0x3d, 0xb9, 0xaf, 0x19, 0xe2, 0x08, 0x44, 0xe1, 0x57,
0xc6, 0x29, 0xdc, 0x47, 0x57, 0x62, 0xf4, 0x01, 0x21, 0x73, 0xd9, 0x85, 0x3c, 0x80, 0xd3, 0xeb,
0xd8, 0xe5, 0x1b, 0x90, 0xdf, 0xb0, 0x33, 0x7d, 0xf8, 0x08, 0xfb, 0x59, 0x2e, 0xe6, 0x67, 0xbf,
0xce, 0x09, 0x3f, 0xa3, 0x5b, 0x75, 0xcc, 0x3d, 0x0b, 0x9d, 0x83, 0x8a, 0x40, 0xe1, 0x56, 0xe1,
0x03, 0xd0, 0xdb, 0x50, 0x24, 0x92, 0x32, 0x93, 0x68, 0xac, 0x5e, 0x4c, 0x3e, 0x53, 0x80, 0xa7,
0xc2, 0xf0, 0x51, 0x07, 0x1a, 0x8e, 0xab, 0xda, 0x6e, 0x77, 0x68, 0x39, 0xf4, 0x9e, 0xa9, 0xe1,
0x54, 0x57, 0xe5, 0x30, 0x07, 0x11, 0x68, 0x37, 0x9d, 0xfe, 0x16, 0xc7, 0x54, 0xea, 0x94, 0xd2,
0x5b, 0xa2, 0xbb, 0x50, 0xc3, 0xa6, 0xe6, 0x33, 0x2a, 0x64, 0x66, 0x54, 0xc5, 0xa6, 0x26, 0xd8,
0xf8, 0xf7, 0x53, 0xcc, 0x7e, 0x3f, 0xff, 0x27, 0x41, 0x2b, 0x7e, 0x41, 0xb3, 0x04, 0xd1, 0x77,
0x19, 0x11, 0x66, 0x17, 0x34, 0xd6, 0xc3, 0xc5, 0x25, 0x29, 0x9c, 0x44, 0xd6, 0xe1, 0x25, 0x5f,
0x1a, 0xfa, 0xcb, 0x73, 0x33, 0x96, 0x2f, 0x25, 0x58, 0x8c, 0xee, 0x35, 0xcb, 0xb9, 0xff, 0x09,
0x8a, 0xba, 0xb9, 0x67, 0x79, 0xc7, 0x5e, 0x1a, 0xe3, 0x67, 0x64, 0x2f, 0x86, 0x2c, 0x1b, 0x70,
0x76, 0x1d, 0xbb, 0x1d, 0xd3, 0xc1, 0xb6, 0x7b, 0x5b, 0x37, 0x07, 0x56, 0x7f, 0x4b, 0x75, 0xf7,
0x67, 0xf0, 0x91, 0x90, 0xb9, 0xe7, 0x22, 0xe6, 0x2e, 0xff, 0x4c, 0x82, 0x73, 0xc9, 0xfb, 0xf1,
0xa3, 0xb7, 0xa1, 0xbc, 0xa7, 0xe3, 0x81, 0x46, 0x74, 0x26, 0x51, 0x9d, 0x89, 0x35, 0xf1, 0x95,
0x21, 0x41, 0xe6, 0x27, 0xbc, 0x98, 0x62, 0xa0, 0xdb, 0xae, 0xad, 0x9b, 0xfd, 0x0d, 0xdd, 0x71,
0x15, 0x86, 0x1f, 0xd0, 0x67, 0x3e, 0xbb, 0x65, 0xfe, 0xaf, 0x04, 0x4b, 0xeb, 0xd8, 0xbd, 0x23,
0x42, 0x2d, 0xf9, 0x5d, 0x77, 0x5c, 0xbd, 0xe7, 0x3c, 0xdb, 0x02, 0x24, 0x43, 0x3e, 0x95, 0xbf,
0x27, 0xc1, 0x85, 0x54, 0x61, 0xb8, 0xea, 0x78, 0x28, 0xf1, 0x02, 0x6d, 0x72, 0x28, 0xf9, 0x17,
0x7c, 0xf4, 0x89, 0x3a, 0x18, 0xe1, 0x2d, 0x55, 0xb7, 0x59, 0x28, 0x99, 0x32, 0xb0, 0xfe, 0x52,
0x82, 0xf3, 0xeb, 0xd8, 0xdd, 0xf2, 0xd2, 0xcc, 0x0b, 0xd4, 0xce, 0xe4, 0x6a, 0x43, 0xfe, 0x7f,
0x76, 0x99, 0x89, 0xd2, 0xbe, 0x10, 0xf5, 0x2d, 0x51, 0x3f, 0x08, 0x38, 0xe4, 0x1d, 0x56, 0x0b,
0x70, 0xe5, 0xc9, 0xff, 0x2d, 0xc1, 0x99, 0x5b, 0xbd, 0x47, 0x23, 0xdd, 0xc6, 0x1c, 0x69, 0xc3,
0xea, 0x1d, 0x4c, 0xaf, 0x5a, 0xbf, 0x6e, 0xca, 0x85, 0xea, 0xa6, 0x09, 0xd5, 0x2f, 0x95, 0x43,
0xc1, 0x03, 0xac, 0x3a, 0x2f, 0x56, 0x8e, 0xa7, 0x79, 0xa8, 0x7d, 0xc2, 0xeb, 0x25, 0x9a, 0x56,
0xa3, 0x76, 0x21, 0x25, 0xdb, 0x45, 0xa0, 0xc4, 0x4a, 0xaa, 0xba, 0xd6, 0xa1, 0xee, 0x60, 0x7c,
0x30, 0x4d, 0x12, 0xad, 0x11, 0x42, 0x91, 0xfc, 0x36, 0x60, 0x61, 0x64, 0xee, 0x91, 0x16, 0x00,
0x6b, 0x5c, 0x51, 0xac, 0x12, 0x9f, 0x1c, 0x89, 0xe3, 0x84, 0xe8, 0x23, 0x98, 0x8f, 0xf2, 0x2a,
0x66, 0xe2, 0x15, 0x25, 0x43, 0x1d, 0x68, 0x6a, 0xb6, 0x35, 0x1c, 0x62, 0xad, 0xeb, 0x78, 0xac,
0x4a, 0xd9, 0x58, 0x71, 0x3a, 0x8f, 0x95, 0xfc, 0x95, 0x04, 0x8b, 0x0f, 0x55, 0xb7, 0xb7, 0xbf,
0x66, 0x70, 0x63, 0x9d, 0xc1, 0xd5, 0xdf, 0x87, 0xca, 0x63, 0x7e, 0x11, 0x5e, 0x3c, 0xbf, 0x90,
0x20, 0x50, 0xf0, 0xca, 0x15, 0x9f, 0x42, 0xfe, 0x56, 0x82, 0x53, 0xb4, 0xe1, 0xf2, 0xa4, 0xfb,
0xee, 0x83, 0xce, 0x84, 0xa6, 0x0b, 0x5d, 0x81, 0x86, 0xa1, 0xda, 0x07, 0xdb, 0x3e, 0x4e, 0x91,
0xe2, 0x44, 0xa0, 0xf2, 0x13, 0x00, 0xbe, 0xda, 0x74, 0xfa, 0x53, 0xc8, 0xff, 0x0e, 0xcc, 0xf1,
0x5d, 0x79, 0xfc, 0x99, 0x74, 0xb1, 0x1e, 0xba, 0xfc, 0x07, 0x09, 0x1a, 0x7e, 0x46, 0xa1, 0x5e,
0xd5, 0x80, 0x9c, 0xf0, 0xa5, 0x5c, 0x67, 0x0d, 0xbd, 0x0f, 0x25, 0xd6, 0x8c, 0x73, 0xde, 0x97,
0xc3, 0xbc, 0x79, 0xa3, 0x1e, 0x48, 0x4b, 0x14, 0xa0, 0x70, 0x22, 0xa2, 0x23, 0x11, 0x85, 0x85,
0x57, 0xfb, 0x10, 0xd4, 0x81, 0xf9, 0x70, 0x11, 0xeb, 0xf9, 0xcc, 0x72, 0x5a, 0xf4, 0x5d, 0x53,
0x5d, 0x95, 0x06, 0xdf, 0x46, 0xa8, 0x86, 0x75, 0xe4, 0xbf, 0x17, 0xa1, 0x1a, 0x38, 0x65, 0xec,
0x24, 0xd1, 0x2b, 0xcd, 0x4d, 0xce, 0x23, 0xf9, 0x78, 0x27, 0x75, 0x19, 0x1a, 0x3a, 0xad, 0x5d,
0xba, 0xdc, 0x14, 0x69, 0xb2, 0xa9, 0x28, 0x75, 0x06, 0xe5, 0x7e, 0x81, 0x96, 0xa0, 0x6a, 0x8e,
0x8c, 0xae, 0xb5, 0xd7, 0xb5, 0xad, 0x43, 0x87, 0xb7, 0x64, 0x15, 0x73, 0x64, 0xfc, 0xeb, 0x9e,
0x62, 0x1d, 0x3a, 0x7e, 0xd5, 0x5f, 0x3a, 0x66, 0xd5, 0xbf, 0x04, 0x55, 0x43, 0x7d, 0x42, 0xb8,
0x76, 0xcd, 0x91, 0x41, 0xbb, 0xb5, 0xbc, 0x52, 0x31, 0xd4, 0x27, 0x8a, 0x75, 0x78, 0x7f, 0x64,
0xa0, 0x15, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0xdb, 0xbd, 0x32, 0x6d, 0xf7, 0x1a, 0x04, 0x7e,
0xd7, 0x6f, 0xf9, 0xe2, 0xfd, 0x43, 0x65, 0x86, 0xfe, 0x41, 0x33, 0x06, 0x3e, 0x23, 0xc8, 0xde,
0x3f, 0x68, 0xc6, 0x40, 0xb0, 0x79, 0x07, 0xe6, 0x76, 0x69, 0x45, 0xe8, 0xb4, 0xaa, 0xa9, 0x11,
0xea, 0x1e, 0x29, 0x06, 0x59, 0xe1, 0xa8, 0x78, 0xe8, 0xe8, 0x3d, 0xa8, 0xd0, 0x54, 0x4c, 0x69,
0x6b, 0x99, 0x68, 0x7d, 0x02, 0x42, 0xad, 0xe1, 0x81, 0xab, 0x52, 0xea, 0x7a, 0x36, 0x6a, 0x41,
0x80, 0xde, 0x80, 0x93, 0x3d, 0x1b, 0xab, 0x2e, 0xd6, 0x6e, 0x1f, 0xdd, 0xb1, 0x8c, 0xa1, 0x4a,
0x8d, 0xa9, 0xd5, 0xa0, 0x23, 0x8d, 0xa4, 0x9f, 0x48, 0x60, 0xe8, 0x89, 0xd5, 0x3d, 0xdb, 0x32,
0x5a, 0xf3, 0x2c, 0x30, 0x84, 0xa1, 0xe8, 0x3c, 0x80, 0x17, 0xba, 0x55, 0xb7, 0xd5, 0xa4, 0xb7,
0x58, 0xe1, 0x90, 0x5b, 0xae, 0xfc, 0x05, 0x9c, 0xf2, 0x2d, 0x24, 0x70, 0x1b, 0xf1, 0x8b, 0x95,
0xa6, 0xbd, 0xd8, 0xf1, 0xb5, 0xfc, 0x9f, 0x0a, 0xb0, 0xb8, 0xad, 0x3e, 0xc6, 0xcf, 0xbf, 0x6d,
0xc8, 0x14, 0x8f, 0x37, 0x60, 0x81, 0x76, 0x0a, 0xab, 0x01, 0x79, 0xc6, 0x64, 0xe0, 0xe0, 0x75,
0xc6, 0x09, 0xd1, 0x87, 0xa4, 0x74, 0xc0, 0xbd, 0x83, 0x2d, 0x4b, 0xf7, 0xb3, 0xef, 0xf9, 0x04,
0x3e, 0x77, 0x04, 0x96, 0x12, 0xa4, 0x40, 0x5b, 0xf1, 0xd0, 0xc6, 0xf2, 0xee, 0x2b, 0x63, 0xfb,
0x51, 0x5f, 0xfb, 0xd1, 0x08, 0x87, 0x5a, 0x30, 0xc7, 0xb3, 0x3b, 0xf5, 0xfb, 0xb2, 0xe2, 0x2d,
0xd1, 0x16, 0x9c, 0x64, 0x27, 0xd8, 0xe6, 0x46, 0xcd, 0x0e, 0x5f, 0xce, 0x74, 0xf8, 0x24, 0xd2,
0xb0, 0x4f, 0x54, 0x8e, 0xeb, 0x13, 0x2d, 0x98, 0xe3, 0x76, 0x4a, 0x63, 0x41, 0x59, 0xf1, 0x96,
0xe4, 0x9a, 0x75, 0x3a, 0xe3, 0xd3, 0xcd, 0x7e, 0xab, 0x4a, 0x7f, 0xf3, 0x01, 0xa4, 0xe5, 0x02,
0x5f, 0x9f, 0x13, 0x26, 0x27, 0x1f, 0x40, 0x59, 0x58, 0x78, 0x2e, 0xb3, 0x85, 0x0b, 0x9a, 0x68,
0x8c, 0xce, 0x47, 0x62, 0xb4, 0xfc, 0x47, 0x09, 0x6a, 0x6b, 0xe4, 0x48, 0x1b, 0x56, 0x9f, 0x66,
0x94, 0xcb, 0xd0, 0xb0, 0x71, 0xcf, 0xb2, 0xb5, 0x2e, 0x36, 0x5d, 0x5b, 0xc7, 0xac, 0x3b, 0x2f,
0x28, 0x75, 0x06, 0xbd, 0xcb, 0x80, 0x04, 0x8d, 0x84, 0x5d, 0xc7, 0x55, 0x8d, 0x61, 0x77, 0x8f,
0xb8, 0x77, 0x8e, 0xa1, 0x09, 0x28, 0xf5, 0xee, 0x8b, 0x50, 0xf3, 0xd1, 0x5c, 0x8b, 0xee, 0x5f,
0x50, 0xaa, 0x02, 0xb6, 0x63, 0xa1, 0x97, 0xa1, 0x41, 0x75, 0xda, 0x1d, 0x58, 0xfd, 0x2e, 0xe9,
0x64, 0x79, 0xb2, 0xa9, 0x69, 0x5c, 0x2c, 0x72, 0x57, 0x61, 0x2c, 0x47, 0xff, 0x1c, 0xf3, 0x74,
0x23, 0xb0, 0xb6, 0xf5, 0xcf, 0x31, 0xc9, 0xf5, 0x75, 0x92, 0x3b, 0xef, 0x5b, 0x1a, 0xde, 0x99,
0xb2, 0xd2, 0xc8, 0x30, 0xc5, 0x3c, 0x07, 0x15, 0x71, 0x02, 0x7e, 0x24, 0x1f, 0x80, 0xee, 0x41,
0xc3, 0x2b, 0x42, 0xbb, 0xac, 0xd7, 0x2a, 0xa4, 0x56, 0x7e, 0x81, 0xec, 0xe7, 0x28, 0x75, 0x8f,
0x8c, 0x2e, 0xe5, 0x7b, 0x50, 0x0b, 0xfe, 0x4c, 0x76, 0xdd, 0x8e, 0x1a, 0x8a, 0x00, 0x10, 0x6b,
0xbc, 0x3f, 0x32, 0xc8, 0x9d, 0xf2, 0xc0, 0xe2, 0x2d, 0xe5, 0x2f, 0x25, 0xa8, 0xf3, 0x94, 0xbd,
0x2d, 0x26, 0xf0, 0xf4, 0x68, 0x12, 0x3d, 0x1a, 0xfd, 0x1b, 0xfd, 0x73, 0x78, 0x44, 0xf7, 0x72,
0x62, 0x10, 0xa0, 0x4c, 0x68, 0x75, 0x1c, 0xca, 0xd7, 0x59, 0x7a, 0xfb, 0xa7, 0xc4, 0xd0, 0xf8,
0xd5, 0x50, 0x43, 0x6b, 0xc1, 0x9c, 0xaa, 0x69, 0x36, 0x76, 0x1c, 0x2e, 0x87, 0xb7, 0x24, 0xbf,
0x3c, 0xc6, 0xb6, 0xe3, 0x99, 0x7c, 0x5e, 0xf1, 0x96, 0xe8, 0x3d, 0x28, 0x8b, 0x72, 0x3a, 0x9f,
0x54, 0x42, 0x05, 0xe5, 0xe4, 0xbd, 0xa8, 0xa0, 0x90, 0x7f, 0x93, 0x83, 0x06, 0x57, 0xd8, 0x6d,
0x9e, 0x53, 0xc7, 0x3b, 0xdf, 0x6d, 0xa8, 0xed, 0xf9, 0xbe, 0x3f, 0x6e, 0xe6, 0x14, 0x0c, 0x11,
0x21, 0x9a, 0x49, 0x0e, 0x18, 0xce, 0xea, 0x85, 0x99, 0xb2, 0x7a, 0xf1, 0xb8, 0x11, 0x2c, 0x5e,
0xe7, 0x95, 0x12, 0xea, 0x3c, 0xf9, 0x3f, 0xa0, 0x1a, 0x60, 0x40, 0x23, 0x34, 0x1b, 0x56, 0x71,
0x8d, 0x79, 0x4b, 0x74, 0xd3, 0xaf, 0x6d, 0x98, 0xaa, 0xce, 0x24, 0xc8, 0x12, 0x29, 0x6b, 0xe4,
0x9f, 0x4b, 0x50, 0xe2, 0x9c, 0x2f, 0x40, 0x95, 0x07, 0x1d, 0x5a, 0xf7, 0x31, 0xee, 0xc0, 0x41,
0xa4, 0xf0, 0x7b, 0x76, 0x51, 0xe7, 0x0c, 0x94, 0x23, 0xf1, 0x66, 0x8e, 0xa7, 0x05, 0xef, 0xa7,
0x40, 0x90, 0x21, 0x3f, 0xd1, 0xf8, 0xf2, 0xad, 0x44, 0x07, 0xed, 0x0a, 0xee, 0x59, 0x8f, 0xb1,
0x7d, 0x34, 0xfb, 0x38, 0xf3, 0xdd, 0x80, 0x41, 0x67, 0xec, 0x0f, 0x05, 0x01, 0x7a, 0xd7, 0x57,
0x77, 0x3e, 0x69, 0x9a, 0x13, 0x8c, 0x30, 0xdc, 0x1c, 0x7d, 0xb5, 0x7f, 0x9f, 0x0d, 0x66, 0xc3,
0x47, 0x99, 0xb6, 0xae, 0x79, 0x26, 0x6d, 0x87, 0xfc, 0x03, 0x09, 0xce, 0xac, 0x63, 0xf7, 0x5e,
0xb8, 0xb9, 0x7f, 0xd1, 0x52, 0x19, 0xd0, 0x4e, 0x12, 0x6a, 0x96, 0x5b, 0x6f, 0x43, 0x59, 0x8c,
0x29, 0xd8, 0xc8, 0x5c, 0xac, 0xe5, 0xff, 0x91, 0xa0, 0xc5, 0x77, 0xa1, 0x7b, 0x92, 0x92, 0x7a,
0x80, 0x5d, 0xac, 0x7d, 0xd7, 0x7d, 0xf3, 0xef, 0x25, 0x68, 0x06, 0x23, 0x3e, 0x0d, 0xda, 0x6f,
0x41, 0x91, 0x8e, 0x27, 0xb8, 0x04, 0x13, 0x8d, 0x95, 0x61, 0x93, 0x90, 0x41, 0xcb, 0xbc, 0x1d,
0x91, 0x9c, 0xf8, 0xd2, 0x4f, 0x3b, 0xf9, 0xe3, 0xa7, 0x1d, 0x9e, 0x86, 0xad, 0x11, 0xe1, 0xcb,
0xc6, 0xa1, 0x3e, 0x40, 0xfe, 0x3a, 0x07, 0x2d, 0xbf, 0x1f, 0xf9, 0xce, 0xe3, 0x7e, 0x4a, 0xb5,
0x9a, 0x7f, 0x46, 0xd5, 0x6a, 0x61, 0xf6, 0x58, 0x5f, 0x4c, 0x8a, 0xf5, 0xbf, 0xcb, 0x41, 0xc3,
0xd7, 0xda, 0xd6, 0x40, 0x35, 0xd1, 0x22, 0x94, 0x86, 0x03, 0xd5, 0x9f, 0x3e, 0xf2, 0x15, 0xda,
0x16, 0x75, 0x4e, 0x58, 0x4f, 0xaf, 0x25, 0xdd, 0x61, 0xca, 0x45, 0x28, 0x11, 0x16, 0xa4, 0x1d,
0x64, 0x0d, 0x05, 0x6d, 0xea, 0x79, 0x6d, 0xc5, 0x8c, 0x85, 0xf4, 0xf3, 0xd7, 0x00, 0xf1, 0x1b,
0xee, 0xea, 0x66, 0xd7, 0xc1, 0x3d, 0xcb, 0xd4, 0xd8, 0xdd, 0x17, 0x95, 0x26, 0xff, 0xa5, 0x63,
0x6e, 0x33, 0x38, 0x7a, 0x0b, 0x0a, 0xee, 0xd1, 0x90, 0x45, 0xf1, 0x46, 0x62, 0x74, 0xf4, 0xe5,
0xda, 0x39, 0x1a, 0x62, 0x85, 0xa2, 0xa3, 0x25, 0x00, 0xc2, 0xca, 0xb5, 0xd5, 0xc7, 0x3c, 0x25,
0x16, 0x94, 0x00, 0x84, 0x58, 0xb3, 0xa7, 0xc3, 0x39, 0x96, 0x3a, 0xf8, 0x52, 0xfe, 0x26, 0x07,
0x4d, 0x9f, 0xa5, 0x82, 0x9d, 0xd1, 0xc0, 0x4d, 0xd5, 0xdf, 0xf8, 0x66, 0x70, 0x52, 0xdd, 0xf0,
0x21, 0x54, 0xf9, 0x7d, 0x1e, 0xc3, 0x1e, 0x80, 0x91, 0x6c, 0x8c, 0x31, 0xd0, 0xe2, 0x33, 0x32,
0xd0, 0xd2, 0x31, 0x0d, 0x54, 0xde, 0x86, 0x45, 0x2f, 0xee, 0xf9, 0x08, 0x9b, 0xd8, 0x55, 0xc7,
0x14, 0x1c, 0x17, 0xa0, 0xca, 0xf2, 0x19, 0x4b, 0xe4, 0xac, 0x54, 0x87, 0x5d, 0xd1, 0xe1, 0xca,
0xff, 0x09, 0xa7, 0x68, 0xdc, 0x88, 0x8e, 0x72, 0xb3, 0xcc, 0xd5, 0x65, 0xd1, 0x08, 0x90, 0xa2,
0x9f, 0x59, 0x77, 0x45, 0x09, 0xc1, 0xe4, 0x0d, 0x78, 0x29, 0xc2, 0x7f, 0x86, 0xbc, 0x40, 0x4a,
0xa1, 0xc5, 0xed, 0xf0, 0x67, 0xe2, 0xe9, 0xb3, 0xdf, 0x79, 0x31, 0xb9, 0xed, 0xea, 0x5a, 0xd4,
0xbe, 0x34, 0xf4, 0x01, 0x54, 0x4c, 0x7c, 0xd8, 0x0d, 0x06, 0xdf, 0x0c, 0x03, 0xba, 0xb2, 0x89,
0x0f, 0xe9, 0x5f, 0xf2, 0x7d, 0x38, 0x1d, 0x13, 0x75, 0x96, 0xb3, 0xff, 0x56, 0x82, 0x33, 0x6b,
0xb6, 0x35, 0xfc, 0x44, 0xb7, 0xdd, 0x91, 0x3a, 0x08, 0x7f, 0x28, 0x7a, 0x3e, 0x6d, 0xdc, 0x47,
0x81, 0x34, 0xcc, 0xe2, 0xf2, 0xb5, 0x04, 0x73, 0x8d, 0x0b, 0xc5, 0x0f, 0x1d, 0x48, 0xda, 0x7f,
0xc9, 0x27, 0x09, 0xcf, 0xf1, 0x26, 0x24, 0x9b, 0x2c, 0x55, 0x4a, 0xe2, 0xd4, 0x27, 0x3f, 0xed,
0xd4, 0x27, 0xc5, 0xf3, 0x0b, 0xcf, 0xc8, 0xf3, 0x8f, 0xdd, 0x86, 0x7c, 0x04, 0xe1, 0x89, 0x1c,
0x0d, 0xb9, 0x53, 0x8d, 0xf2, 0x6e, 0x03, 0xf8, 0xd3, 0x29, 0xfe, 0xca, 0x27, 0x0b, 0x9b, 0x00,
0x15, 0xb9, 0x2d, 0x11, 0x65, 0xe9, 0x54, 0x39, 0x34, 0x2f, 0x79, 0x00, 0xed, 0x24, 0x2b, 0x9d,
0xc5, 0xf2, 0xbf, 0xce, 0x01, 0xb0, 0x27, 0x61, 0x3b, 0xaa, 0x73, 0x30, 0x5d, 0x45, 0x79, 0x09,
0xea, 0xbe, 0xc1, 0xf8, 0xfe, 0x1e, 0xb4, 0x22, 0x8d, 0xb8, 0x84, 0x28, 0x6c, 0x09, 0x4e, 0xac,
0xd8, 0xd5, 0x28, 0x9f, 0x80, 0xd7, 0x30, 0xa3, 0x88, 0x04, 0x3d, 0x74, 0x16, 0x2a, 0xb6, 0x75,
0xd8, 0x25, 0x6e, 0xa6, 0x79, 0xaf, 0xda, 0x6c, 0xeb, 0x90, 0x38, 0x9f, 0x86, 0x4e, 0xc3, 0x9c,
0xab, 0x3a, 0x07, 0x84, 0x7f, 0x89, 0xa5, 0x3b, 0xb2, 0xec, 0x68, 0xe8, 0x14, 0x14, 0xf7, 0xf4,
0x01, 0x76, 0x5a, 0x73, 0x94, 0x25, 0x5b, 0xa0, 0xb7, 0xbd, 0xf7, 0x1c, 0xe5, 0xcc, 0xdf, 0xa3,
0xd9, 0x93, 0x8e, 0x6f, 0x25, 0x98, 0xf7, 0xb5, 0x46, 0x03, 0x10, 0x89, 0x69, 0x34, 0x9e, 0xdd,
0xb1, 0x34, 0x16, 0x2a, 0x1a, 0x29, 0x9f, 0x58, 0x18, 0x21, 0x8b, 0x5a, 0x3e, 0xc9, 0xb8, 0xba,
0x9c, 0x9c, 0x8b, 0x1c, 0x5a, 0xd7, 0xbc, 0x2f, 0x3c, 0x25, 0xdb, 0x3a, 0xec, 0x68, 0x42, 0x1b,
0xec, 0x59, 0x1b, 0xab, 0x42, 0x89, 0x36, 0xee, 0xd0, 0x97, 0x6d, 0x97, 0xa0, 0x8e, 0x6d, 0xdb,
0xb2, 0xbb, 0x06, 0x76, 0x1c, 0xb5, 0x8f, 0x79, 0xd1, 0x55, 0xa3, 0xc0, 0x4d, 0x06, 0x93, 0xbf,
0xc9, 0x43, 0xc3, 0x3f, 0x8a, 0xf7, 0x5d, 0x47, 0xd7, 0xbc, 0xef, 0x3a, 0xba, 0x46, 0x82, 0xb9,
0xcd, 0x42, 0x61, 0x20, 0x98, 0x73, 0x48, 0x47, 0x23, 0x79, 0x90, 0x38, 0x98, 0x69, 0x69, 0xd8,
0xbf, 0x58, 0xf0, 0x40, 0xfc, 0x5e, 0x43, 0xf6, 0x51, 0xc8, 0x60, 0x1f, 0xc5, 0x0c, 0xf6, 0x51,
0x4a, 0xb0, 0x8f, 0x45, 0x28, 0xed, 0x8e, 0x7a, 0x07, 0xd8, 0xe5, 0xe5, 0x11, 0x5f, 0x85, 0xed,
0xa6, 0x1c, 0xb1, 0x1b, 0x61, 0x1e, 0x95, 0xa0, 0x79, 0x9c, 0x85, 0x0a, 0xfb, 0xb8, 0xd0, 0x75,
0x1d, 0x3a, 0x65, 0xcd, 0x2b, 0x65, 0x06, 0xd8, 0x71, 0xd0, 0x3b, 0x5e, 0xef, 0x50, 0x4d, 0x72,
0x74, 0x1a, 0x71, 0x22, 0x16, 0xe2, 0x75, 0x0e, 0x97, 0xa1, 0x41, 0x9f, 0xf3, 0x3e, 0x1a, 0x61,
0xfb, 0x48, 0xdd, 0x1d, 0xe0, 0x56, 0x8d, 0x8a, 0x53, 0x27, 0xd0, 0x07, 0x1e, 0x90, 0x28, 0x84,
0xa2, 0xe9, 0xa6, 0x86, 0x9f, 0x60, 0xad, 0x55, 0xa7, 0x48, 0x54, 0xd5, 0x1d, 0x06, 0x92, 0x3f,
0x03, 0xe4, 0xef, 0x31, 0x5b, 0x57, 0x18, 0xb9, 0xc4, 0x5c, 0xf4, 0x12, 0xe5, 0x5f, 0x48, 0xb0,
0x10, 0xdc, 0x6c, 0xda, 0xd4, 0xf8, 0x01, 0x54, 0xd9, 0x34, 0xba, 0x4b, 0x5c, 0x93, 0xf7, 0x85,
0xe7, 0xc7, 0x6a, 0x4f, 0x01, 0xdd, 0x8f, 0x50, 0x97, 0xa0, 0x7e, 0x68, 0xd9, 0x07, 0xba, 0xd9,
0xef, 0x12, 0xc9, 0x3c, 0x87, 0xa8, 0x71, 0xe0, 0x7d, 0x02, 0x93, 0xbf, 0x92, 0x60, 0xe9, 0xe3,
0xa1, 0xa6, 0xba, 0x38, 0x50, 0x23, 0xcc, 0xfa, 0x74, 0xe6, 0x2d, 0xef, 0xf5, 0x4a, 0x2e, 0xdb,
0x44, 0x95, 0x61, 0xcb, 0x9b, 0x70, 0x46, 0xc1, 0x0e, 0x36, 0xb5, 0xd0, 0x8f, 0xd3, 0x4a, 0x21,
0x0f, 0xa1, 0x9d, 0xc4, 0x6e, 0x96, 0xbb, 0x67, 0xc5, 0x5a, 0xd7, 0x26, 0x6c, 0x5d, 0x1e, 0x7b,
0x48, 0x8d, 0x40, 0xf7, 0x71, 0xe5, 0xbf, 0x4a, 0xb0, 0x70, 0x4b, 0xf3, 0xf6, 0x7b, 0x6e, 0x35,
0x61, 0xb4, 0x66, 0xca, 0xc7, 0x6b, 0xa6, 0x67, 0x15, 0x48, 0x78, 0x38, 0x35, 0x47, 0x86, 0x97,
0x26, 0x6c, 0xfa, 0x6d, 0xf7, 0xea, 0x8f, 0x24, 0x58, 0x88, 0x75, 0xfc, 0xa8, 0x01, 0xf0, 0xb1,
0xd9, 0xe3, 0xa3, 0x90, 0xe6, 0x09, 0x54, 0x83, 0xb2, 0x37, 0x18, 0x69, 0x4a, 0xa8, 0x0a, 0x73,
0x3b, 0x16, 0xc5, 0x6e, 0xe6, 0x50, 0x13, 0x6a, 0x8c, 0x70, 0xd4, 0xeb, 0x61, 0xc7, 0x69, 0xe6,
0x05, 0xe4, 0x9e, 0xaa, 0x0f, 0x46, 0x36, 0x6e, 0x16, 0x50, 0x1d, 0x2a, 0x3b, 0x16, 0x7f, 0xf0,
0xd3, 0x2c, 0x22, 0x04, 0x0d, 0xef, 0xf5, 0x0f, 0x27, 0x2a, 0x05, 0x60, 0x1e, 0xd9, 0xdc, 0xd5,
0xbd, 0x60, 0x6f, 0x4c, 0x1a, 0x46, 0x74, 0x1a, 0x4e, 0x7e, 0x6c, 0x6a, 0x78, 0x4f, 0x37, 0xb1,
0xe6, 0xff, 0xd4, 0x3c, 0x81, 0x4e, 0xc2, 0x7c, 0xc7, 0x34, 0xb1, 0x1d, 0x00, 0x4a, 0x04, 0xb8,
0x89, 0xed, 0x3e, 0x0e, 0x00, 0x73, 0x68, 0x01, 0xea, 0x9b, 0xfa, 0x93, 0x00, 0x28, 0xbf, 0xfa,
0xe7, 0xd3, 0x50, 0x59, 0x53, 0x5d, 0xf5, 0x8e, 0x65, 0xd9, 0x1a, 0x1a, 0x02, 0xa2, 0x8f, 0xe2,
0x8c, 0xa1, 0x65, 0x8a, 0xd7, 0xa3, 0xe8, 0x8d, 0x94, 0xb2, 0x26, 0x8e, 0xca, 0xcd, 0xa5, 0x7d,
0x25, 0x85, 0x22, 0x82, 0x2e, 0x9f, 0x40, 0x06, 0xdd, 0x91, 0x34, 0xdc, 0x3b, 0x7a, 0xef, 0xc0,
0xfb, 0xdc, 0x3f, 0x66, 0xc7, 0x08, 0xaa, 0xb7, 0x63, 0xe4, 0x51, 0x2a, 0x5f, 0xb0, 0x97, 0x8b,
0x9e, 0xbf, 0xc8, 0x27, 0xd0, 0x23, 0x38, 0xb5, 0x8e, 0x03, 0xf1, 0xc1, 0xdb, 0x70, 0x35, 0x7d,
0xc3, 0x18, 0xf2, 0x31, 0xb7, 0xdc, 0x80, 0x22, 0x9d, 0xae, 0xa1, 0xa4, 0x10, 0x12, 0xfc, 0xf7,
0x8b, 0xf6, 0x72, 0x3a, 0x82, 0xe0, 0xf6, 0x19, 0xcc, 0x47, 0x9e, 0x88, 0xa3, 0x57, 0x13, 0xc8,
0x92, 0x1f, 0xfb, 0xb7, 0xaf, 0x66, 0x41, 0x15, 0x7b, 0xf5, 0xa1, 0x11, 0x7e, 0x52, 0x87, 0x56,
0x12, 0xe8, 0x13, 0x9f, 0xf7, 0xb6, 0x5f, 0xcd, 0x80, 0x29, 0x36, 0x32, 0xa0, 0x19, 0x7d, 0xb2,
0x8c, 0xae, 0x8e, 0x65, 0x10, 0x36, 0xb7, 0xd7, 0x32, 0xe1, 0x8a, 0xed, 0x8e, 0xa8, 0x11, 0xc4,
0x9e, 0xcc, 0xa2, 0xeb, 0xc9, 0x6c, 0xd2, 0xde, 0xf2, 0xb6, 0x6f, 0x64, 0xc6, 0x17, 0x5b, 0xff,
0x17, 0x9b, 0xea, 0x27, 0x3d, 0x3b, 0x45, 0x6f, 0x26, 0xb3, 0x1b, 0xf3, 0x5e, 0xb6, 0xbd, 0x7a,
0x1c, 0x12, 0x21, 0xc4, 0x17, 0x74, 0x1c, 0x9f, 0xf0, 0x74, 0x33, 0xea, 0x77, 0x1e, 0xbf, 0xf4,
0x37, 0xa9, 0xed, 0x37, 0x8f, 0x41, 0x21, 0x04, 0xb0, 0xa2, 0x8f, 0xc2, 0x3d, 0x37, 0xbc, 0x31,
0xd1, 0x6a, 0xa6, 0xf3, 0xc1, 0x4f, 0x61, 0x3e, 0xf2, 0xb0, 0x22, 0xd1, 0x6b, 0x92, 0x1f, 0x5f,
0xb4, 0xc7, 0xa5, 0x55, 0xe6, 0x92, 0x91, 0xaf, 0x1b, 0x28, 0xc5, 0xfa, 0x13, 0xbe, 0x80, 0xb4,
0xaf, 0x66, 0x41, 0x15, 0x07, 0x71, 0x68, 0xb8, 0x8c, 0x7c, 0x21, 0x40, 0xd7, 0x92, 0x79, 0x24,
0x7f, 0xdd, 0x68, 0xbf, 0x9e, 0x11, 0x5b, 0x6c, 0xda, 0x05, 0x58, 0xc7, 0xee, 0x26, 0x76, 0x6d,
0x62, 0x23, 0x57, 0x12, 0x55, 0xee, 0x23, 0x78, 0xdb, 0xbc, 0x32, 0x11, 0x4f, 0x6c, 0xf0, 0x6f,
0x80, 0xbc, 0x14, 0x1b, 0x78, 0xd6, 0x73, 0x69, 0xec, 0x10, 0x95, 0x4d, 0x3c, 0x27, 0xdd, 0xcd,
0x23, 0x68, 0x6e, 0xaa, 0x26, 0x69, 0x9f, 0x7d, 0xbe, 0xd7, 0x12, 0x05, 0x8b, 0xa2, 0xa5, 0x68,
0x2b, 0x15, 0x5b, 0x1c, 0xe6, 0x50, 0xe4, 0x50, 0x55, 0xb8, 0x20, 0x8e, 0xc6, 0x16, 0x5f, 0x1b,
0x11, 0xc4, 0x94, 0xd8, 0x32, 0x06, 0x5f, 0x6c, 0xfc, 0x54, 0xa2, 0xff, 0x7a, 0x10, 0x41, 0x78,
0xa8, 0xbb, 0xfb, 0x5b, 0x03, 0xd5, 0x74, 0xb2, 0x88, 0x40, 0x11, 0x8f, 0x21, 0x02, 0xc7, 0x17,
0x22, 0x68, 0x50, 0x0f, 0xcd, 0x28, 0x51, 0xd2, 0xdb, 0x9c, 0xa4, 0x29, 0x69, 0x7b, 0x65, 0x32,
0xa2, 0xd8, 0x65, 0x1f, 0xea, 0x9e, 0xbd, 0x32, 0xe5, 0xbe, 0x9a, 0x26, 0xa9, 0x8f, 0x93, 0xe2,
0x6e, 0xc9, 0xa8, 0x41, 0x77, 0x8b, 0x8f, 0x60, 0x50, 0xb6, 0xd1, 0xdd, 0x38, 0x77, 0x4b, 0x9f,
0xeb, 0xb0, 0x78, 0x12, 0x19, 0x77, 0x26, 0x07, 0xab, 0xc4, 0xe9, 0x6d, 0x62, 0x3c, 0x49, 0x99,
0x9e, 0xca, 0x27, 0xd0, 0x43, 0x28, 0xb1, 0xce, 0x0b, 0xbd, 0x3c, 0xbe, 0x29, 0xe3, 0xdc, 0x2f,
0x4f, 0xc0, 0x12, 0x8c, 0x0f, 0xe0, 0x74, 0x4a, 0x4b, 0x96, 0x98, 0xe7, 0xc6, 0xb7, 0x6f, 0x93,
0xbc, 0x5c, 0x05, 0x14, 0x7f, 0xda, 0x9f, 0x78, 0x4d, 0xa9, 0xff, 0x01, 0x90, 0x61, 0x8b, 0xf8,
0xab, 0xfd, 0xc4, 0x2d, 0x52, 0x1f, 0xf7, 0x4f, 0xda, 0xe2, 0x01, 0x80, 0xdf, 0x78, 0x25, 0xde,
0x47, 0xac, 0x2f, 0x9b, 0xc0, 0x72, 0xf5, 0x6f, 0x25, 0x28, 0x7b, 0x2f, 0x61, 0x5e, 0x40, 0x71,
0xff, 0x02, 0xaa, 0xed, 0x4f, 0x61, 0x3e, 0xf2, 0xa4, 0x3e, 0xd1, 0x79, 0x92, 0x9f, 0xdd, 0x4f,
0xba, 0xa1, 0x87, 0xfc, 0x9f, 0x92, 0x45, 0xe2, 0x7d, 0x25, 0xad, 0x62, 0x8f, 0xe6, 0xdc, 0x09,
0x8c, 0x9f, 0x7b, 0x86, 0xbd, 0x0f, 0x10, 0xc8, 0x80, 0xe3, 0x3f, 0x4f, 0x92, 0xa0, 0x3e, 0x49,
0xe0, 0xcd, 0x63, 0xc6, 0x8d, 0x09, 0xec, 0x1c, 0xe2, 0x5d, 0xd1, 0x31, 0x47, 0x8a, 0x77, 0xa5,
0x0c, 0x57, 0x12, 0xe3, 0x6c, 0xfa, 0xec, 0xe4, 0xb9, 0xf8, 0xdb, 0xed, 0x9b, 0xff, 0xfe, 0x66,
0x5f, 0x77, 0xf7, 0x47, 0xbb, 0xe4, 0x97, 0x1b, 0x0c, 0xf5, 0x75, 0xdd, 0xe2, 0x7f, 0xdd, 0xf0,
0x0c, 0xfd, 0x06, 0xa5, 0xbe, 0x41, 0xf6, 0x18, 0xee, 0xee, 0x96, 0xe8, 0xea, 0xe6, 0x3f, 0x02,
0x00, 0x00, 0xff, 0xff, 0xa0, 0x19, 0x51, 0xc3, 0xf7, 0x3f, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -871,8 +871,8 @@ message GetImportStateResponse {
repeated int64 id_list = 4; // auto generated ids if the primary key is autoid
repeated common.KeyValuePair infos = 5; // more information about the task, progress percent, file path, failed reason, etc.
int64 id = 6; // id of an import task
bool heuristic_data_queryable = 7; // A flag indicating (heuristically) whether import data are queryable (i.e. loaded in query nodes)
bool heuristic_data_indexed = 8; // A flag indicating (heuristically) whether import data are indexed.
bool data_queryable = 7; // A flag indicating whether import data are queryable (i.e. loaded in query nodes)
bool data_indexed = 8; // A flag indicating whether import data are indexed.
}
message ListImportTasksRequest {

View File

@ -5100,17 +5100,17 @@ func (m *GetImportStateRequest) GetTask() int64 {
}
type GetImportStateResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
State commonpb.ImportState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.ImportState" json:"state,omitempty"`
RowCount int64 `protobuf:"varint,3,opt,name=row_count,json=rowCount,proto3" json:"row_count,omitempty"`
IdList []int64 `protobuf:"varint,4,rep,packed,name=id_list,json=idList,proto3" json:"id_list,omitempty"`
Infos []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=infos,proto3" json:"infos,omitempty"`
Id int64 `protobuf:"varint,6,opt,name=id,proto3" json:"id,omitempty"`
HeuristicDataQueryable bool `protobuf:"varint,7,opt,name=heuristic_data_queryable,json=heuristicDataQueryable,proto3" json:"heuristic_data_queryable,omitempty"`
HeuristicDataIndexed bool `protobuf:"varint,8,opt,name=heuristic_data_indexed,json=heuristicDataIndexed,proto3" json:"heuristic_data_indexed,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
State commonpb.ImportState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.ImportState" json:"state,omitempty"`
RowCount int64 `protobuf:"varint,3,opt,name=row_count,json=rowCount,proto3" json:"row_count,omitempty"`
IdList []int64 `protobuf:"varint,4,rep,packed,name=id_list,json=idList,proto3" json:"id_list,omitempty"`
Infos []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=infos,proto3" json:"infos,omitempty"`
Id int64 `protobuf:"varint,6,opt,name=id,proto3" json:"id,omitempty"`
DataQueryable bool `protobuf:"varint,7,opt,name=data_queryable,json=dataQueryable,proto3" json:"data_queryable,omitempty"`
DataIndexed bool `protobuf:"varint,8,opt,name=data_indexed,json=dataIndexed,proto3" json:"data_indexed,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *GetImportStateResponse) Reset() { *m = GetImportStateResponse{} }
@ -5180,16 +5180,16 @@ func (m *GetImportStateResponse) GetId() int64 {
return 0
}
func (m *GetImportStateResponse) GetHeuristicDataQueryable() bool {
func (m *GetImportStateResponse) GetDataQueryable() bool {
if m != nil {
return m.HeuristicDataQueryable
return m.DataQueryable
}
return false
}
func (m *GetImportStateResponse) GetHeuristicDataIndexed() bool {
func (m *GetImportStateResponse) GetDataIndexed() bool {
if m != nil {
return m.HeuristicDataIndexed
return m.DataIndexed
}
return false
}
@ -7125,333 +7125,332 @@ func init() {
func init() { proto.RegisterFile("milvus.proto", fileDescriptor_02345ba45cc0e303) }
var fileDescriptor_02345ba45cc0e303 = []byte{
// 5207 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x7d, 0xdd, 0x6f, 0x1c, 0x47,
0x72, 0x38, 0x67, 0x97, 0xfb, 0x55, 0xfb, 0xc1, 0x65, 0xf3, 0x6b, 0xbd, 0x92, 0x2c, 0x6a, 0x6c,
0xd9, 0xb4, 0x74, 0xa6, 0x6c, 0xca, 0x92, 0x7d, 0xb2, 0xcf, 0xb6, 0x28, 0x5a, 0x12, 0x61, 0x7d,
0xd0, 0x43, 0xf9, 0x7e, 0xb8, 0x9f, 0x21, 0x4c, 0x86, 0x3b, 0xcd, 0xe5, 0x58, 0xb3, 0x33, 0xab,
0x99, 0x59, 0x52, 0xf4, 0x43, 0x60, 0xe4, 0x0e, 0xf9, 0x40, 0x9c, 0x3b, 0x1c, 0x72, 0x88, 0x71,
0x0f, 0x09, 0x72, 0xf9, 0x7a, 0x4b, 0x90, 0xbb, 0x7b, 0x08, 0x90, 0x04, 0x48, 0x02, 0xdc, 0x43,
0x60, 0x04, 0xc8, 0xe5, 0x29, 0x08, 0xfc, 0x07, 0x04, 0x79, 0x4b, 0x80, 0xcb, 0x5b, 0x02, 0x04,
0xfd, 0x31, 0xb3, 0x3d, 0xb3, 0x3d, 0xcb, 0x25, 0xf7, 0x64, 0x51, 0x6f, 0x3b, 0xd5, 0x55, 0xdd,
0xd5, 0xd5, 0xd5, 0xd5, 0xd5, 0x5d, 0xd5, 0xbd, 0x50, 0xe9, 0x58, 0xf6, 0x6e, 0xcf, 0x5f, 0xee,
0x7a, 0x6e, 0xe0, 0xa2, 0x19, 0xf1, 0x6b, 0x99, 0x7d, 0x34, 0x2b, 0x2d, 0xb7, 0xd3, 0x71, 0x1d,
0x06, 0x6c, 0x56, 0xfc, 0xd6, 0x0e, 0xee, 0x18, 0xfc, 0x6b, 0xb1, 0xed, 0xba, 0x6d, 0x1b, 0x5f,
0xa0, 0x5f, 0x5b, 0xbd, 0xed, 0x0b, 0x26, 0xf6, 0x5b, 0x9e, 0xd5, 0x0d, 0x5c, 0x8f, 0x61, 0xa8,
0x7f, 0xa0, 0x00, 0xba, 0xe6, 0x61, 0x23, 0xc0, 0x57, 0x6d, 0xcb, 0xf0, 0x35, 0xfc, 0xb0, 0x87,
0xfd, 0x00, 0xbd, 0x02, 0x93, 0x5b, 0x86, 0x8f, 0x1b, 0xca, 0xa2, 0xb2, 0x54, 0x5e, 0x39, 0xb9,
0x1c, 0x6b, 0x98, 0x37, 0x78, 0xdb, 0x6f, 0xaf, 0x1a, 0x3e, 0xd6, 0x28, 0x26, 0x5a, 0x80, 0x82,
0xb9, 0xa5, 0x3b, 0x46, 0x07, 0x37, 0x32, 0x8b, 0xca, 0x52, 0x49, 0xcb, 0x9b, 0x5b, 0x77, 0x8c,
0x0e, 0x46, 0x2f, 0xc2, 0x54, 0xcb, 0xb5, 0x6d, 0xdc, 0x0a, 0x2c, 0xd7, 0x61, 0x08, 0x59, 0x8a,
0x50, 0xeb, 0x83, 0x29, 0xe2, 0x2c, 0xe4, 0x0c, 0xc2, 0x43, 0x63, 0x92, 0x16, 0xb3, 0x0f, 0xd5,
0x87, 0xfa, 0x9a, 0xe7, 0x76, 0x1f, 0x17, 0x77, 0x51, 0xa3, 0x59, 0xb1, 0xd1, 0xdf, 0x57, 0x60,
0xfa, 0xaa, 0x1d, 0x60, 0xef, 0x98, 0x0a, 0xe5, 0x47, 0x19, 0x58, 0x60, 0xa3, 0x76, 0x2d, 0x42,
0x7f, 0x92, 0x5c, 0xce, 0x43, 0x9e, 0xe9, 0x1d, 0x65, 0xb3, 0xa2, 0xf1, 0x2f, 0x74, 0x0a, 0xc0,
0xdf, 0x31, 0x3c, 0xd3, 0xd7, 0x9d, 0x5e, 0xa7, 0x91, 0x5b, 0x54, 0x96, 0x72, 0x5a, 0x89, 0x41,
0xee, 0xf4, 0x3a, 0x48, 0x83, 0xe9, 0x96, 0xeb, 0xf8, 0x96, 0x1f, 0x60, 0xa7, 0xb5, 0xaf, 0xdb,
0x78, 0x17, 0xdb, 0x8d, 0xfc, 0xa2, 0xb2, 0x54, 0x5b, 0x39, 0x2b, 0xe5, 0xfb, 0x5a, 0x1f, 0xfb,
0x16, 0x41, 0xd6, 0xea, 0xad, 0x04, 0xe4, 0x4a, 0xe1, 0x8b, 0xb7, 0x27, 0xeb, 0x4a, 0x23, 0xab,
0xfe, 0x40, 0x81, 0x39, 0xa2, 0x39, 0xc7, 0x42, 0x42, 0x8c, 0xad, 0x4c, 0x23, 0xab, 0xfe, 0x44,
0x81, 0xd9, 0x9b, 0x86, 0x7f, 0x3c, 0xc6, 0xed, 0x14, 0x40, 0x60, 0x75, 0xb0, 0xee, 0x07, 0x46,
0xa7, 0x4b, 0xc7, 0x6e, 0x52, 0x2b, 0x11, 0xc8, 0x26, 0x01, 0x30, 0xa6, 0x27, 0x1b, 0x59, 0xf5,
0x5b, 0x50, 0x59, 0x75, 0x5d, 0x5b, 0xc3, 0x7e, 0xd7, 0x75, 0x7c, 0x8c, 0x2e, 0x42, 0xde, 0x0f,
0x8c, 0xa0, 0xe7, 0x73, 0x6e, 0x4f, 0x48, 0xb9, 0xdd, 0xa4, 0x28, 0x1a, 0x47, 0x25, 0xaa, 0xbc,
0x6b, 0xd8, 0x3d, 0xc6, 0x6c, 0x51, 0x63, 0x1f, 0xea, 0x47, 0x50, 0xdb, 0x0c, 0x3c, 0xcb, 0x69,
0xff, 0x12, 0x2b, 0x2f, 0x85, 0x95, 0x7f, 0xa9, 0xc0, 0x33, 0x6b, 0xd4, 0xe4, 0x6d, 0x1d, 0x93,
0x99, 0xa2, 0x42, 0xa5, 0x0f, 0x59, 0x5f, 0xa3, 0x32, 0xcf, 0x6a, 0x31, 0x58, 0x62, 0x54, 0x72,
0xa9, 0xa3, 0xf2, 0x69, 0x0e, 0x9a, 0xb2, 0xde, 0x8d, 0x23, 0xc7, 0x6f, 0x44, 0x33, 0x39, 0x43,
0x89, 0x12, 0xf3, 0x90, 0xaf, 0x2e, 0xfd, 0xd6, 0x36, 0x29, 0x20, 0x9a, 0xf0, 0xc9, 0xee, 0x65,
0x25, 0xdd, 0x5b, 0x81, 0xb9, 0x5d, 0xcb, 0x0b, 0x7a, 0x86, 0xad, 0xb7, 0x76, 0x0c, 0xc7, 0xc1,
0x36, 0x15, 0x18, 0x31, 0x71, 0xd9, 0xa5, 0x92, 0x36, 0xc3, 0x0b, 0xaf, 0xb1, 0x32, 0x22, 0x35,
0x1f, 0xbd, 0x06, 0xf3, 0xdd, 0x9d, 0x7d, 0xdf, 0x6a, 0x0d, 0x10, 0xe5, 0x28, 0xd1, 0x6c, 0x58,
0x1a, 0xa3, 0x3a, 0x0f, 0xd3, 0x2d, 0x6a, 0x25, 0x4d, 0x9d, 0x88, 0x8f, 0xc9, 0x33, 0x4f, 0xe5,
0x59, 0xe7, 0x05, 0xf7, 0x42, 0x38, 0x61, 0x2b, 0x44, 0xee, 0x05, 0x2d, 0x81, 0xa0, 0x40, 0x09,
0x66, 0x78, 0xe1, 0x87, 0x41, 0xab, 0x4f, 0x13, 0xb7, 0x6f, 0xc5, 0xa4, 0x7d, 0x6b, 0x40, 0x81,
0xda, 0x6b, 0xec, 0x37, 0x4a, 0x94, 0xcd, 0xf0, 0x13, 0xad, 0xc3, 0x94, 0x1f, 0x18, 0x5e, 0xa0,
0x77, 0x5d, 0xdf, 0x22, 0x72, 0xf1, 0x1b, 0xb0, 0x98, 0x5d, 0x2a, 0xaf, 0x2c, 0x4a, 0x07, 0xe9,
0x7d, 0xbc, 0xbf, 0x66, 0x04, 0xc6, 0x86, 0x61, 0x79, 0x5a, 0x8d, 0x12, 0x6e, 0x84, 0x74, 0x72,
0x23, 0x5a, 0x1e, 0xcb, 0x88, 0xca, 0xd4, 0xb9, 0x22, 0x53, 0x67, 0xf5, 0x6f, 0x14, 0x98, 0xbb,
0xe5, 0x1a, 0xe6, 0xf1, 0x98, 0x5c, 0x67, 0xa1, 0xe6, 0xe1, 0xae, 0x6d, 0xb5, 0x0c, 0x32, 0x1e,
0x5b, 0xd8, 0xa3, 0xd3, 0x2b, 0xa7, 0x55, 0x39, 0xf4, 0x0e, 0x05, 0xb2, 0x09, 0x94, 0x6b, 0x64,
0xd5, 0xcf, 0x15, 0x68, 0x68, 0xd8, 0xc6, 0x86, 0x8f, 0x8f, 0xd1, 0x2a, 0x91, 0x6f, 0x64, 0x89,
0x03, 0xf2, 0xec, 0x0d, 0x1c, 0x08, 0xf3, 0x2c, 0x30, 0x02, 0xcb, 0x0f, 0xac, 0x96, 0xff, 0xc4,
0xf9, 0x23, 0xa6, 0xe7, 0x7b, 0x0a, 0x9c, 0x4e, 0xe5, 0x6f, 0x1c, 0xfb, 0xf3, 0x3a, 0xe4, 0xc8,
0x2f, 0xbf, 0x91, 0xa1, 0xd3, 0xe1, 0x4c, 0xda, 0x74, 0xf8, 0x26, 0xb1, 0xef, 0x74, 0x3e, 0x30,
0x7c, 0xf5, 0xdf, 0x15, 0x98, 0xdf, 0xdc, 0x71, 0xf7, 0xfa, 0x2c, 0x3d, 0x0e, 0x49, 0xc5, 0x4d,
0x73, 0x36, 0x61, 0x9a, 0xd1, 0xab, 0x30, 0x19, 0xec, 0x77, 0x31, 0x55, 0xbb, 0xda, 0xca, 0xa9,
0x65, 0x89, 0xbf, 0xbe, 0x4c, 0x98, 0xbc, 0xb7, 0xdf, 0xc5, 0x1a, 0x45, 0x45, 0x2f, 0x41, 0x3d,
0x21, 0xfb, 0xd0, 0xa6, 0x4d, 0xc5, 0x85, 0xef, 0x5f, 0xc9, 0x7d, 0xf1, 0x76, 0xa6, 0x3e, 0xa9,
0xfe, 0x67, 0x06, 0x16, 0x06, 0x7a, 0x3a, 0x8e, 0xcc, 0x65, 0x2c, 0x64, 0xa4, 0x2c, 0x90, 0x19,
0x26, 0xa0, 0x5a, 0x26, 0xf1, 0x9b, 0xb3, 0x4b, 0x59, 0xad, 0x2a, 0x58, 0x78, 0xd3, 0x47, 0x2f,
0x03, 0x1a, 0x30, 0xbc, 0xcc, 0xbe, 0x4f, 0x6a, 0xd3, 0x49, 0xcb, 0x4b, 0xad, 0xbb, 0xd4, 0xf4,
0x32, 0x49, 0x4c, 0x6a, 0xb3, 0x12, 0xdb, 0xeb, 0xa3, 0x57, 0x61, 0xd6, 0x72, 0x6e, 0xe3, 0x8e,
0xeb, 0xed, 0xeb, 0x5d, 0xec, 0xb5, 0xb0, 0x13, 0x18, 0x6d, 0xec, 0x37, 0xf2, 0x94, 0xa3, 0x99,
0xb0, 0x6c, 0xa3, 0x5f, 0x84, 0x2e, 0xc3, 0xc2, 0xc3, 0x1e, 0xf6, 0xf6, 0x75, 0x1f, 0x7b, 0xbb,
0x56, 0x0b, 0xeb, 0xc6, 0xae, 0x61, 0xd9, 0xc6, 0x96, 0x8d, 0x1b, 0x85, 0xc5, 0xec, 0x52, 0x51,
0x9b, 0xa3, 0xc5, 0x9b, 0xac, 0xf4, 0x6a, 0x58, 0xa8, 0xfe, 0x54, 0x81, 0x79, 0xe6, 0x6f, 0x6f,
0x18, 0x5e, 0x60, 0x1d, 0x03, 0x3b, 0xd7, 0x0d, 0xf9, 0x60, 0x78, 0x6c, 0x77, 0x50, 0x8d, 0xa0,
0xd4, 0x38, 0xff, 0x58, 0x81, 0x59, 0xe2, 0x01, 0x3f, 0x4d, 0x3c, 0xff, 0xa5, 0x02, 0x33, 0x37,
0x0d, 0xff, 0x69, 0x62, 0xf9, 0x4b, 0xbe, 0x06, 0x46, 0x3c, 0x3f, 0xd1, 0x0d, 0xe3, 0x8b, 0x30,
0x15, 0x67, 0x3a, 0xf4, 0xab, 0x6a, 0x31, 0xae, 0x7d, 0xc9, 0x62, 0x99, 0x93, 0x2c, 0x96, 0xea,
0x5f, 0xf5, 0xd7, 0xc8, 0xa7, 0xab, 0x83, 0xea, 0x5f, 0x2b, 0x70, 0xea, 0x06, 0x0e, 0x22, 0xae,
0x8f, 0xc5, 0x12, 0x3a, 0xaa, 0x52, 0x7d, 0x97, 0x39, 0x00, 0x52, 0xe6, 0x9f, 0xc8, 0xfa, 0xfa,
0xdb, 0x19, 0x98, 0x23, 0xab, 0xce, 0xf1, 0x50, 0x82, 0x51, 0xb6, 0x51, 0x12, 0x45, 0xc9, 0x49,
0x67, 0x42, 0xb8, 0x6a, 0xe7, 0x47, 0x5e, 0xb5, 0xd5, 0x9f, 0x64, 0x98, 0xb7, 0x21, 0x4a, 0x63,
0x9c, 0x61, 0x91, 0xf0, 0x9a, 0x91, 0xf2, 0xaa, 0x42, 0x25, 0x82, 0xac, 0xaf, 0x85, 0xcb, 0x6f,
0x0c, 0x76, 0x5c, 0x57, 0x5f, 0xf5, 0x33, 0x05, 0xe6, 0xc3, 0xfd, 0xea, 0x26, 0x6e, 0x77, 0xb0,
0x13, 0x1c, 0x5d, 0x87, 0x92, 0x1a, 0x90, 0x91, 0x68, 0xc0, 0x49, 0x28, 0xf9, 0xac, 0x9d, 0x68,
0x2b, 0xda, 0x07, 0xa8, 0x7f, 0xa7, 0xc0, 0xc2, 0x00, 0x3b, 0xe3, 0x0c, 0x62, 0x03, 0x0a, 0x96,
0x63, 0xe2, 0x47, 0x11, 0x37, 0xe1, 0x27, 0x29, 0xd9, 0xea, 0x59, 0xb6, 0x19, 0xb1, 0x11, 0x7e,
0xa2, 0x33, 0x50, 0xc1, 0x0e, 0xf1, 0x31, 0x74, 0x8a, 0x4b, 0x15, 0xb9, 0xa8, 0x95, 0x19, 0x6c,
0x9d, 0x80, 0x08, 0xf1, 0xb6, 0x85, 0x29, 0x71, 0x8e, 0x11, 0xf3, 0x4f, 0xf5, 0x77, 0x14, 0x98,
0x21, 0x5a, 0xc8, 0xb9, 0xf7, 0x1f, 0xaf, 0x34, 0x17, 0xa1, 0x2c, 0xa8, 0x19, 0xef, 0x88, 0x08,
0x52, 0x1f, 0xc0, 0x6c, 0x9c, 0x9d, 0x71, 0xa4, 0xf9, 0x2c, 0x40, 0x34, 0x56, 0x6c, 0x36, 0x64,
0x35, 0x01, 0xa2, 0x7e, 0x96, 0x09, 0x4f, 0xae, 0xa9, 0x98, 0x9e, 0xf0, 0x31, 0x1a, 0x1d, 0x12,
0xd1, 0x9e, 0x97, 0x28, 0x84, 0x16, 0xaf, 0x41, 0x05, 0x3f, 0x0a, 0x3c, 0x43, 0xef, 0x1a, 0x9e,
0xd1, 0x61, 0xd3, 0x6a, 0x24, 0xd3, 0x5b, 0xa6, 0x64, 0x1b, 0x94, 0x8a, 0x34, 0x42, 0x55, 0x84,
0x35, 0x92, 0x67, 0x8d, 0x50, 0x08, 0x5d, 0x30, 0xfe, 0x91, 0x38, 0x7b, 0x5c, 0x9b, 0x8f, 0xbb,
0x40, 0xe2, 0x5d, 0xc9, 0x25, 0xbb, 0xf2, 0xa7, 0x0a, 0xd4, 0x69, 0x17, 0xd6, 0x78, 0xb4, 0xc2,
0x72, 0x9d, 0x04, 0x8d, 0x92, 0xa0, 0x19, 0x32, 0xf7, 0xbe, 0x0e, 0x79, 0x2e, 0xf7, 0xec, 0xa8,
0x72, 0xe7, 0x04, 0x07, 0x74, 0x43, 0xfd, 0x23, 0x05, 0xe6, 0x12, 0x22, 0x1f, 0x47, 0xe1, 0xef,
0x01, 0x62, 0x3d, 0x34, 0xfb, 0xdd, 0x0e, 0xd7, 0xe9, 0xb3, 0xd2, 0x45, 0x29, 0x29, 0x24, 0x6d,
0xda, 0x4a, 0x40, 0x7c, 0xf5, 0xe7, 0x0a, 0x9c, 0xbc, 0x81, 0x03, 0x8a, 0xba, 0x4a, 0x8c, 0xce,
0x86, 0xe7, 0xb6, 0x3d, 0xec, 0xfb, 0x4f, 0xaf, 0x7e, 0xfc, 0x1e, 0x73, 0xec, 0x64, 0x5d, 0x1a,
0x47, 0xfe, 0x67, 0xa0, 0x42, 0xdb, 0xc0, 0xa6, 0xee, 0xb9, 0x7b, 0x3e, 0xd7, 0xa3, 0x32, 0x87,
0x69, 0xee, 0x1e, 0x55, 0x88, 0xc0, 0x0d, 0x0c, 0x9b, 0x21, 0xf0, 0x15, 0x85, 0x42, 0x48, 0x31,
0x9d, 0x83, 0x21, 0x63, 0xa4, 0x72, 0xfc, 0xf4, 0xca, 0xf8, 0x4f, 0x14, 0x98, 0x4b, 0x74, 0x65,
0x1c, 0xd9, 0x5e, 0x62, 0x6e, 0x27, 0xeb, 0x4c, 0x6d, 0xe5, 0xb4, 0x94, 0x46, 0x68, 0x8c, 0x61,
0xa3, 0xd3, 0x50, 0xde, 0x36, 0x2c, 0x5b, 0xf7, 0xb0, 0xe1, 0xbb, 0x0e, 0xef, 0x28, 0x10, 0x90,
0x46, 0x21, 0xea, 0xcf, 0x14, 0x16, 0x1e, 0x7c, 0xca, 0x2d, 0xde, 0x1f, 0x67, 0xa0, 0xba, 0xee,
0xf8, 0xd8, 0x0b, 0x8e, 0xff, 0xd6, 0x04, 0xbd, 0x03, 0x65, 0xda, 0x31, 0x5f, 0x37, 0x8d, 0xc0,
0xe0, 0xab, 0xd9, 0xb3, 0xd2, 0x38, 0xc1, 0x75, 0x82, 0xb7, 0x66, 0x04, 0x86, 0xc6, 0xa4, 0xe3,
0x93, 0xdf, 0xe8, 0x04, 0x94, 0x76, 0x0c, 0x7f, 0x47, 0x7f, 0x80, 0xf7, 0x99, 0xbf, 0x58, 0xd5,
0x8a, 0x04, 0xf0, 0x3e, 0xde, 0xf7, 0xd1, 0x33, 0x50, 0x74, 0x7a, 0x1d, 0x36, 0xc1, 0x0a, 0x8b,
0xca, 0x52, 0x55, 0x2b, 0x38, 0xbd, 0x0e, 0x9d, 0x5e, 0xff, 0x94, 0x81, 0xda, 0xed, 0x1e, 0xd9,
0x08, 0xd1, 0x28, 0x47, 0xcf, 0x0e, 0x8e, 0xa6, 0x8c, 0xe7, 0x20, 0xcb, 0x5c, 0x0a, 0x42, 0xd1,
0x90, 0x32, 0xbe, 0xbe, 0xe6, 0x6b, 0x04, 0x89, 0x9e, 0xf0, 0xf7, 0x5a, 0x2d, 0xee, 0x9d, 0x65,
0x29, 0xb3, 0x25, 0x02, 0x61, 0xbe, 0xd9, 0x09, 0x28, 0x61, 0xcf, 0x8b, 0x7c, 0x37, 0xda, 0x15,
0xec, 0x79, 0xac, 0x50, 0x85, 0x8a, 0xd1, 0x7a, 0xe0, 0xb8, 0x7b, 0x36, 0x36, 0xdb, 0xd8, 0xa4,
0xc3, 0x5e, 0xd4, 0x62, 0x30, 0xa6, 0x18, 0x64, 0xe0, 0xf5, 0x96, 0x13, 0xd0, 0x55, 0x3d, 0x4b,
0x14, 0x83, 0x40, 0xae, 0x39, 0x01, 0x29, 0x36, 0xb1, 0x8d, 0x03, 0x4c, 0x8b, 0x0b, 0xac, 0x98,
0x41, 0x78, 0x71, 0xaf, 0x1b, 0x51, 0x17, 0x59, 0x31, 0x83, 0x90, 0xe2, 0x93, 0x50, 0xea, 0x87,
0x31, 0x4a, 0xfd, 0xc3, 0x4a, 0x0a, 0x50, 0xbf, 0x54, 0xa0, 0xba, 0x46, 0xab, 0x7a, 0x0a, 0x94,
0x0e, 0xc1, 0x24, 0x7e, 0xd4, 0xf5, 0xf8, 0xd4, 0xa1, 0xbf, 0x87, 0xea, 0x91, 0xfa, 0xbf, 0x59,
0xa8, 0x6e, 0x62, 0xc3, 0x6b, 0xed, 0x3c, 0x15, 0xa7, 0x31, 0x75, 0xc8, 0x9a, 0xbe, 0xcd, 0xbb,
0x47, 0x7e, 0xa2, 0xf3, 0x30, 0xdd, 0xb5, 0x8d, 0x16, 0xde, 0x71, 0x6d, 0x13, 0x7b, 0x7a, 0xdb,
0x73, 0x7b, 0x2c, 0x78, 0x55, 0xd1, 0xea, 0x42, 0xc1, 0x0d, 0x02, 0x47, 0xaf, 0x43, 0xd1, 0xf4,
0x6d, 0x9d, 0x6e, 0x63, 0x0b, 0xd4, 0xc4, 0xca, 0xfb, 0xb7, 0xe6, 0xdb, 0x74, 0x17, 0x5b, 0x30,
0xd9, 0x0f, 0xf4, 0x1c, 0x54, 0xdd, 0x5e, 0xd0, 0xed, 0x05, 0x3a, 0x9b, 0xa0, 0x8d, 0x22, 0x65,
0xaf, 0xc2, 0x80, 0x74, 0xfe, 0xfa, 0xe8, 0x3a, 0x54, 0x7d, 0x2a, 0xca, 0xd0, 0x83, 0x2d, 0x8d,
0xea, 0x49, 0x55, 0x18, 0x1d, 0x77, 0x61, 0x5f, 0x82, 0x7a, 0xe0, 0x19, 0xbb, 0xd8, 0x16, 0xa2,
0x6b, 0x40, 0xd5, 0x72, 0x8a, 0xc1, 0xfb, 0x91, 0xb5, 0x0b, 0x30, 0xd3, 0xee, 0x19, 0x9e, 0xe1,
0x04, 0x18, 0x0b, 0xd8, 0x65, 0x8a, 0x8d, 0xa2, 0xa2, 0x3e, 0x41, 0x0d, 0x32, 0xce, 0x43, 0x1a,
0xa5, 0xca, 0x6a, 0x19, 0xe7, 0xa1, 0xfa, 0x3e, 0x4c, 0xde, 0xb4, 0x02, 0x2a, 0x58, 0x32, 0xd9,
0x15, 0xba, 0x7f, 0xa0, 0x53, 0xfa, 0x19, 0x28, 0x7a, 0xee, 0x1e, 0x33, 0x5e, 0xc4, 0xbb, 0xaa,
0x68, 0x05, 0xcf, 0xdd, 0xa3, 0x96, 0x89, 0xe6, 0x31, 0xb8, 0x1e, 0x66, 0xbe, 0x62, 0x46, 0xe3,
0x5f, 0xea, 0x5f, 0x28, 0x7d, 0x65, 0x22, 0x76, 0xc7, 0x3f, 0x9a, 0xe1, 0x79, 0x07, 0x0a, 0x1e,
0xa3, 0x1f, 0x1a, 0x5d, 0x15, 0x5b, 0xa2, 0xc6, 0x33, 0xa4, 0x1a, 0x59, 0xef, 0xd4, 0xef, 0x28,
0x50, 0xb9, 0x6e, 0xf7, 0xfc, 0xc7, 0xa1, 0xfc, 0xb2, 0x70, 0x41, 0x56, 0x1a, 0x2e, 0x50, 0xbf,
0x9f, 0x81, 0x2a, 0x67, 0x63, 0x1c, 0xef, 0x21, 0x95, 0x95, 0x4d, 0x28, 0x93, 0x26, 0x75, 0x1f,
0xb7, 0xc3, 0xc3, 0x90, 0xf2, 0xca, 0x8a, 0xd4, 0x57, 0x8e, 0xb1, 0x41, 0x03, 0xd8, 0x9b, 0x94,
0xe8, 0x3d, 0x27, 0xf0, 0xf6, 0x35, 0x68, 0x45, 0x80, 0xe6, 0x7d, 0x98, 0x4a, 0x14, 0x13, 0x25,
0x7a, 0x80, 0xf7, 0xf9, 0xae, 0x83, 0xfc, 0x44, 0xaf, 0x89, 0xf9, 0x06, 0x69, 0xcb, 0xdf, 0x2d,
0xd7, 0x69, 0x5f, 0xf5, 0x3c, 0x63, 0x9f, 0xe7, 0x23, 0x5c, 0xc9, 0xbc, 0xa1, 0xa8, 0x7f, 0x9f,
0x81, 0xca, 0x07, 0x3d, 0xec, 0xed, 0x3f, 0x49, 0xbb, 0x14, 0x9a, 0xd3, 0x49, 0xc1, 0x9c, 0x0e,
0x98, 0x82, 0x9c, 0xc4, 0x14, 0x48, 0x0c, 0x5a, 0x5e, 0x6a, 0xd0, 0x64, 0x73, 0xbd, 0x70, 0xa8,
0xb9, 0x5e, 0x4c, 0x9b, 0xeb, 0xea, 0x9f, 0x2b, 0x91, 0x08, 0xc7, 0x9a, 0x8d, 0x31, 0x3f, 0x26,
0x73, 0x68, 0x3f, 0x66, 0xe4, 0xd9, 0xf8, 0x63, 0x05, 0x4a, 0xdf, 0xc4, 0xad, 0xc0, 0xf5, 0x88,
0xfd, 0x91, 0x90, 0x29, 0x23, 0xf8, 0x94, 0x99, 0xa4, 0x4f, 0x79, 0x11, 0x8a, 0x96, 0xa9, 0x1b,
0x44, 0xbf, 0x68, 0xbb, 0xc3, 0x7c, 0x99, 0x82, 0x65, 0x52, 0x45, 0x1c, 0xfd, 0xf4, 0xfc, 0x73,
0x05, 0x2a, 0x8c, 0x67, 0x9f, 0x51, 0xbe, 0x29, 0x34, 0xa7, 0xc8, 0x94, 0x9e, 0x7f, 0x44, 0x1d,
0xbd, 0x39, 0xd1, 0x6f, 0xf6, 0x2a, 0x00, 0x11, 0x32, 0x27, 0x67, 0x73, 0x66, 0x51, 0xca, 0x2d,
0x23, 0xa7, 0x02, 0xbf, 0x39, 0xa1, 0x95, 0x08, 0x15, 0xad, 0x62, 0xb5, 0x00, 0x39, 0x4a, 0xad,
0xfe, 0x8f, 0x02, 0x33, 0xd7, 0x0c, 0xbb, 0xb5, 0x66, 0xf9, 0x81, 0xe1, 0xb4, 0xc6, 0xf0, 0x5e,
0xae, 0x40, 0xc1, 0xed, 0xea, 0x36, 0xde, 0x0e, 0x38, 0x4b, 0x67, 0x86, 0xf4, 0x88, 0x89, 0x41,
0xcb, 0xbb, 0xdd, 0x5b, 0x78, 0x3b, 0x40, 0x6f, 0x41, 0xd1, 0xed, 0xea, 0x9e, 0xd5, 0xde, 0x09,
0xb8, 0xf4, 0x47, 0x20, 0x2e, 0xb8, 0x5d, 0x8d, 0x50, 0x08, 0x87, 0x12, 0x93, 0x87, 0x3c, 0x94,
0x50, 0xff, 0x65, 0xa0, 0xfb, 0x63, 0xcc, 0x81, 0x2b, 0x50, 0xb4, 0x9c, 0x40, 0x37, 0x2d, 0x3f,
0x14, 0xc1, 0x29, 0xb9, 0x0e, 0x39, 0x01, 0xed, 0x01, 0x1d, 0x53, 0x27, 0x20, 0x6d, 0xa3, 0x77,
0x01, 0xb6, 0x6d, 0xd7, 0xe0, 0xd4, 0x4c, 0x06, 0xa7, 0xe5, 0xd3, 0x87, 0xa0, 0x85, 0xf4, 0x25,
0x4a, 0x44, 0x6a, 0xe8, 0x0f, 0xe9, 0x3f, 0x2b, 0x30, 0xb7, 0x81, 0x3d, 0x96, 0x84, 0x12, 0xf0,
0xf3, 0xc3, 0x75, 0x67, 0xdb, 0x8d, 0x1f, 0xe1, 0x2a, 0x89, 0x23, 0xdc, 0x5f, 0xce, 0xb1, 0x65,
0x6c, 0xcb, 0xc1, 0x02, 0x09, 0xe1, 0x96, 0x23, 0x0c, 0x97, 0xb0, 0x2d, 0x5b, 0x2d, 0x65, 0x98,
0x38, 0xbf, 0xe2, 0xce, 0x55, 0xfd, 0x5d, 0x96, 0x20, 0x21, 0xed, 0xd4, 0xd1, 0x15, 0x76, 0x1e,
0xb8, 0xa5, 0x4f, 0xd8, 0xfd, 0x17, 0x20, 0x61, 0x3b, 0x52, 0x0c, 0xd1, 0x0f, 0x15, 0x58, 0x4c,
0xe7, 0x6a, 0x9c, 0x25, 0xfa, 0x5d, 0xc8, 0x59, 0xce, 0xb6, 0x1b, 0x9e, 0x57, 0x9d, 0x93, 0xce,
0x05, 0x79, 0xbb, 0x8c, 0x50, 0xfd, 0x79, 0x06, 0xea, 0x1f, 0xb0, 0xe8, 0xfb, 0x57, 0x3e, 0xfc,
0x1d, 0xdc, 0xd1, 0x7d, 0xeb, 0x13, 0x1c, 0x0e, 0x7f, 0x07, 0x77, 0x36, 0xad, 0x4f, 0x70, 0x4c,
0x33, 0x72, 0x71, 0xcd, 0x18, 0x7e, 0x1c, 0x2b, 0x9e, 0x47, 0x16, 0xe2, 0xe7, 0x91, 0xf3, 0x90,
0x77, 0x5c, 0x13, 0xaf, 0xaf, 0xf1, 0xfd, 0x1a, 0xff, 0xea, 0xab, 0x5a, 0xe9, 0x70, 0xaa, 0x46,
0x9a, 0xa2, 0x55, 0x98, 0x2c, 0x87, 0x8c, 0xf0, 0xc8, 0x3e, 0xd5, 0xef, 0x2a, 0xd0, 0xbc, 0x81,
0x83, 0xa4, 0x54, 0x9f, 0x9c, 0xfe, 0x7d, 0x4f, 0x81, 0x13, 0x52, 0x86, 0xc6, 0x51, 0xbd, 0x37,
0xe3, 0xaa, 0x27, 0x3f, 0x2a, 0x1d, 0x68, 0x92, 0x6b, 0xdd, 0xab, 0x50, 0x59, 0xeb, 0x75, 0x3a,
0x91, 0x33, 0x76, 0x06, 0x2a, 0x1e, 0xfb, 0xc9, 0x36, 0x53, 0x6c, 0x65, 0x2e, 0x73, 0x18, 0xd9,
0x32, 0xa9, 0xe7, 0xa1, 0xca, 0x49, 0x38, 0xd7, 0x4d, 0x28, 0x7a, 0xfc, 0x37, 0xc7, 0x8f, 0xbe,
0xd5, 0x39, 0x98, 0xd1, 0x70, 0x9b, 0x28, 0xbd, 0x77, 0xcb, 0x72, 0x1e, 0xf0, 0x66, 0xd4, 0x6f,
0x2b, 0x30, 0x1b, 0x87, 0xf3, 0xba, 0x2e, 0x43, 0xc1, 0x30, 0x4d, 0x0f, 0xfb, 0xfe, 0xd0, 0x61,
0xb9, 0xca, 0x70, 0xb4, 0x10, 0x59, 0x90, 0x5c, 0x66, 0x64, 0xc9, 0xa9, 0x3a, 0x4c, 0xdf, 0xc0,
0xc1, 0x6d, 0x1c, 0x78, 0x63, 0x05, 0xc5, 0x1b, 0x64, 0x5b, 0x43, 0x89, 0xb9, 0x5a, 0x84, 0x9f,
0xea, 0x67, 0x0a, 0x20, 0xb1, 0x85, 0x71, 0x86, 0x59, 0x94, 0x72, 0x26, 0x2e, 0x65, 0x96, 0x96,
0xd4, 0xe9, 0xba, 0x0e, 0x76, 0x02, 0xd1, 0x11, 0xab, 0x46, 0xd0, 0x30, 0x53, 0x03, 0xdd, 0x72,
0x0d, 0x73, 0xd5, 0xb0, 0xc7, 0x73, 0x1c, 0x4e, 0x01, 0xf8, 0x5e, 0x4b, 0xe7, 0xf3, 0x38, 0xc3,
0xed, 0x92, 0xd7, 0xba, 0xc3, 0xa6, 0xf2, 0x69, 0x28, 0x9b, 0x7e, 0xc0, 0x8b, 0xc3, 0x18, 0x2d,
0x98, 0x7e, 0xc0, 0xca, 0x69, 0x62, 0xaa, 0x8f, 0x0d, 0x1b, 0x9b, 0xba, 0x10, 0xe2, 0x9a, 0xa4,
0x68, 0x75, 0x56, 0xb0, 0x19, 0xc1, 0x25, 0x93, 0x2b, 0x27, 0x9d, 0x5c, 0xf7, 0x61, 0xe1, 0xb6,
0xe1, 0xf4, 0x0c, 0xfb, 0x9a, 0xdb, 0xe9, 0x1a, 0xb1, 0x5c, 0xc6, 0xa4, 0xa1, 0x54, 0x24, 0x86,
0xf2, 0x59, 0x96, 0xda, 0xc6, 0x9c, 0x73, 0xda, 0xa7, 0x49, 0x4d, 0x80, 0xa8, 0x3e, 0x34, 0x06,
0xab, 0x1f, 0x67, 0x40, 0x29, 0x53, 0x61, 0x55, 0xa2, 0xf5, 0xee, 0xc3, 0xd4, 0x77, 0xe0, 0x19,
0x9a, 0x66, 0x18, 0x82, 0x62, 0xa7, 0xea, 0xc9, 0x0a, 0x14, 0x49, 0x05, 0xbf, 0x91, 0xa1, 0x26,
0x70, 0xa0, 0x86, 0x71, 0x18, 0xbf, 0x12, 0x3f, 0xcc, 0x7e, 0x3e, 0x25, 0xcb, 0x36, 0xde, 0x22,
0x37, 0xd6, 0x4b, 0x30, 0x85, 0x1f, 0xe1, 0x56, 0x2f, 0xb0, 0x9c, 0xf6, 0x86, 0x6d, 0x38, 0x77,
0x5c, 0xbe, 0x24, 0x25, 0xc1, 0xe8, 0x79, 0xa8, 0x12, 0xe9, 0xbb, 0xbd, 0x80, 0xe3, 0xb1, 0xb5,
0x29, 0x0e, 0x24, 0xf5, 0x91, 0xfe, 0xda, 0x38, 0xc0, 0x26, 0xc7, 0x63, 0x0b, 0x55, 0x12, 0x3c,
0x20, 0x4a, 0x02, 0xf6, 0x0f, 0x23, 0xca, 0x7f, 0x55, 0x12, 0xa2, 0xe4, 0x35, 0x3c, 0x29, 0x51,
0xde, 0x04, 0xe8, 0x60, 0xaf, 0x8d, 0xd7, 0xa9, 0xf1, 0x67, 0x7b, 0xff, 0x25, 0xa9, 0xf1, 0xef,
0x57, 0x70, 0x3b, 0x24, 0xd0, 0x04, 0x5a, 0xf5, 0x06, 0xcc, 0x48, 0x50, 0x88, 0x5d, 0xf3, 0xdd,
0x9e, 0xd7, 0xc2, 0xe1, 0xf1, 0x51, 0xf8, 0x49, 0xd6, 0xc1, 0xc0, 0xf0, 0xda, 0x38, 0xe0, 0x4a,
0xcb, 0xbf, 0xd4, 0xcb, 0x34, 0xfe, 0x43, 0x8f, 0x1a, 0x62, 0x9a, 0x1a, 0x8f, 0x65, 0x2b, 0x03,
0xb1, 0xec, 0x6d, 0x1a, 0x6c, 0x11, 0xe9, 0xc6, 0xcc, 0x43, 0xd8, 0x26, 0x55, 0x61, 0x93, 0x5f,
0xb5, 0x08, 0x3f, 0xd5, 0x5f, 0x28, 0x50, 0x5d, 0xef, 0x74, 0xdd, 0x7e, 0x9c, 0x61, 0xe4, 0xcd,
0xe8, 0xe0, 0x39, 0x6d, 0x46, 0x76, 0x4e, 0xfb, 0x1c, 0x54, 0xe3, 0xf9, 0xf9, 0xec, 0x64, 0xa8,
0xd2, 0x12, 0xf3, 0xf2, 0x4f, 0x40, 0xc9, 0x73, 0xf7, 0x74, 0x62, 0x4a, 0x4d, 0x9e, 0xf1, 0x50,
0xf4, 0xdc, 0x3d, 0x62, 0x60, 0x4d, 0x34, 0x0b, 0xb9, 0x6d, 0xcb, 0x8e, 0x92, 0x75, 0xd8, 0x07,
0x7a, 0x93, 0x6c, 0xd5, 0x58, 0x44, 0x34, 0x3f, 0xea, 0x8e, 0x29, 0xa4, 0x50, 0x3f, 0x82, 0x5a,
0xd8, 0xeb, 0x31, 0xef, 0x98, 0x04, 0x86, 0xff, 0x20, 0x4c, 0x46, 0x60, 0x1f, 0xea, 0x79, 0x16,
0x28, 0xa3, 0xf5, 0xc7, 0x06, 0x1d, 0xc1, 0x24, 0xc1, 0xe0, 0x73, 0x89, 0xfe, 0x56, 0x7f, 0x91,
0x81, 0xf9, 0x24, 0xf6, 0x38, 0x2c, 0x5d, 0x8e, 0xcf, 0x1f, 0xf9, 0xed, 0x01, 0xb1, 0x35, 0x3e,
0x77, 0xf8, 0x08, 0xb4, 0xdc, 0x9e, 0x13, 0x70, 0x03, 0x44, 0x46, 0xe0, 0x1a, 0xf9, 0x46, 0x0b,
0x50, 0xb0, 0x4c, 0xdd, 0x26, 0xbb, 0x3a, 0xb6, 0x26, 0xe5, 0x2d, 0xf3, 0x16, 0xd9, 0xf1, 0xbd,
0x1e, 0x7a, 0x5a, 0x23, 0x67, 0x30, 0x30, 0x7c, 0x54, 0x83, 0x8c, 0x65, 0xf2, 0xe8, 0x46, 0xc6,
0x32, 0xd1, 0x1b, 0xd0, 0xd8, 0xc1, 0x3d, 0x8f, 0x26, 0xb4, 0xd1, 0xd3, 0x17, 0x9d, 0xe6, 0xdd,
0xf2, 0x44, 0x5c, 0xa2, 0x0f, 0xf3, 0x51, 0xf9, 0x9a, 0x11, 0x18, 0x1f, 0x84, 0xa5, 0xe8, 0x35,
0x98, 0x4f, 0x50, 0xf2, 0xf8, 0x2c, 0xf5, 0xa6, 0x8b, 0xda, 0x6c, 0x8c, 0x6e, 0x9d, 0x95, 0xa9,
0x0d, 0x98, 0x27, 0x1d, 0x60, 0x82, 0xb8, 0x47, 0x86, 0x2d, 0x74, 0xc4, 0xbe, 0xaf, 0xc0, 0xc2,
0x40, 0xd1, 0x38, 0x23, 0x72, 0x55, 0x54, 0x92, 0xf2, 0xca, 0x79, 0xa9, 0x41, 0x92, 0xab, 0x40,
0xa8, 0x51, 0x3f, 0x60, 0x5e, 0x93, 0xc6, 0xf2, 0x30, 0x1f, 0x73, 0x56, 0xcf, 0x12, 0xd4, 0xf7,
0xac, 0x60, 0x47, 0xa7, 0xb7, 0x56, 0xa8, 0xcb, 0xc2, 0x02, 0xdb, 0x45, 0xad, 0x46, 0xe0, 0x9b,
0x04, 0x4c, 0xdc, 0x16, 0x5f, 0xfd, 0x4d, 0x05, 0x66, 0x62, 0x6c, 0x8d, 0x23, 0xa6, 0xb7, 0x88,
0x37, 0xc7, 0x2a, 0xe2, 0x92, 0x5a, 0x94, 0x4a, 0x8a, 0xb7, 0x46, 0x4d, 0x76, 0x44, 0xa1, 0xfe,
0x9b, 0x02, 0x65, 0xa1, 0x84, 0x6c, 0x13, 0x79, 0x59, 0x7f, 0x9b, 0x18, 0x01, 0x46, 0x12, 0xc3,
0x73, 0xd0, 0x37, 0x64, 0x42, 0x5e, 0xbb, 0x90, 0x58, 0x67, 0xfa, 0xe8, 0x26, 0xd4, 0x98, 0x98,
0x22, 0xd6, 0xa5, 0xa7, 0x37, 0x51, 0xca, 0xa0, 0xe1, 0x99, 0x9c, 0x4b, 0xad, 0xea, 0x0b, 0x5f,
0x2c, 0xca, 0xe9, 0x9a, 0x98, 0xb6, 0x94, 0x1b, 0xd8, 0xb4, 0x55, 0x44, 0x52, 0xe2, 0xf8, 0xda,
0xd8, 0x30, 0xb1, 0x17, 0xf5, 0x2d, 0xfa, 0x26, 0x9e, 0x26, 0xfb, 0xad, 0x93, 0x8d, 0x00, 0x37,
0xc9, 0xc0, 0x40, 0x64, 0x8f, 0x80, 0x5e, 0x80, 0x29, 0xb3, 0x13, 0xbb, 0x32, 0x15, 0xba, 0xc6,
0x66, 0x47, 0xb8, 0x2b, 0x15, 0x63, 0x68, 0x32, 0xce, 0xd0, 0x7f, 0x29, 0xd1, 0x65, 0x53, 0x0f,
0x9b, 0xd8, 0x09, 0x2c, 0xc3, 0x3e, 0xba, 0x4e, 0x36, 0xa1, 0xd8, 0xf3, 0xb1, 0x27, 0xac, 0x20,
0xd1, 0x37, 0x29, 0xeb, 0x1a, 0xbe, 0xbf, 0xe7, 0x7a, 0x26, 0xe7, 0x32, 0xfa, 0x1e, 0x92, 0xa5,
0xc8, 0xae, 0x2d, 0xca, 0xb3, 0x14, 0x2f, 0xc3, 0x42, 0xc7, 0x35, 0xad, 0x6d, 0x4b, 0x96, 0xdc,
0x48, 0xc8, 0xe6, 0xc2, 0xe2, 0x18, 0x9d, 0xfa, 0xc3, 0x0c, 0x2c, 0x7c, 0xd8, 0x35, 0xbf, 0x82,
0x3e, 0x2f, 0x42, 0xd9, 0xb5, 0xcd, 0x8d, 0x78, 0xb7, 0x45, 0x10, 0xc1, 0x70, 0xf0, 0x5e, 0x84,
0xc1, 0x8e, 0xec, 0x45, 0xd0, 0xd0, 0x0c, 0xce, 0x23, 0xc9, 0x26, 0x3f, 0x4c, 0x36, 0x6d, 0x58,
0x60, 0x61, 0xe3, 0xc7, 0x2c, 0x1a, 0xf5, 0x63, 0x98, 0x23, 0xa6, 0x99, 0x34, 0xf3, 0xa1, 0x8f,
0xbd, 0x31, 0x2d, 0xce, 0x49, 0x28, 0x85, 0x35, 0x87, 0xc9, 0xb5, 0x7d, 0x80, 0x7a, 0x13, 0x66,
0x13, 0x6d, 0x1d, 0xb1, 0x47, 0xea, 0x22, 0x80, 0xe6, 0xda, 0xf8, 0x3d, 0x27, 0xb0, 0x82, 0x7d,
0xe2, 0x04, 0x08, 0x4e, 0x15, 0xfd, 0x4d, 0x30, 0x48, 0x1b, 0x43, 0x30, 0x7e, 0x15, 0xa6, 0xd9,
0x8c, 0x23, 0x35, 0x1d, 0x5d, 0xb8, 0xaf, 0x43, 0x1e, 0xd3, 0x46, 0xf8, 0xa1, 0xc0, 0x69, 0xb9,
0x89, 0x8d, 0xb8, 0xd5, 0x38, 0xba, 0xfa, 0x2b, 0x30, 0xb5, 0xe6, 0xb9, 0xdd, 0xf1, 0x5a, 0xa7,
0x3e, 0x86, 0x8d, 0x45, 0x67, 0xb1, 0x48, 0x00, 0x74, 0xb3, 0xfa, 0x0f, 0x0a, 0xcc, 0xdf, 0xed,
0x62, 0xcf, 0x08, 0x30, 0x91, 0xc5, 0x78, 0x2d, 0x0d, 0x9b, 0x5f, 0x31, 0x2e, 0xb2, 0x71, 0x2e,
0xd0, 0x5b, 0xb1, 0xfb, 0x5a, 0xf2, 0xcd, 0x43, 0x82, 0x4b, 0x21, 0x09, 0xfc, 0xcf, 0x14, 0x98,
0xde, 0xc4, 0x64, 0x0d, 0x19, 0x8f, 0xfd, 0x8b, 0x30, 0x49, 0x38, 0x1a, 0x75, 0x90, 0x28, 0x32,
0x3a, 0x07, 0xd3, 0x96, 0xd3, 0xb2, 0x7b, 0x26, 0xd6, 0x49, 0x5f, 0x75, 0xe2, 0x68, 0xf1, 0x85,
0x7b, 0x8a, 0x17, 0x10, 0x96, 0xc9, 0xf2, 0xa8, 0x3e, 0x62, 0x2a, 0x19, 0xe5, 0xcc, 0xb0, 0xe6,
0x94, 0xc3, 0x34, 0x77, 0x09, 0x72, 0xa4, 0x99, 0x70, 0xb1, 0x96, 0x53, 0xf5, 0xb5, 0x5a, 0x63,
0xd8, 0xea, 0x77, 0x14, 0x40, 0xa2, 0x88, 0xc6, 0x99, 0xc0, 0x5f, 0x17, 0xa3, 0xe7, 0xd9, 0xa1,
0xac, 0xb3, 0x9e, 0x46, 0x71, 0x73, 0x61, 0xa4, 0xe8, 0x30, 0x8e, 0x33, 0x52, 0xa4, 0x5f, 0x43,
0x47, 0x4a, 0x10, 0x02, 0x45, 0x16, 0x47, 0x8a, 0x6a, 0xa2, 0x64, 0xa4, 0x08, 0xcf, 0xe1, 0x48,
0x31, 0x0e, 0xc3, 0x91, 0xa2, 0xcd, 0x29, 0x87, 0x69, 0xee, 0x12, 0xe4, 0x48, 0x33, 0x07, 0x0b,
0x29, 0x1c, 0x29, 0x8a, 0xad, 0xfe, 0x5a, 0x34, 0x52, 0x9c, 0x81, 0xb1, 0x2e, 0x99, 0xe4, 0x99,
0xe4, 0x0f, 0xd4, 0x16, 0x3e, 0x50, 0x1c, 0x5d, 0x7d, 0x1e, 0x6a, 0x1a, 0x66, 0x9b, 0xec, 0xbe,
0x75, 0x14, 0xce, 0x61, 0xd9, 0xbc, 0x3b, 0x0b, 0x53, 0x1b, 0x9e, 0xb5, 0x6b, 0xd9, 0xb8, 0x3d,
0xcc, 0xcc, 0xfe, 0xad, 0x02, 0x73, 0x5c, 0xf7, 0x78, 0x9d, 0x47, 0x1f, 0xf8, 0x37, 0x13, 0x96,
0xf4, 0xb9, 0x14, 0x67, 0x55, 0xe4, 0x3d, 0xb4, 0xa6, 0x64, 0x79, 0x0e, 0x15, 0xa0, 0x1b, 0xf2,
0x2d, 0x6a, 0xc1, 0x2c, 0x2f, 0x8d, 0x3a, 0x45, 0x55, 0xe1, 0x73, 0xa5, 0x2f, 0x0c, 0xae, 0x0f,
0xef, 0xd0, 0x23, 0x50, 0x0a, 0xe1, 0xbc, 0x8f, 0xc4, 0x47, 0x44, 0x84, 0xd6, 0x00, 0x22, 0x0e,
0x42, 0x05, 0x79, 0x5e, 0x1e, 0xaa, 0x89, 0x0b, 0x58, 0x13, 0xe8, 0xe8, 0x3d, 0x8e, 0xa4, 0x60,
0xc7, 0x7b, 0x73, 0x20, 0x31, 0xb1, 0x87, 0xf7, 0x2a, 0x39, 0xb9, 0x7f, 0xaa, 0x50, 0x7d, 0x70,
0x5a, 0x56, 0xd7, 0xb0, 0xb9, 0x3e, 0x9c, 0x85, 0x5a, 0x37, 0x04, 0x89, 0x07, 0xf9, 0xd5, 0x08,
0x4a, 0xb3, 0x9f, 0x2e, 0x1d, 0x6a, 0x3e, 0xdf, 0x9c, 0x88, 0xa6, 0x18, 0xb3, 0xa0, 0xd9, 0x91,
0x2c, 0x28, 0x21, 0x23, 0xe8, 0xab, 0x65, 0x28, 0x45, 0xcd, 0x93, 0xdd, 0x54, 0xf5, 0x86, 0x67,
0x38, 0x81, 0x1b, 0x3a, 0x02, 0x47, 0x9a, 0xed, 0xab, 0xb4, 0x4e, 0x36, 0x32, 0xbc, 0x1b, 0xa3,
0x0d, 0x68, 0x9f, 0x4c, 0xfd, 0x6f, 0x05, 0xca, 0x94, 0x15, 0xce, 0xc8, 0xaa, 0xc0, 0x27, 0xe7,
0x26, 0xb5, 0x4e, 0x51, 0xea, 0x5a, 0x9f, 0x2c, 0xa6, 0xaa, 0x99, 0xa3, 0xa8, 0xea, 0x73, 0x50,
0x0d, 0x7f, 0x8b, 0x6b, 0x77, 0x25, 0x04, 0xf2, 0xf5, 0xbb, 0xd0, 0x66, 0x32, 0xa4, 0x4b, 0x78,
0x79, 0x45, 0x95, 0x6f, 0xb7, 0x45, 0x39, 0x6b, 0x21, 0x89, 0xfa, 0x69, 0x64, 0xf2, 0x28, 0xc2,
0xd1, 0xad, 0xc3, 0x1b, 0x09, 0xeb, 0xb0, 0x98, 0xce, 0x45, 0xc2, 0xd1, 0x22, 0x7b, 0xea, 0x18,
0x0b, 0x63, 0xee, 0xa9, 0x69, 0xb5, 0x16, 0x1e, 0xbe, 0xa7, 0x16, 0x19, 0x89, 0x28, 0xd4, 0x9f,
0x29, 0xb0, 0xc0, 0x7d, 0x9d, 0x48, 0x57, 0x9e, 0x80, 0x48, 0xd0, 0x37, 0xb8, 0xc5, 0xcf, 0x52,
0x9f, 0xec, 0xa5, 0x61, 0x3e, 0x59, 0xc4, 0xa7, 0xe0, 0x94, 0x9d, 0x85, 0xd2, 0x6d, 0x8a, 0xf4,
0xde, 0xa3, 0x00, 0x35, 0xa0, 0xb0, 0x8b, 0x3d, 0xdf, 0x72, 0x1d, 0x3e, 0xff, 0xc3, 0xcf, 0x73,
0x67, 0xa0, 0x18, 0x5e, 0xe9, 0x43, 0x05, 0xc8, 0x5e, 0xb5, 0xed, 0xfa, 0x04, 0xaa, 0x40, 0x71,
0x9d, 0xdf, 0x5b, 0xab, 0x2b, 0xe7, 0xde, 0x85, 0x19, 0x89, 0xef, 0x87, 0xa6, 0xa1, 0x7a, 0xd5,
0xa4, 0x9b, 0x84, 0x7b, 0x2e, 0x01, 0xd6, 0x27, 0xd0, 0x3c, 0x20, 0x0d, 0x77, 0xdc, 0x5d, 0x8a,
0x78, 0xdd, 0x73, 0x3b, 0x14, 0xae, 0x9c, 0x7b, 0x19, 0x66, 0x65, 0x9c, 0xa2, 0x12, 0xe4, 0x68,
0xcf, 0xeb, 0x13, 0x08, 0x20, 0xaf, 0xe1, 0x5d, 0xf7, 0x01, 0xae, 0x2b, 0x2b, 0x3f, 0x3a, 0x0f,
0x55, 0xc6, 0x3b, 0xbf, 0x80, 0x8e, 0x74, 0xa8, 0x27, 0x9f, 0x79, 0x42, 0x5f, 0x93, 0x1f, 0x71,
0xcb, 0x5f, 0x83, 0x6a, 0x0e, 0x53, 0x1c, 0x75, 0x02, 0x7d, 0x04, 0xb5, 0xf8, 0x1b, 0x49, 0x48,
0x1e, 0xb9, 0x97, 0x3e, 0xa4, 0x74, 0x50, 0xe5, 0x3a, 0x54, 0x63, 0x2f, 0x1d, 0x21, 0xf9, 0x60,
0xca, 0x5e, 0x43, 0x6a, 0xca, 0x8f, 0x54, 0xc4, 0x47, 0x88, 0x18, 0xf7, 0xf1, 0xc7, 0x47, 0x52,
0xb8, 0x97, 0xbe, 0x50, 0x72, 0x10, 0xf7, 0x06, 0x4c, 0x0f, 0xbc, 0x0d, 0x82, 0x5e, 0x4e, 0x31,
0x62, 0xf2, 0x37, 0x44, 0x0e, 0x6a, 0x62, 0x0f, 0xd0, 0xe0, 0xfb, 0x3d, 0x68, 0x59, 0x3e, 0x02,
0x69, 0xcf, 0x18, 0x35, 0x2f, 0x8c, 0x8c, 0x1f, 0x09, 0xee, 0xd7, 0x15, 0x58, 0x48, 0x79, 0xbe,
0x03, 0x5d, 0x4c, 0x3b, 0xb1, 0x1c, 0xf2, 0x18, 0x49, 0xf3, 0xb5, 0xc3, 0x11, 0x45, 0x8c, 0x38,
0x30, 0x95, 0x78, 0xca, 0x02, 0x9d, 0x4f, 0xbd, 0x7f, 0x3b, 0xf8, 0xb4, 0x47, 0xf3, 0x6b, 0xa3,
0x21, 0x47, 0xed, 0xdd, 0x87, 0xa9, 0xc4, 0x3b, 0x0e, 0x29, 0xed, 0xc9, 0x5f, 0x7b, 0x38, 0x68,
0x40, 0xbf, 0x05, 0xd5, 0xd8, 0x83, 0x0b, 0x29, 0x1a, 0x2f, 0x7b, 0x94, 0xe1, 0xa0, 0xaa, 0xef,
0x43, 0x45, 0x7c, 0x17, 0x01, 0x2d, 0xa5, 0xcd, 0xa5, 0x81, 0x8a, 0x0f, 0x33, 0x95, 0xfa, 0xf7,
0x99, 0x87, 0x4c, 0xa5, 0x81, 0x2b, 0xe0, 0xa3, 0x4f, 0x25, 0xa1, 0xfe, 0xa1, 0x53, 0xe9, 0xd0,
0x4d, 0x7c, 0x5b, 0xa1, 0x81, 0x15, 0xc9, 0x7d, 0x79, 0xb4, 0x92, 0xa6, 0x9b, 0xe9, 0x2f, 0x03,
0x34, 0x2f, 0x1e, 0x8a, 0x26, 0x92, 0xe2, 0x03, 0xa8, 0xc5, 0x6f, 0x85, 0xa7, 0x48, 0x51, 0x7a,
0x91, 0xbe, 0x79, 0x7e, 0x24, 0xdc, 0xa8, 0xb1, 0x0f, 0xa1, 0x2c, 0xbc, 0xdc, 0x88, 0x5e, 0x1c,
0xa2, 0xc7, 0xe2, 0x33, 0x86, 0x07, 0x49, 0xf2, 0x03, 0x28, 0x45, 0x0f, 0x2e, 0xa2, 0xb3, 0xa9,
0xfa, 0x7b, 0x98, 0x2a, 0x37, 0x01, 0xfa, 0xaf, 0x29, 0xa2, 0x17, 0xa4, 0x75, 0x0e, 0x3c, 0xb7,
0x78, 0x50, 0xa5, 0x51, 0xf7, 0xd9, 0x65, 0x9b, 0x61, 0xdd, 0x17, 0x6f, 0x87, 0x1d, 0x54, 0xed,
0x0e, 0x54, 0x63, 0x77, 0x3a, 0xd3, 0xa6, 0xb0, 0xe4, 0xaa, 0x6d, 0xf3, 0xdc, 0x28, 0xa8, 0xd1,
0xf8, 0xed, 0x40, 0x35, 0x76, 0xc3, 0x2e, 0xa5, 0x25, 0xd9, 0x85, 0xc2, 0x94, 0x96, 0xa4, 0x17,
0xf6, 0xd4, 0x09, 0xf4, 0xa9, 0x70, 0x99, 0x2f, 0x76, 0x61, 0x12, 0xbd, 0x3a, 0xb4, 0x1e, 0xd9,
0x7d, 0xd1, 0xe6, 0xca, 0x61, 0x48, 0x22, 0x16, 0xb8, 0x56, 0x31, 0x91, 0xa6, 0x6b, 0xd5, 0x61,
0x46, 0x6a, 0x13, 0xf2, 0xec, 0xce, 0x1c, 0x52, 0x53, 0x6e, 0xc7, 0x0a, 0x17, 0xea, 0x9a, 0xf2,
0xed, 0x47, 0xfc, 0x3a, 0x19, 0xab, 0x94, 0x1d, 0x6e, 0xa7, 0x54, 0x1a, 0xbb, 0x30, 0x35, 0x6a,
0xa5, 0x1a, 0xe4, 0xd9, 0x9d, 0x8e, 0x94, 0x4a, 0x63, 0xf7, 0x94, 0x9a, 0xc3, 0x71, 0xd8, 0x9e,
0x77, 0x02, 0x6d, 0x40, 0x8e, 0xe6, 0x0b, 0xa0, 0x33, 0xc3, 0xae, 0x3b, 0x0c, 0xab, 0x31, 0x76,
0x23, 0x42, 0x9d, 0x40, 0x77, 0x21, 0x47, 0xe3, 0xac, 0x29, 0x35, 0x8a, 0x77, 0x16, 0x9a, 0x43,
0x51, 0x42, 0x16, 0x4d, 0xa8, 0x88, 0x89, 0xca, 0x29, 0x4b, 0x96, 0x24, 0x95, 0xbb, 0x39, 0x0a,
0x66, 0xd8, 0x0a, 0x9b, 0x46, 0xfd, 0xdc, 0x89, 0xf4, 0x69, 0x34, 0x90, 0x97, 0x91, 0x3e, 0x8d,
0x06, 0x53, 0x31, 0xd4, 0x09, 0xf4, 0x5b, 0x0a, 0x34, 0xd2, 0xb2, 0x67, 0x51, 0xaa, 0x07, 0x34,
0x2c, 0x05, 0xb8, 0x79, 0xe9, 0x90, 0x54, 0x11, 0x2f, 0x9f, 0xd0, 0x58, 0xec, 0x40, 0xbe, 0xec,
0x85, 0xb4, 0xfa, 0x52, 0x72, 0x40, 0x9b, 0xaf, 0x8c, 0x4e, 0x10, 0xb5, 0xbd, 0x05, 0x65, 0x21,
0x0e, 0x9c, 0x62, 0x79, 0x07, 0x03, 0xd8, 0x29, 0xa3, 0x2a, 0x09, 0x29, 0x33, 0xf5, 0xa6, 0x49,
0x96, 0x29, 0xca, 0x28, 0xe6, 0x6c, 0xa6, 0xa8, 0x77, 0x2c, 0x47, 0x53, 0x9d, 0x40, 0x18, 0x2a,
0x62, 0xc6, 0x65, 0x8a, 0x36, 0x4a, 0x92, 0x35, 0x9b, 0x2f, 0x8d, 0x80, 0x19, 0x35, 0xa3, 0x03,
0xf4, 0x33, 0x1e, 0x53, 0xd6, 0xba, 0x81, 0xa4, 0xcb, 0xe6, 0x8b, 0x07, 0xe2, 0x89, 0xcb, 0xbe,
0x90, 0xc3, 0x98, 0x22, 0xfd, 0xc1, 0x2c, 0xc7, 0x11, 0xf6, 0x22, 0x83, 0x79, 0x72, 0x29, 0x7b,
0x91, 0xd4, 0x94, 0xbc, 0xe6, 0x85, 0x91, 0xf1, 0xa3, 0xfe, 0x3c, 0x84, 0x7a, 0x32, 0xaf, 0x30,
0x65, 0x8f, 0x9b, 0x92, 0xdd, 0xd8, 0x7c, 0x79, 0x44, 0x6c, 0x71, 0x3d, 0x3c, 0x31, 0xc8, 0xd3,
0xff, 0xb3, 0x82, 0x1d, 0x9a, 0xd2, 0x36, 0x4a, 0xaf, 0xc5, 0xec, 0xb9, 0x51, 0x7a, 0x1d, 0xcb,
0x95, 0xe3, 0x8b, 0x17, 0xcd, 0x00, 0x49, 0x5b, 0xbc, 0xc4, 0x2c, 0xad, 0x94, 0x75, 0x26, 0x9e,
0xd3, 0xc4, 0xdc, 0xcf, 0x78, 0x66, 0x09, 0x3a, 0x37, 0x52, 0xfa, 0xc9, 0x30, 0xf7, 0x53, 0x9e,
0xaa, 0xc2, 0xb6, 0x6e, 0x89, 0xc4, 0x99, 0x94, 0xad, 0x94, 0x3c, 0xf3, 0x26, 0x65, 0xeb, 0x96,
0x92, 0x8b, 0x43, 0x27, 0x56, 0x3d, 0x99, 0x85, 0x30, 0xfc, 0x2c, 0x24, 0x19, 0x9d, 0x3e, 0xf8,
0xb8, 0xa2, 0x9e, 0x0c, 0xf9, 0xa7, 0x34, 0x90, 0x92, 0x19, 0x30, 0x42, 0x03, 0xc9, 0xc0, 0x79,
0x4a, 0x03, 0x29, 0xf1, 0xf5, 0x11, 0x7c, 0xd7, 0x58, 0x10, 0x3b, 0x65, 0x29, 0x94, 0x05, 0xba,
0x53, 0x96, 0x42, 0x69, 0xfc, 0x9d, 0x79, 0xf4, 0xfd, 0x00, 0x75, 0x8a, 0x95, 0x1b, 0x88, 0x60,
0x1f, 0xc4, 0xfe, 0x5d, 0x28, 0x86, 0x51, 0x67, 0xf4, 0x7c, 0xaa, 0x8b, 0x78, 0x88, 0x0a, 0xef,
0xc3, 0x54, 0xe2, 0x04, 0x2f, 0x45, 0x45, 0xe5, 0x91, 0xe8, 0x83, 0xc7, 0x13, 0xfa, 0xb1, 0xcd,
0x14, 0x21, 0x0c, 0xc4, 0x87, 0x53, 0x4c, 0xfd, 0x60, 0x90, 0x54, 0x6c, 0x80, 0x30, 0x36, 0xb4,
0x01, 0x21, 0xac, 0x39, 0xb4, 0x01, 0x31, 0xb6, 0xc7, 0xf7, 0xab, 0xb1, 0x40, 0x4e, 0xda, 0x7e,
0x55, 0x16, 0x46, 0x4b, 0xdb, 0xaf, 0x4a, 0x23, 0x43, 0x4c, 0xfd, 0x93, 0xa7, 0xa1, 0x29, 0xea,
0x9f, 0x72, 0x0c, 0x7d, 0xd0, 0x78, 0x6c, 0x41, 0x59, 0x38, 0x4b, 0x47, 0xc3, 0xe4, 0x20, 0x1e,
0xf8, 0xa7, 0xf8, 0x25, 0x92, 0x63, 0x79, 0x75, 0x62, 0xa5, 0x07, 0x95, 0x0d, 0xcf, 0x7d, 0x14,
0x3e, 0x11, 0xfa, 0x15, 0x79, 0x15, 0x57, 0x5a, 0x50, 0x63, 0x08, 0x3a, 0x7e, 0x14, 0xe8, 0xee,
0xd6, 0xc7, 0xe8, 0xe4, 0x32, 0xfb, 0x6f, 0x87, 0xe5, 0xf0, 0xbf, 0x1d, 0x96, 0xaf, 0x5b, 0x36,
0xbe, 0xcb, 0x73, 0x64, 0xff, 0xa3, 0x30, 0xe4, 0x86, 0x66, 0x74, 0x3e, 0xae, 0xf1, 0xbf, 0x97,
0x78, 0xef, 0x51, 0x70, 0x77, 0xeb, 0xe3, 0x55, 0xe3, 0x8b, 0xb7, 0x0b, 0x90, 0x5b, 0x59, 0x7e,
0x75, 0xf9, 0x15, 0xa8, 0x59, 0x11, 0x7a, 0xdb, 0xeb, 0xb6, 0x56, 0xcb, 0x8c, 0x68, 0x83, 0xd4,
0xb3, 0xa1, 0xfc, 0xff, 0x8b, 0x6d, 0x2b, 0xd8, 0xe9, 0x6d, 0x91, 0x21, 0xb8, 0xc0, 0xd0, 0x5e,
0xb6, 0x5c, 0xfe, 0xeb, 0x82, 0xe5, 0x04, 0xd8, 0x73, 0x0c, 0x9b, 0xfd, 0xed, 0x04, 0x87, 0x76,
0xb7, 0xfe, 0x50, 0x51, 0xb6, 0xf2, 0x14, 0x74, 0xf1, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0xf4,
0x28, 0xc4, 0x6e, 0xd8, 0x62, 0x00, 0x00,
// 5191 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3d, 0x5d, 0x6f, 0x1c, 0x47,
0x72, 0x9c, 0x5d, 0xee, 0x57, 0xed, 0x07, 0x97, 0xcd, 0xaf, 0xf5, 0x4a, 0xb2, 0xa8, 0xb1, 0x65,
0xd3, 0xd2, 0x99, 0xb2, 0x29, 0x4b, 0xbe, 0x93, 0x7d, 0xb6, 0x45, 0xd1, 0x92, 0x08, 0xeb, 0x83,
0x1e, 0xca, 0x17, 0x5c, 0x0c, 0x61, 0x32, 0xdc, 0x69, 0x2e, 0xc7, 0x9a, 0x9d, 0x59, 0xcd, 0xcc,
0x92, 0xa2, 0x1f, 0x02, 0x23, 0x77, 0xc8, 0x07, 0xe2, 0xdc, 0xe1, 0x90, 0x43, 0x8c, 0x7b, 0x48,
0x90, 0xcb, 0xd7, 0x5b, 0x82, 0xdc, 0xdd, 0x43, 0x80, 0x24, 0x40, 0x12, 0xe0, 0x1e, 0x02, 0x23,
0x48, 0x2e, 0x4f, 0x41, 0xe0, 0x1f, 0x10, 0xe4, 0x2d, 0x01, 0x92, 0xb7, 0x04, 0x38, 0xf4, 0xc7,
0xcc, 0xf6, 0xcc, 0xf6, 0x2c, 0x97, 0xdc, 0x93, 0x45, 0xbd, 0xed, 0x54, 0x57, 0x75, 0x57, 0x57,
0x57, 0x57, 0x57, 0x77, 0x55, 0xf7, 0x42, 0xa5, 0x63, 0xd9, 0xbb, 0x3d, 0x7f, 0xb9, 0xeb, 0xb9,
0x81, 0x8b, 0x66, 0xc4, 0xaf, 0x65, 0xf6, 0xd1, 0xac, 0xb4, 0xdc, 0x4e, 0xc7, 0x75, 0x18, 0xb0,
0x59, 0xf1, 0x5b, 0x3b, 0xb8, 0x63, 0xf0, 0xaf, 0xc5, 0xb6, 0xeb, 0xb6, 0x6d, 0x7c, 0x81, 0x7e,
0x6d, 0xf5, 0xb6, 0x2f, 0x98, 0xd8, 0x6f, 0x79, 0x56, 0x37, 0x70, 0x3d, 0x86, 0xa1, 0xfe, 0x81,
0x02, 0xe8, 0x9a, 0x87, 0x8d, 0x00, 0x5f, 0xb5, 0x2d, 0xc3, 0xd7, 0xf0, 0xc3, 0x1e, 0xf6, 0x03,
0xf4, 0x0a, 0x4c, 0x6e, 0x19, 0x3e, 0x6e, 0x28, 0x8b, 0xca, 0x52, 0x79, 0xe5, 0xe4, 0x72, 0xac,
0x61, 0xde, 0xe0, 0x6d, 0xbf, 0xbd, 0x6a, 0xf8, 0x58, 0xa3, 0x98, 0x68, 0x01, 0x0a, 0xe6, 0x96,
0xee, 0x18, 0x1d, 0xdc, 0xc8, 0x2c, 0x2a, 0x4b, 0x25, 0x2d, 0x6f, 0x6e, 0xdd, 0x31, 0x3a, 0x18,
0xbd, 0x08, 0x53, 0x2d, 0xd7, 0xb6, 0x71, 0x2b, 0xb0, 0x5c, 0x87, 0x21, 0x64, 0x29, 0x42, 0xad,
0x0f, 0xa6, 0x88, 0xb3, 0x90, 0x33, 0x08, 0x0f, 0x8d, 0x49, 0x5a, 0xcc, 0x3e, 0x54, 0x1f, 0xea,
0x6b, 0x9e, 0xdb, 0x7d, 0x5c, 0xdc, 0x45, 0x8d, 0x66, 0xc5, 0x46, 0x7f, 0x5f, 0x81, 0xe9, 0xab,
0x76, 0x80, 0xbd, 0x63, 0x2a, 0x94, 0x1f, 0x66, 0x60, 0x81, 0x8d, 0xda, 0xb5, 0x08, 0xfd, 0x49,
0x72, 0x39, 0x0f, 0x79, 0xa6, 0x77, 0x94, 0xcd, 0x8a, 0xc6, 0xbf, 0xd0, 0x29, 0x00, 0x7f, 0xc7,
0xf0, 0x4c, 0x5f, 0x77, 0x7a, 0x9d, 0x46, 0x6e, 0x51, 0x59, 0xca, 0x69, 0x25, 0x06, 0xb9, 0xd3,
0xeb, 0x20, 0x0d, 0xa6, 0x5b, 0xae, 0xe3, 0x5b, 0x7e, 0x80, 0x9d, 0xd6, 0xbe, 0x6e, 0xe3, 0x5d,
0x6c, 0x37, 0xf2, 0x8b, 0xca, 0x52, 0x6d, 0xe5, 0xac, 0x94, 0xef, 0x6b, 0x7d, 0xec, 0x5b, 0x04,
0x59, 0xab, 0xb7, 0x12, 0x90, 0x2b, 0x85, 0xcf, 0xdf, 0x9a, 0xac, 0x2b, 0x8d, 0xac, 0xfa, 0x7d,
0x05, 0xe6, 0x88, 0xe6, 0x1c, 0x0b, 0x09, 0x31, 0xb6, 0x32, 0x8d, 0xac, 0xfa, 0x63, 0x05, 0x66,
0x6f, 0x1a, 0xfe, 0xf1, 0x18, 0xb7, 0x53, 0x00, 0x81, 0xd5, 0xc1, 0xba, 0x1f, 0x18, 0x9d, 0x2e,
0x1d, 0xbb, 0x49, 0xad, 0x44, 0x20, 0x9b, 0x04, 0xc0, 0x98, 0x9e, 0x6c, 0x64, 0xd5, 0x6f, 0x42,
0x65, 0xd5, 0x75, 0x6d, 0x0d, 0xfb, 0x5d, 0xd7, 0xf1, 0x31, 0xba, 0x08, 0x79, 0x3f, 0x30, 0x82,
0x9e, 0xcf, 0xb9, 0x3d, 0x21, 0xe5, 0x76, 0x93, 0xa2, 0x68, 0x1c, 0x95, 0xa8, 0xf2, 0xae, 0x61,
0xf7, 0x18, 0xb3, 0x45, 0x8d, 0x7d, 0xa8, 0x1f, 0x42, 0x6d, 0x33, 0xf0, 0x2c, 0xa7, 0xfd, 0x0b,
0xac, 0xbc, 0x14, 0x56, 0xfe, 0x85, 0x02, 0xcf, 0xac, 0x51, 0x93, 0xb7, 0x75, 0x4c, 0x66, 0x8a,
0x0a, 0x95, 0x3e, 0x64, 0x7d, 0x8d, 0xca, 0x3c, 0xab, 0xc5, 0x60, 0x89, 0x51, 0xc9, 0xa5, 0x8e,
0xca, 0x27, 0x39, 0x68, 0xca, 0x7a, 0x37, 0x8e, 0x1c, 0xbf, 0x1e, 0xcd, 0xe4, 0x0c, 0x25, 0x4a,
0xcc, 0x43, 0xbe, 0xba, 0xf4, 0x5b, 0xdb, 0xa4, 0x80, 0x68, 0xc2, 0x27, 0xbb, 0x97, 0x95, 0x74,
0x6f, 0x05, 0xe6, 0x76, 0x2d, 0x2f, 0xe8, 0x19, 0xb6, 0xde, 0xda, 0x31, 0x1c, 0x07, 0xdb, 0x54,
0x60, 0xc4, 0xc4, 0x65, 0x97, 0x4a, 0xda, 0x0c, 0x2f, 0xbc, 0xc6, 0xca, 0x88, 0xd4, 0x7c, 0xf4,
0x1a, 0xcc, 0x77, 0x77, 0xf6, 0x7d, 0xab, 0x35, 0x40, 0x94, 0xa3, 0x44, 0xb3, 0x61, 0x69, 0x8c,
0xea, 0x3c, 0x4c, 0xb7, 0xa8, 0x95, 0x34, 0x75, 0x22, 0x3e, 0x26, 0xcf, 0x3c, 0x95, 0x67, 0x9d,
0x17, 0xdc, 0x0b, 0xe1, 0x84, 0xad, 0x10, 0xb9, 0x17, 0xb4, 0x04, 0x82, 0x02, 0x25, 0x98, 0xe1,
0x85, 0x1f, 0x04, 0xad, 0x3e, 0x4d, 0xdc, 0xbe, 0x15, 0x93, 0xf6, 0xad, 0x01, 0x05, 0x6a, 0xaf,
0xb1, 0xdf, 0x28, 0x51, 0x36, 0xc3, 0x4f, 0xb4, 0x0e, 0x53, 0x7e, 0x60, 0x78, 0x81, 0xde, 0x75,
0x7d, 0x8b, 0xc8, 0xc5, 0x6f, 0xc0, 0x62, 0x76, 0xa9, 0xbc, 0xb2, 0x28, 0x1d, 0xa4, 0xf7, 0xf0,
0xfe, 0x9a, 0x11, 0x18, 0x1b, 0x86, 0xe5, 0x69, 0x35, 0x4a, 0xb8, 0x11, 0xd2, 0xc9, 0x8d, 0x68,
0x79, 0x2c, 0x23, 0x2a, 0x53, 0xe7, 0x8a, 0x4c, 0x9d, 0xd5, 0xbf, 0x51, 0x60, 0xee, 0x96, 0x6b,
0x98, 0xc7, 0x63, 0x72, 0x9d, 0x85, 0x9a, 0x87, 0xbb, 0xb6, 0xd5, 0x32, 0xc8, 0x78, 0x6c, 0x61,
0x8f, 0x4e, 0xaf, 0x9c, 0x56, 0xe5, 0xd0, 0x3b, 0x14, 0xc8, 0x26, 0x50, 0xae, 0x91, 0x55, 0x3f,
0x53, 0xa0, 0xa1, 0x61, 0x1b, 0x1b, 0x3e, 0x3e, 0x46, 0xab, 0x44, 0xbe, 0x91, 0x25, 0x0e, 0xc8,
0xb3, 0x37, 0x70, 0x20, 0xcc, 0xb3, 0xc0, 0x08, 0x2c, 0x3f, 0xb0, 0x5a, 0xfe, 0x13, 0xe7, 0x8f,
0x98, 0x9e, 0xef, 0x2a, 0x70, 0x3a, 0x95, 0xbf, 0x71, 0xec, 0xcf, 0xeb, 0x90, 0x23, 0xbf, 0xfc,
0x46, 0x86, 0x4e, 0x87, 0x33, 0x69, 0xd3, 0xe1, 0x1b, 0xc4, 0xbe, 0xd3, 0xf9, 0xc0, 0xf0, 0xd5,
0xff, 0x50, 0x60, 0x7e, 0x73, 0xc7, 0xdd, 0xeb, 0xb3, 0xf4, 0x38, 0x24, 0x15, 0x37, 0xcd, 0xd9,
0x84, 0x69, 0x46, 0xaf, 0xc2, 0x64, 0xb0, 0xdf, 0xc5, 0x54, 0xed, 0x6a, 0x2b, 0xa7, 0x96, 0x25,
0xfe, 0xfa, 0x32, 0x61, 0xf2, 0xde, 0x7e, 0x17, 0x6b, 0x14, 0x15, 0xbd, 0x04, 0xf5, 0x84, 0xec,
0x43, 0x9b, 0x36, 0x15, 0x17, 0xbe, 0x7f, 0x25, 0xf7, 0xf9, 0x5b, 0x99, 0xfa, 0xa4, 0xfa, 0x5f,
0x19, 0x58, 0x18, 0xe8, 0xe9, 0x38, 0x32, 0x97, 0xb1, 0x90, 0x91, 0xb2, 0x40, 0x66, 0x98, 0x80,
0x6a, 0x99, 0xc4, 0x6f, 0xce, 0x2e, 0x65, 0xb5, 0xaa, 0x60, 0xe1, 0x4d, 0x1f, 0xbd, 0x0c, 0x68,
0xc0, 0xf0, 0x32, 0xfb, 0x3e, 0xa9, 0x4d, 0x27, 0x2d, 0x2f, 0xb5, 0xee, 0x52, 0xd3, 0xcb, 0x24,
0x31, 0xa9, 0xcd, 0x4a, 0x6c, 0xaf, 0x8f, 0x5e, 0x85, 0x59, 0xcb, 0xb9, 0x8d, 0x3b, 0xae, 0xb7,
0xaf, 0x77, 0xb1, 0xd7, 0xc2, 0x4e, 0x60, 0xb4, 0xb1, 0xdf, 0xc8, 0x53, 0x8e, 0x66, 0xc2, 0xb2,
0x8d, 0x7e, 0x11, 0xba, 0x0c, 0x0b, 0x0f, 0x7b, 0xd8, 0xdb, 0xd7, 0x7d, 0xec, 0xed, 0x5a, 0x2d,
0xac, 0x1b, 0xbb, 0x86, 0x65, 0x1b, 0x5b, 0x36, 0x6e, 0x14, 0x16, 0xb3, 0x4b, 0x45, 0x6d, 0x8e,
0x16, 0x6f, 0xb2, 0xd2, 0xab, 0x61, 0xa1, 0xfa, 0x13, 0x05, 0xe6, 0x99, 0xbf, 0xbd, 0x61, 0x78,
0x81, 0x75, 0x0c, 0xec, 0x5c, 0x37, 0xe4, 0x83, 0xe1, 0xb1, 0xdd, 0x41, 0x35, 0x82, 0x52, 0xe3,
0xfc, 0x23, 0x05, 0x66, 0x89, 0x07, 0xfc, 0x34, 0xf1, 0xfc, 0x97, 0x0a, 0xcc, 0xdc, 0x34, 0xfc,
0xa7, 0x89, 0xe5, 0x2f, 0xf8, 0x1a, 0x18, 0xf1, 0xfc, 0x44, 0x37, 0x8c, 0x2f, 0xc2, 0x54, 0x9c,
0xe9, 0xd0, 0xaf, 0xaa, 0xc5, 0xb8, 0xf6, 0x25, 0x8b, 0x65, 0x4e, 0xb2, 0x58, 0xaa, 0x7f, 0xd5,
0x5f, 0x23, 0x9f, 0xae, 0x0e, 0xaa, 0x7f, 0xad, 0xc0, 0xa9, 0x1b, 0x38, 0x88, 0xb8, 0x3e, 0x16,
0x4b, 0xe8, 0xa8, 0x4a, 0xf5, 0x1d, 0xe6, 0x00, 0x48, 0x99, 0x7f, 0x22, 0xeb, 0xeb, 0x6f, 0x67,
0x60, 0x8e, 0xac, 0x3a, 0xc7, 0x43, 0x09, 0x46, 0xd9, 0x46, 0x49, 0x14, 0x25, 0x27, 0x9d, 0x09,
0xe1, 0xaa, 0x9d, 0x1f, 0x79, 0xd5, 0x56, 0x7f, 0x9c, 0x61, 0xde, 0x86, 0x28, 0x8d, 0x71, 0x86,
0x45, 0xc2, 0x6b, 0x46, 0xca, 0xab, 0x0a, 0x95, 0x08, 0xb2, 0xbe, 0x16, 0x2e, 0xbf, 0x31, 0xd8,
0x71, 0x5d, 0x7d, 0xd5, 0x4f, 0x15, 0x98, 0x0f, 0xf7, 0xab, 0x9b, 0xb8, 0xdd, 0xc1, 0x4e, 0x70,
0x74, 0x1d, 0x4a, 0x6a, 0x40, 0x46, 0xa2, 0x01, 0x27, 0xa1, 0xe4, 0xb3, 0x76, 0xa2, 0xad, 0x68,
0x1f, 0xa0, 0xfe, 0x9d, 0x02, 0x0b, 0x03, 0xec, 0x8c, 0x33, 0x88, 0x0d, 0x28, 0x58, 0x8e, 0x89,
0x1f, 0x45, 0xdc, 0x84, 0x9f, 0xa4, 0x64, 0xab, 0x67, 0xd9, 0x66, 0xc4, 0x46, 0xf8, 0x89, 0xce,
0x40, 0x05, 0x3b, 0xc4, 0xc7, 0xd0, 0x29, 0x2e, 0x55, 0xe4, 0xa2, 0x56, 0x66, 0xb0, 0x75, 0x02,
0x22, 0xc4, 0xdb, 0x16, 0xa6, 0xc4, 0x39, 0x46, 0xcc, 0x3f, 0xd5, 0xdf, 0x51, 0x60, 0x86, 0x68,
0x21, 0xe7, 0xde, 0x7f, 0xbc, 0xd2, 0x5c, 0x84, 0xb2, 0xa0, 0x66, 0xbc, 0x23, 0x22, 0x48, 0x7d,
0x00, 0xb3, 0x71, 0x76, 0xc6, 0x91, 0xe6, 0xb3, 0x00, 0xd1, 0x58, 0xb1, 0xd9, 0x90, 0xd5, 0x04,
0x88, 0xfa, 0x69, 0x26, 0x3c, 0xb9, 0xa6, 0x62, 0x7a, 0xc2, 0xc7, 0x68, 0x74, 0x48, 0x44, 0x7b,
0x5e, 0xa2, 0x10, 0x5a, 0xbc, 0x06, 0x15, 0xfc, 0x28, 0xf0, 0x0c, 0xbd, 0x6b, 0x78, 0x46, 0x87,
0x4d, 0xab, 0x91, 0x4c, 0x6f, 0x99, 0x92, 0x6d, 0x50, 0x2a, 0xd2, 0x08, 0x55, 0x11, 0xd6, 0x48,
0x9e, 0x35, 0x42, 0x21, 0x74, 0xc1, 0xf8, 0x47, 0xe2, 0xec, 0x71, 0x6d, 0x3e, 0xee, 0x02, 0x89,
0x77, 0x25, 0x97, 0xec, 0xca, 0x9f, 0x2a, 0x50, 0xa7, 0x5d, 0x58, 0xe3, 0xd1, 0x0a, 0xcb, 0x75,
0x12, 0x34, 0x4a, 0x82, 0x66, 0xc8, 0xdc, 0xfb, 0x1a, 0xe4, 0xb9, 0xdc, 0xb3, 0xa3, 0xca, 0x9d,
0x13, 0x1c, 0xd0, 0x0d, 0xf5, 0x8f, 0x14, 0x98, 0x4b, 0x88, 0x7c, 0x1c, 0x85, 0xbf, 0x07, 0x88,
0xf5, 0xd0, 0xec, 0x77, 0x3b, 0x5c, 0xa7, 0xcf, 0x4a, 0x17, 0xa5, 0xa4, 0x90, 0xb4, 0x69, 0x2b,
0x01, 0xf1, 0xd5, 0x9f, 0x29, 0x70, 0xf2, 0x06, 0x0e, 0x28, 0xea, 0x2a, 0x31, 0x3a, 0x1b, 0x9e,
0xdb, 0xf6, 0xb0, 0xef, 0x3f, 0xbd, 0xfa, 0xf1, 0x7b, 0xcc, 0xb1, 0x93, 0x75, 0x69, 0x1c, 0xf9,
0x9f, 0x81, 0x0a, 0x6d, 0x03, 0x9b, 0xba, 0xe7, 0xee, 0xf9, 0x5c, 0x8f, 0xca, 0x1c, 0xa6, 0xb9,
0x7b, 0x54, 0x21, 0x02, 0x37, 0x30, 0x6c, 0x86, 0xc0, 0x57, 0x14, 0x0a, 0x21, 0xc5, 0x74, 0x0e,
0x86, 0x8c, 0x91, 0xca, 0xf1, 0xd3, 0x2b, 0xe3, 0x3f, 0x51, 0x60, 0x2e, 0xd1, 0x95, 0x71, 0x64,
0x7b, 0x89, 0xb9, 0x9d, 0xac, 0x33, 0xb5, 0x95, 0xd3, 0x52, 0x1a, 0xa1, 0x31, 0x86, 0x8d, 0x4e,
0x43, 0x79, 0xdb, 0xb0, 0x6c, 0xdd, 0xc3, 0x86, 0xef, 0x3a, 0xbc, 0xa3, 0x40, 0x40, 0x1a, 0x85,
0xa8, 0x3f, 0x55, 0x58, 0x78, 0xf0, 0x29, 0xb7, 0x78, 0x7f, 0x9c, 0x81, 0xea, 0xba, 0xe3, 0x63,
0x2f, 0x38, 0xfe, 0x5b, 0x13, 0xf4, 0x36, 0x94, 0x69, 0xc7, 0x7c, 0xdd, 0x34, 0x02, 0x83, 0xaf,
0x66, 0xcf, 0x4a, 0xe3, 0x04, 0xd7, 0x09, 0xde, 0x9a, 0x11, 0x18, 0x1a, 0x93, 0x8e, 0x4f, 0x7e,
0xa3, 0x13, 0x50, 0xda, 0x31, 0xfc, 0x1d, 0xfd, 0x01, 0xde, 0x67, 0xfe, 0x62, 0x55, 0x2b, 0x12,
0xc0, 0x7b, 0x78, 0xdf, 0x47, 0xcf, 0x40, 0xd1, 0xe9, 0x75, 0xd8, 0x04, 0x2b, 0x2c, 0x2a, 0x4b,
0x55, 0xad, 0xe0, 0xf4, 0x3a, 0x74, 0x7a, 0xfd, 0x53, 0x06, 0x6a, 0xb7, 0x7b, 0x64, 0x23, 0x44,
0xa3, 0x1c, 0x3d, 0x3b, 0x38, 0x9a, 0x32, 0x9e, 0x83, 0x2c, 0x73, 0x29, 0x08, 0x45, 0x43, 0xca,
0xf8, 0xfa, 0x9a, 0xaf, 0x11, 0x24, 0x7a, 0xc2, 0xdf, 0x6b, 0xb5, 0xb8, 0x77, 0x96, 0xa5, 0xcc,
0x96, 0x08, 0x84, 0xf9, 0x66, 0x27, 0xa0, 0x84, 0x3d, 0x2f, 0xf2, 0xdd, 0x68, 0x57, 0xb0, 0xe7,
0xb1, 0x42, 0x15, 0x2a, 0x46, 0xeb, 0x81, 0xe3, 0xee, 0xd9, 0xd8, 0x6c, 0x63, 0x93, 0x0e, 0x7b,
0x51, 0x8b, 0xc1, 0x98, 0x62, 0x90, 0x81, 0xd7, 0x5b, 0x4e, 0x40, 0x57, 0xf5, 0x2c, 0x51, 0x0c,
0x02, 0xb9, 0xe6, 0x04, 0xa4, 0xd8, 0xc4, 0x36, 0x0e, 0x30, 0x2d, 0x2e, 0xb0, 0x62, 0x06, 0xe1,
0xc5, 0xbd, 0x6e, 0x44, 0x5d, 0x64, 0xc5, 0x0c, 0x42, 0x8a, 0x4f, 0x42, 0xa9, 0x1f, 0xc6, 0x28,
0xf5, 0x0f, 0x2b, 0x29, 0x40, 0xfd, 0x42, 0x81, 0xea, 0x1a, 0xad, 0xea, 0x29, 0x50, 0x3a, 0x04,
0x93, 0xf8, 0x51, 0xd7, 0xe3, 0x53, 0x87, 0xfe, 0x1e, 0xaa, 0x47, 0xea, 0xff, 0x67, 0xa1, 0xba,
0x89, 0x0d, 0xaf, 0xb5, 0xf3, 0x54, 0x9c, 0xc6, 0xd4, 0x21, 0x6b, 0xfa, 0x36, 0xef, 0x1e, 0xf9,
0x89, 0xce, 0xc3, 0x74, 0xd7, 0x36, 0x5a, 0x78, 0xc7, 0xb5, 0x4d, 0xec, 0xe9, 0x6d, 0xcf, 0xed,
0xb1, 0xe0, 0x55, 0x45, 0xab, 0x0b, 0x05, 0x37, 0x08, 0x1c, 0xbd, 0x0e, 0x45, 0xd3, 0xb7, 0x75,
0xba, 0x8d, 0x2d, 0x50, 0x13, 0x2b, 0xef, 0xdf, 0x9a, 0x6f, 0xd3, 0x5d, 0x6c, 0xc1, 0x64, 0x3f,
0xd0, 0x73, 0x50, 0x75, 0x7b, 0x41, 0xb7, 0x17, 0xe8, 0x6c, 0x82, 0x36, 0x8a, 0x94, 0xbd, 0x0a,
0x03, 0xd2, 0xf9, 0xeb, 0xa3, 0xeb, 0x50, 0xf5, 0xa9, 0x28, 0x43, 0x0f, 0xb6, 0x34, 0xaa, 0x27,
0x55, 0x61, 0x74, 0xdc, 0x85, 0x7d, 0x09, 0xea, 0x81, 0x67, 0xec, 0x62, 0x5b, 0x88, 0xae, 0x01,
0x55, 0xcb, 0x29, 0x06, 0xef, 0x47, 0xd6, 0x2e, 0xc0, 0x4c, 0xbb, 0x67, 0x78, 0x86, 0x13, 0x60,
0x2c, 0x60, 0x97, 0x29, 0x36, 0x8a, 0x8a, 0xfa, 0x04, 0x35, 0xc8, 0x38, 0x0f, 0x69, 0x94, 0x2a,
0xab, 0x65, 0x9c, 0x87, 0xea, 0x7b, 0x30, 0x79, 0xd3, 0x0a, 0xa8, 0x60, 0xc9, 0x64, 0x57, 0xe8,
0xfe, 0x81, 0x4e, 0xe9, 0x67, 0xa0, 0xe8, 0xb9, 0x7b, 0xcc, 0x78, 0x11, 0xef, 0xaa, 0xa2, 0x15,
0x3c, 0x77, 0x8f, 0x5a, 0x26, 0x9a, 0xc7, 0xe0, 0x7a, 0x98, 0xf9, 0x8a, 0x19, 0x8d, 0x7f, 0xa9,
0x7f, 0xa1, 0xf4, 0x95, 0x89, 0xd8, 0x1d, 0xff, 0x68, 0x86, 0xe7, 0x6d, 0x28, 0x78, 0x8c, 0x7e,
0x68, 0x74, 0x55, 0x6c, 0x89, 0x1a, 0xcf, 0x90, 0x6a, 0x64, 0xbd, 0x53, 0xbf, 0xad, 0x40, 0xe5,
0xba, 0xdd, 0xf3, 0x1f, 0x87, 0xf2, 0xcb, 0xc2, 0x05, 0x59, 0x69, 0xb8, 0x40, 0xfd, 0x5e, 0x06,
0xaa, 0x9c, 0x8d, 0x71, 0xbc, 0x87, 0x54, 0x56, 0x36, 0xa1, 0x4c, 0x9a, 0xd4, 0x7d, 0xdc, 0x0e,
0x0f, 0x43, 0xca, 0x2b, 0x2b, 0x52, 0x5f, 0x39, 0xc6, 0x06, 0x0d, 0x60, 0x6f, 0x52, 0xa2, 0x77,
0x9d, 0xc0, 0xdb, 0xd7, 0xa0, 0x15, 0x01, 0x9a, 0xf7, 0x61, 0x2a, 0x51, 0x4c, 0x94, 0xe8, 0x01,
0xde, 0xe7, 0xbb, 0x0e, 0xf2, 0x13, 0xbd, 0x26, 0xe6, 0x1b, 0xa4, 0x2d, 0x7f, 0xb7, 0x5c, 0xa7,
0x7d, 0xd5, 0xf3, 0x8c, 0x7d, 0x9e, 0x8f, 0x70, 0x25, 0xf3, 0x55, 0x45, 0xfd, 0xfb, 0x0c, 0x54,
0xde, 0xef, 0x61, 0x6f, 0xff, 0x49, 0xda, 0xa5, 0xd0, 0x9c, 0x4e, 0x0a, 0xe6, 0x74, 0xc0, 0x14,
0xe4, 0x24, 0xa6, 0x40, 0x62, 0xd0, 0xf2, 0x52, 0x83, 0x26, 0x9b, 0xeb, 0x85, 0x43, 0xcd, 0xf5,
0x62, 0xda, 0x5c, 0x57, 0xff, 0x5c, 0x89, 0x44, 0x38, 0xd6, 0x6c, 0x8c, 0xf9, 0x31, 0x99, 0x43,
0xfb, 0x31, 0x23, 0xcf, 0xc6, 0x1f, 0x29, 0x50, 0xfa, 0x06, 0x6e, 0x05, 0xae, 0x47, 0xec, 0x8f,
0x84, 0x4c, 0x19, 0xc1, 0xa7, 0xcc, 0x24, 0x7d, 0xca, 0x8b, 0x50, 0xb4, 0x4c, 0xdd, 0x20, 0xfa,
0x45, 0xdb, 0x1d, 0xe6, 0xcb, 0x14, 0x2c, 0x93, 0x2a, 0xe2, 0xe8, 0xa7, 0xe7, 0x9f, 0x29, 0x50,
0x61, 0x3c, 0xfb, 0x8c, 0xf2, 0x0d, 0xa1, 0x39, 0x45, 0xa6, 0xf4, 0xfc, 0x23, 0xea, 0xe8, 0xcd,
0x89, 0x7e, 0xb3, 0x57, 0x01, 0x88, 0x90, 0x39, 0x39, 0x9b, 0x33, 0x8b, 0x52, 0x6e, 0x19, 0x39,
0x15, 0xf8, 0xcd, 0x09, 0xad, 0x44, 0xa8, 0x68, 0x15, 0xab, 0x05, 0xc8, 0x51, 0x6a, 0xf5, 0xff,
0x14, 0x98, 0xb9, 0x66, 0xd8, 0xad, 0x35, 0xcb, 0x0f, 0x0c, 0xa7, 0x35, 0x86, 0xf7, 0x72, 0x05,
0x0a, 0x6e, 0x57, 0xb7, 0xf1, 0x76, 0xc0, 0x59, 0x3a, 0x33, 0xa4, 0x47, 0x4c, 0x0c, 0x5a, 0xde,
0xed, 0xde, 0xc2, 0xdb, 0x01, 0x7a, 0x13, 0x8a, 0x6e, 0x57, 0xf7, 0xac, 0xf6, 0x4e, 0xc0, 0xa5,
0x3f, 0x02, 0x71, 0xc1, 0xed, 0x6a, 0x84, 0x42, 0x38, 0x94, 0x98, 0x3c, 0xe4, 0xa1, 0x84, 0xfa,
0xaf, 0x03, 0xdd, 0x1f, 0x63, 0x0e, 0x5c, 0x81, 0xa2, 0xe5, 0x04, 0xba, 0x69, 0xf9, 0xa1, 0x08,
0x4e, 0xc9, 0x75, 0xc8, 0x09, 0x68, 0x0f, 0xe8, 0x98, 0x3a, 0x01, 0x69, 0x1b, 0xbd, 0x03, 0xb0,
0x6d, 0xbb, 0x06, 0xa7, 0x66, 0x32, 0x38, 0x2d, 0x9f, 0x3e, 0x04, 0x2d, 0xa4, 0x2f, 0x51, 0x22,
0x52, 0x43, 0x7f, 0x48, 0xff, 0x45, 0x81, 0xb9, 0x0d, 0xec, 0xb1, 0x24, 0x94, 0x80, 0x9f, 0x1f,
0xae, 0x3b, 0xdb, 0x6e, 0xfc, 0x08, 0x57, 0x49, 0x1c, 0xe1, 0xfe, 0x62, 0x8e, 0x2d, 0x63, 0x5b,
0x0e, 0x16, 0x48, 0x08, 0xb7, 0x1c, 0x61, 0xb8, 0x84, 0x6d, 0xd9, 0x6a, 0x29, 0xc3, 0xc4, 0xf9,
0x15, 0x77, 0xae, 0xea, 0xef, 0xb2, 0x04, 0x09, 0x69, 0xa7, 0x8e, 0xae, 0xb0, 0xf3, 0xc0, 0x2d,
0x7d, 0xc2, 0xee, 0xbf, 0x00, 0x09, 0xdb, 0x91, 0x62, 0x88, 0x7e, 0xa0, 0xc0, 0x62, 0x3a, 0x57,
0xe3, 0x2c, 0xd1, 0xef, 0x40, 0xce, 0x72, 0xb6, 0xdd, 0xf0, 0xbc, 0xea, 0x9c, 0x74, 0x2e, 0xc8,
0xdb, 0x65, 0x84, 0xea, 0xcf, 0x32, 0x50, 0x7f, 0x9f, 0x45, 0xdf, 0xbf, 0xf4, 0xe1, 0xef, 0xe0,
0x8e, 0xee, 0x5b, 0x1f, 0xe3, 0x70, 0xf8, 0x3b, 0xb8, 0xb3, 0x69, 0x7d, 0x8c, 0x63, 0x9a, 0x91,
0x8b, 0x6b, 0xc6, 0xf0, 0xe3, 0x58, 0xf1, 0x3c, 0xb2, 0x10, 0x3f, 0x8f, 0x9c, 0x87, 0xbc, 0xe3,
0x9a, 0x78, 0x7d, 0x8d, 0xef, 0xd7, 0xf8, 0x57, 0x5f, 0xd5, 0x4a, 0x87, 0x53, 0x35, 0xd2, 0x14,
0xad, 0xc2, 0x64, 0x39, 0x64, 0x84, 0x47, 0xf6, 0xa9, 0x7e, 0x47, 0x81, 0xe6, 0x0d, 0x1c, 0x24,
0xa5, 0xfa, 0xe4, 0xf4, 0xef, 0xbb, 0x0a, 0x9c, 0x90, 0x32, 0x34, 0x8e, 0xea, 0xbd, 0x11, 0x57,
0x3d, 0xf9, 0x51, 0xe9, 0x40, 0x93, 0x5c, 0xeb, 0x5e, 0x85, 0xca, 0x5a, 0xaf, 0xd3, 0x89, 0x9c,
0xb1, 0x33, 0x50, 0xf1, 0xd8, 0x4f, 0xb6, 0x99, 0x62, 0x2b, 0x73, 0x99, 0xc3, 0xc8, 0x96, 0x49,
0x3d, 0x0f, 0x55, 0x4e, 0xc2, 0xb9, 0x6e, 0x42, 0xd1, 0xe3, 0xbf, 0x39, 0x7e, 0xf4, 0xad, 0xce,
0xc1, 0x8c, 0x86, 0xdb, 0x44, 0xe9, 0xbd, 0x5b, 0x96, 0xf3, 0x80, 0x37, 0xa3, 0x7e, 0x4b, 0x81,
0xd9, 0x38, 0x9c, 0xd7, 0x75, 0x19, 0x0a, 0x86, 0x69, 0x7a, 0xd8, 0xf7, 0x87, 0x0e, 0xcb, 0x55,
0x86, 0xa3, 0x85, 0xc8, 0x82, 0xe4, 0x32, 0x23, 0x4b, 0x4e, 0xd5, 0x61, 0xfa, 0x06, 0x0e, 0x6e,
0xe3, 0xc0, 0x1b, 0x2b, 0x28, 0xde, 0x20, 0xdb, 0x1a, 0x4a, 0xcc, 0xd5, 0x22, 0xfc, 0x54, 0x3f,
0x55, 0x00, 0x89, 0x2d, 0x8c, 0x33, 0xcc, 0xa2, 0x94, 0x33, 0x71, 0x29, 0xb3, 0xb4, 0xa4, 0x4e,
0xd7, 0x75, 0xb0, 0x13, 0x88, 0x8e, 0x58, 0x35, 0x82, 0x86, 0x99, 0x1a, 0xe8, 0x96, 0x6b, 0x98,
0xab, 0x86, 0x3d, 0x9e, 0xe3, 0x70, 0x0a, 0xc0, 0xf7, 0x5a, 0x3a, 0x9f, 0xc7, 0x19, 0x6e, 0x97,
0xbc, 0xd6, 0x1d, 0x36, 0x95, 0x4f, 0x43, 0xd9, 0xf4, 0x03, 0x5e, 0x1c, 0xc6, 0x68, 0xc1, 0xf4,
0x03, 0x56, 0x4e, 0x13, 0x53, 0x7d, 0x6c, 0xd8, 0xd8, 0xd4, 0x85, 0x10, 0xd7, 0x24, 0x45, 0xab,
0xb3, 0x82, 0xcd, 0x08, 0x2e, 0x99, 0x5c, 0x39, 0xe9, 0xe4, 0xba, 0x0f, 0x0b, 0xb7, 0x0d, 0xa7,
0x67, 0xd8, 0xd7, 0xdc, 0x4e, 0xd7, 0x88, 0xe5, 0x32, 0x26, 0x0d, 0xa5, 0x22, 0x31, 0x94, 0xcf,
0xb2, 0xd4, 0x36, 0xe6, 0x9c, 0xd3, 0x3e, 0x4d, 0x6a, 0x02, 0x44, 0xf5, 0xa1, 0x31, 0x58, 0xfd,
0x38, 0x03, 0x4a, 0x99, 0x0a, 0xab, 0x12, 0xad, 0x77, 0x1f, 0xa6, 0xbe, 0x0d, 0xcf, 0xd0, 0x34,
0xc3, 0x10, 0x14, 0x3b, 0x55, 0x4f, 0x56, 0xa0, 0x48, 0x2a, 0xf8, 0x8d, 0x0c, 0x35, 0x81, 0x03,
0x35, 0x8c, 0xc3, 0xf8, 0x95, 0xf8, 0x61, 0xf6, 0xf3, 0x29, 0x59, 0xb6, 0xf1, 0x16, 0xb9, 0xb1,
0x5e, 0x82, 0x29, 0xfc, 0x08, 0xb7, 0x7a, 0x81, 0xe5, 0xb4, 0x37, 0x6c, 0xc3, 0xb9, 0xe3, 0xf2,
0x25, 0x29, 0x09, 0x46, 0xcf, 0x43, 0x95, 0x48, 0xdf, 0xed, 0x05, 0x1c, 0x8f, 0xad, 0x4d, 0x71,
0x20, 0xa9, 0x8f, 0xf4, 0xd7, 0xc6, 0x01, 0x36, 0x39, 0x1e, 0x5b, 0xa8, 0x92, 0xe0, 0x01, 0x51,
0x12, 0xb0, 0x7f, 0x18, 0x51, 0xfe, 0x9b, 0x92, 0x10, 0x25, 0xaf, 0xe1, 0x49, 0x89, 0xf2, 0x26,
0x40, 0x07, 0x7b, 0x6d, 0xbc, 0x4e, 0x8d, 0x3f, 0xdb, 0xfb, 0x2f, 0x49, 0x8d, 0x7f, 0xbf, 0x82,
0xdb, 0x21, 0x81, 0x26, 0xd0, 0xaa, 0x37, 0x60, 0x46, 0x82, 0x42, 0xec, 0x9a, 0xef, 0xf6, 0xbc,
0x16, 0x0e, 0x8f, 0x8f, 0xc2, 0x4f, 0xb2, 0x0e, 0x06, 0x86, 0xd7, 0xc6, 0x01, 0x57, 0x5a, 0xfe,
0xa5, 0x5e, 0xa6, 0xf1, 0x1f, 0x7a, 0xd4, 0x10, 0xd3, 0xd4, 0x78, 0x2c, 0x5b, 0x19, 0x88, 0x65,
0x6f, 0xd3, 0x60, 0x8b, 0x48, 0x37, 0x66, 0x1e, 0xc2, 0x36, 0xa9, 0x0a, 0x9b, 0xfc, 0xaa, 0x45,
0xf8, 0xa9, 0xfe, 0x8f, 0x02, 0xd5, 0xf5, 0x4e, 0xd7, 0xed, 0xc7, 0x19, 0x46, 0xde, 0x8c, 0x0e,
0x9e, 0xd3, 0x66, 0x64, 0xe7, 0xb4, 0xcf, 0x41, 0x35, 0x9e, 0x9f, 0xcf, 0x4e, 0x86, 0x2a, 0x2d,
0x31, 0x2f, 0xff, 0x04, 0x94, 0x3c, 0x77, 0x4f, 0x27, 0xa6, 0xd4, 0xe4, 0x19, 0x0f, 0x45, 0xcf,
0xdd, 0x23, 0x06, 0xd6, 0x44, 0xb3, 0x90, 0xdb, 0xb6, 0xec, 0x28, 0x59, 0x87, 0x7d, 0xa0, 0x37,
0xc8, 0x56, 0x8d, 0x45, 0x44, 0xf3, 0xa3, 0xee, 0x98, 0x42, 0x0a, 0xf5, 0x43, 0xa8, 0x85, 0xbd,
0x1e, 0xf3, 0x8e, 0x49, 0x60, 0xf8, 0x0f, 0xc2, 0x64, 0x04, 0xf6, 0xa1, 0x9e, 0x67, 0x81, 0x32,
0x5a, 0x7f, 0x6c, 0xd0, 0x11, 0x4c, 0x12, 0x0c, 0x3e, 0x97, 0xe8, 0x6f, 0xf5, 0x9f, 0x33, 0x30,
0x9f, 0xc4, 0x1e, 0x87, 0xa5, 0xcb, 0xf1, 0xf9, 0x23, 0xbf, 0x3d, 0x20, 0xb6, 0xc6, 0xe7, 0x0e,
0x1f, 0x81, 0x96, 0xdb, 0x73, 0x02, 0x6e, 0x80, 0xc8, 0x08, 0x5c, 0x23, 0xdf, 0x68, 0x01, 0x0a,
0x96, 0xa9, 0xdb, 0x64, 0x57, 0xc7, 0xd6, 0xa4, 0xbc, 0x65, 0xde, 0x22, 0x3b, 0xbe, 0xd7, 0x43,
0x4f, 0x6b, 0xe4, 0x0c, 0x06, 0x86, 0x8f, 0x6a, 0x90, 0xb1, 0x4c, 0x1e, 0xdd, 0xc8, 0x58, 0x26,
0x51, 0x26, 0x7a, 0x1c, 0x40, 0xb3, 0x6d, 0x79, 0xfa, 0x2d, 0xd1, 0x82, 0x2a, 0x81, 0xbe, 0x1f,
0x02, 0x89, 0x33, 0x46, 0xd1, 0x78, 0x08, 0x96, 0x3a, 0xcc, 0x45, 0xad, 0x4c, 0x60, 0xeb, 0x0c,
0xa4, 0x36, 0x60, 0x9e, 0xb0, 0xc6, 0xba, 0x78, 0x8f, 0x0c, 0x48, 0xe8, 0x62, 0x7d, 0x4f, 0x81,
0x85, 0x81, 0xa2, 0x71, 0x64, 0x7d, 0x55, 0x1c, 0xfe, 0xf2, 0xca, 0x79, 0xa9, 0xa9, 0x91, 0x0f,
0x6e, 0xa8, 0x2b, 0xdf, 0x67, 0xfe, 0x90, 0xc6, 0x32, 0x2c, 0x1f, 0x73, 0xbe, 0xce, 0x12, 0xd4,
0xf7, 0xac, 0x60, 0x47, 0xa7, 0xf7, 0x51, 0xa8, 0x33, 0xc2, 0x42, 0xd6, 0x45, 0xad, 0x46, 0xe0,
0x9b, 0x04, 0x4c, 0x1c, 0x12, 0x5f, 0xfd, 0x4d, 0x05, 0x66, 0x62, 0x6c, 0x8d, 0x23, 0xa6, 0x37,
0x89, 0x9f, 0xc6, 0x2a, 0xe2, 0x92, 0x5a, 0x94, 0x4a, 0x8a, 0xb7, 0x46, 0x8d, 0x71, 0x44, 0xa1,
0xfe, 0xbb, 0x02, 0x65, 0xa1, 0x84, 0x6c, 0x00, 0x79, 0x59, 0x7f, 0x03, 0x18, 0x01, 0x46, 0x12,
0xc3, 0x73, 0xd0, 0x37, 0x51, 0x42, 0xc6, 0xba, 0x90, 0x32, 0x67, 0xfa, 0xe8, 0x26, 0xd4, 0x98,
0x98, 0x22, 0xd6, 0xa5, 0xe7, 0x32, 0x51, 0x32, 0xa0, 0xe1, 0x99, 0x9c, 0x4b, 0xad, 0xea, 0x0b,
0x5f, 0x2c, 0x7e, 0xe9, 0x9a, 0x98, 0xb6, 0x94, 0x1b, 0xd8, 0x8e, 0x55, 0x44, 0x52, 0xe2, 0xd2,
0xda, 0xd8, 0x30, 0xb1, 0x17, 0xf5, 0x2d, 0xfa, 0x26, 0x3e, 0x24, 0xfb, 0xad, 0x13, 0x17, 0x9f,
0x1b, 0x5b, 0x60, 0x20, 0xe2, 0xfd, 0xa3, 0x17, 0x60, 0xca, 0xec, 0xc4, 0x2e, 0x43, 0x85, 0x4e,
0xaf, 0xd9, 0x11, 0x6e, 0x41, 0xc5, 0x18, 0x9a, 0x8c, 0x33, 0xf4, 0xdf, 0x4a, 0x74, 0x8d, 0xd4,
0xc3, 0x26, 0x76, 0x02, 0xcb, 0xb0, 0x8f, 0xae, 0x93, 0x4d, 0x28, 0xf6, 0x7c, 0xec, 0x09, 0x6b,
0x43, 0xf4, 0x4d, 0xca, 0xba, 0x86, 0xef, 0xef, 0xb9, 0x9e, 0xc9, 0xb9, 0x8c, 0xbe, 0x87, 0xe4,
0x1f, 0xb2, 0x0b, 0x89, 0xf2, 0xfc, 0xc3, 0xcb, 0xb0, 0xd0, 0x71, 0x4d, 0x6b, 0xdb, 0x92, 0xa5,
0x2d, 0x12, 0xb2, 0xb9, 0xb0, 0x38, 0x46, 0xa7, 0xfe, 0x20, 0x03, 0x0b, 0x1f, 0x74, 0xcd, 0x2f,
0xa1, 0xcf, 0x8b, 0x50, 0x76, 0x6d, 0x73, 0x23, 0xde, 0x6d, 0x11, 0x44, 0x30, 0x1c, 0xbc, 0x17,
0x61, 0xb0, 0xc3, 0x78, 0x11, 0x34, 0x34, 0x37, 0xf3, 0x48, 0xb2, 0xc9, 0x0f, 0x93, 0x4d, 0x1b,
0x16, 0x58, 0x40, 0xf8, 0x31, 0x8b, 0x46, 0xfd, 0x08, 0xe6, 0x88, 0x69, 0x26, 0xcd, 0x7c, 0xe0,
0x63, 0x6f, 0x4c, 0x8b, 0x73, 0x12, 0x4a, 0x61, 0xcd, 0x61, 0xda, 0x6c, 0x1f, 0xa0, 0xde, 0x84,
0xd9, 0x44, 0x5b, 0x47, 0xec, 0x91, 0xba, 0x08, 0xa0, 0xb9, 0x36, 0x7e, 0xd7, 0x09, 0xac, 0x60,
0x9f, 0x2c, 0xef, 0x82, 0xbb, 0x44, 0x7f, 0x13, 0x0c, 0xd2, 0xc6, 0x10, 0x8c, 0x5f, 0x85, 0x69,
0x36, 0xe3, 0x48, 0x4d, 0x47, 0x17, 0xee, 0xeb, 0x90, 0xc7, 0xb4, 0x11, 0xbe, 0xdd, 0x3f, 0x2d,
0x37, 0xb1, 0x11, 0xb7, 0x1a, 0x47, 0x57, 0x7f, 0x05, 0xa6, 0xd6, 0x3c, 0xb7, 0x3b, 0x5e, 0xeb,
0xd4, 0x7b, 0xb0, 0xb1, 0xe8, 0x06, 0x16, 0x09, 0x80, 0x6e, 0x43, 0xff, 0x41, 0x81, 0xf9, 0xbb,
0x5d, 0xec, 0x19, 0x01, 0x26, 0xb2, 0x18, 0xaf, 0xa5, 0x61, 0xf3, 0x2b, 0xc6, 0x45, 0x36, 0xce,
0x05, 0x7a, 0x33, 0x76, 0x13, 0x4b, 0xbe, 0x2d, 0x48, 0x70, 0x29, 0xa4, 0x77, 0xff, 0x99, 0x02,
0xd3, 0x9b, 0x98, 0xac, 0x21, 0xe3, 0xb1, 0x7f, 0x11, 0x26, 0x09, 0x47, 0xa3, 0x0e, 0x12, 0x45,
0x46, 0xe7, 0x60, 0xda, 0x72, 0x5a, 0x76, 0xcf, 0xc4, 0x3a, 0xe9, 0xab, 0x4e, 0x5c, 0x28, 0xbe,
0x70, 0x4f, 0xf1, 0x02, 0xc2, 0x32, 0x59, 0x1e, 0xd5, 0x47, 0x4c, 0x25, 0xa3, 0x6c, 0x18, 0xd6,
0x9c, 0x72, 0x98, 0xe6, 0x2e, 0x41, 0x8e, 0x34, 0x13, 0x2e, 0xd6, 0x72, 0xaa, 0xbe, 0x56, 0x6b,
0x0c, 0x5b, 0xfd, 0xb6, 0x02, 0x48, 0x14, 0xd1, 0x38, 0x13, 0xf8, 0x6b, 0x62, 0x5c, 0x3c, 0x3b,
0x94, 0x75, 0xd6, 0xd3, 0x28, 0x22, 0x2e, 0x8c, 0x14, 0x1d, 0xc6, 0x71, 0x46, 0x8a, 0xf4, 0x6b,
0xe8, 0x48, 0x09, 0x42, 0xa0, 0xc8, 0xe2, 0x48, 0x51, 0x4d, 0x94, 0x8c, 0x14, 0xe1, 0x39, 0x1c,
0x29, 0xc6, 0x61, 0x38, 0x52, 0xb4, 0x39, 0xe5, 0x30, 0xcd, 0x5d, 0x82, 0x1c, 0x69, 0xe6, 0x60,
0x21, 0x85, 0x23, 0x45, 0xb1, 0xd5, 0x5f, 0x8b, 0x46, 0x8a, 0x33, 0x30, 0xd6, 0xf5, 0x91, 0x3c,
0x93, 0xfc, 0x81, 0xda, 0xc2, 0x07, 0x8a, 0xa3, 0xab, 0xcf, 0x43, 0x4d, 0xc3, 0x6c, 0xfb, 0xdc,
0xb7, 0x8e, 0xc2, 0x09, 0x2b, 0x9b, 0x77, 0x67, 0x61, 0x6a, 0xc3, 0xb3, 0x76, 0x2d, 0x1b, 0xb7,
0x87, 0x99, 0xd9, 0xbf, 0x55, 0x60, 0x8e, 0xeb, 0x1e, 0xaf, 0xf3, 0xe8, 0x03, 0xff, 0x46, 0xc2,
0x92, 0x3e, 0x97, 0xe2, 0xac, 0x8a, 0xbc, 0x87, 0xd6, 0x94, 0x2c, 0xcf, 0xa1, 0x02, 0x74, 0x43,
0xbe, 0x45, 0x2d, 0x98, 0xe5, 0xa5, 0x51, 0xa7, 0xa8, 0x2a, 0x7c, 0xa6, 0xf4, 0x85, 0xc1, 0xf5,
0xe1, 0x6d, 0x7a, 0xb8, 0x49, 0x21, 0x9c, 0xf7, 0x91, 0xf8, 0x88, 0x88, 0xd0, 0x1a, 0x40, 0xc4,
0x41, 0xa8, 0x20, 0xcf, 0xcb, 0x83, 0x30, 0x71, 0x01, 0x6b, 0x02, 0x1d, 0xbd, 0xa1, 0x91, 0x14,
0xec, 0x78, 0xaf, 0x09, 0x24, 0x26, 0xf6, 0xf0, 0x5e, 0x25, 0x27, 0xf7, 0x4f, 0x14, 0xaa, 0x0f,
0x4e, 0xcb, 0xea, 0x1a, 0x36, 0xd7, 0x87, 0xb3, 0x50, 0xeb, 0x86, 0x20, 0xf1, 0x88, 0xbe, 0x1a,
0x41, 0x69, 0x5e, 0xd3, 0xa5, 0x43, 0xcd, 0xe7, 0x9b, 0x13, 0xd1, 0x14, 0x63, 0x16, 0x34, 0x3b,
0x92, 0x05, 0x25, 0x64, 0x04, 0x7d, 0xb5, 0x0c, 0xa5, 0xa8, 0x79, 0xb2, 0x9b, 0xaa, 0xde, 0xf0,
0x0c, 0x27, 0x70, 0x43, 0x47, 0xe0, 0x48, 0xb3, 0x7d, 0x95, 0xd6, 0xc9, 0x46, 0x86, 0x77, 0x63,
0xb4, 0x01, 0xed, 0x93, 0xa9, 0xff, 0xab, 0x40, 0x99, 0xb2, 0xc2, 0x19, 0x59, 0x15, 0xf8, 0xe4,
0xdc, 0xa4, 0xd6, 0x29, 0x4a, 0x5d, 0xeb, 0x93, 0xc5, 0x54, 0x35, 0x73, 0x14, 0x55, 0x7d, 0x0e,
0xaa, 0xe1, 0x6f, 0x71, 0xed, 0xae, 0x84, 0x40, 0xbe, 0x7e, 0x17, 0xda, 0x4c, 0x86, 0x74, 0x09,
0x2f, 0xaf, 0xa8, 0xf2, 0xed, 0xb6, 0x28, 0x67, 0x2d, 0x24, 0x51, 0x3f, 0x89, 0x4c, 0x1e, 0x45,
0x38, 0xba, 0x75, 0xf8, 0x6a, 0xc2, 0x3a, 0x2c, 0xa6, 0x73, 0x91, 0x70, 0xb4, 0xc8, 0x9e, 0x3a,
0xc6, 0xc2, 0x98, 0x7b, 0x6a, 0x5a, 0xad, 0x85, 0x87, 0xef, 0xa9, 0x45, 0x46, 0x22, 0x0a, 0xf5,
0xa7, 0x0a, 0x2c, 0x70, 0x5f, 0x27, 0xd2, 0x95, 0x27, 0x20, 0x12, 0xf4, 0x75, 0x6e, 0xf1, 0xb3,
0xd4, 0x27, 0x7b, 0x69, 0x98, 0x4f, 0x16, 0xf1, 0x29, 0x38, 0x65, 0x67, 0xa1, 0x74, 0x9b, 0x22,
0xbd, 0xfb, 0x28, 0x40, 0x0d, 0x28, 0xec, 0x62, 0xcf, 0xb7, 0x5c, 0x87, 0xcf, 0xff, 0xf0, 0xf3,
0xdc, 0x19, 0x28, 0x86, 0x97, 0xf5, 0x50, 0x01, 0xb2, 0x57, 0x6d, 0xbb, 0x3e, 0x81, 0x2a, 0x50,
0x5c, 0xe7, 0x37, 0xd2, 0xea, 0xca, 0xb9, 0x77, 0x60, 0x46, 0xe2, 0xfb, 0xa1, 0x69, 0xa8, 0x5e,
0x35, 0xe9, 0x26, 0xe1, 0x9e, 0x4b, 0x80, 0xf5, 0x09, 0x34, 0x0f, 0x48, 0xc3, 0x1d, 0x77, 0x97,
0x22, 0x5e, 0xf7, 0xdc, 0x0e, 0x85, 0x2b, 0xe7, 0x5e, 0x86, 0x59, 0x19, 0xa7, 0xa8, 0x04, 0x39,
0xda, 0xf3, 0xfa, 0x04, 0x02, 0xc8, 0x6b, 0x78, 0xd7, 0x7d, 0x80, 0xeb, 0xca, 0xca, 0x0f, 0xcf,
0x43, 0x95, 0xf1, 0xce, 0xaf, 0x96, 0x23, 0x1d, 0xea, 0xc9, 0x07, 0x9c, 0xd0, 0x57, 0xe4, 0x87,
0xd7, 0xf2, 0x77, 0x9e, 0x9a, 0xc3, 0x14, 0x47, 0x9d, 0x40, 0x1f, 0x42, 0x2d, 0xfe, 0xfa, 0x11,
0x92, 0xc7, 0xe4, 0xa5, 0x4f, 0x24, 0x1d, 0x54, 0xb9, 0x0e, 0xd5, 0xd8, 0x1b, 0x46, 0x48, 0x3e,
0x98, 0xb2, 0x77, 0x8e, 0x9a, 0xf2, 0x23, 0x15, 0xf1, 0x79, 0x21, 0xc6, 0x7d, 0xfc, 0x59, 0x91,
0x14, 0xee, 0xa5, 0x6f, 0x8f, 0x1c, 0xc4, 0xbd, 0x01, 0xd3, 0x03, 0xaf, 0x7e, 0xa0, 0x97, 0x53,
0x8c, 0x98, 0xfc, 0x75, 0x90, 0x83, 0x9a, 0xd8, 0x03, 0x34, 0xf8, 0x32, 0x0f, 0x5a, 0x96, 0x8f,
0x40, 0xda, 0x03, 0x45, 0xcd, 0x0b, 0x23, 0xe3, 0x47, 0x82, 0xfb, 0x75, 0x05, 0x16, 0x52, 0x1e,
0xe6, 0x40, 0x17, 0xd3, 0x4e, 0x2c, 0x87, 0x3c, 0x33, 0xd2, 0x7c, 0xed, 0x70, 0x44, 0x11, 0x23,
0x0e, 0x4c, 0x25, 0x1e, 0xa9, 0x40, 0xe7, 0x53, 0x6f, 0xd6, 0x0e, 0x3e, 0xda, 0xd1, 0xfc, 0xca,
0x68, 0xc8, 0x51, 0x7b, 0xf7, 0x61, 0x2a, 0xf1, 0x42, 0x43, 0x4a, 0x7b, 0xf2, 0x77, 0x1c, 0x0e,
0x1a, 0xd0, 0x6f, 0x42, 0x35, 0xf6, 0x94, 0x42, 0x8a, 0xc6, 0xcb, 0x9e, 0x5b, 0x38, 0xa8, 0xea,
0xfb, 0x50, 0x11, 0x5f, 0x3c, 0x40, 0x4b, 0x69, 0x73, 0x69, 0xa0, 0xe2, 0xc3, 0x4c, 0xa5, 0xfe,
0x4d, 0xe5, 0x21, 0x53, 0x69, 0xe0, 0x72, 0xf7, 0xe8, 0x53, 0x49, 0xa8, 0x7f, 0xe8, 0x54, 0x3a,
0x74, 0x13, 0xdf, 0x52, 0x68, 0xc8, 0x44, 0x72, 0x13, 0x1e, 0xad, 0xa4, 0xe9, 0x66, 0xfa, 0x9d,
0xff, 0xe6, 0xc5, 0x43, 0xd1, 0x44, 0x52, 0x7c, 0x00, 0xb5, 0xf8, 0x7d, 0xef, 0x14, 0x29, 0x4a,
0xaf, 0xc8, 0x37, 0xcf, 0x8f, 0x84, 0x1b, 0x35, 0xf6, 0x01, 0x94, 0x85, 0x37, 0x19, 0xd1, 0x8b,
0x43, 0xf4, 0x58, 0x7c, 0xa0, 0xf0, 0x20, 0x49, 0xbe, 0x0f, 0xa5, 0xe8, 0x29, 0x45, 0x74, 0x36,
0x55, 0x7f, 0x0f, 0x53, 0xe5, 0x26, 0x40, 0xff, 0x9d, 0x44, 0xf4, 0x82, 0xb4, 0xce, 0x81, 0x87,
0x14, 0x0f, 0xaa, 0x34, 0xea, 0x3e, 0xbb, 0x46, 0x33, 0xac, 0xfb, 0xe2, 0xbd, 0xaf, 0x83, 0xaa,
0xdd, 0x81, 0x6a, 0xec, 0xb6, 0x66, 0xda, 0x14, 0x96, 0x5c, 0xa2, 0x6d, 0x9e, 0x1b, 0x05, 0x35,
0x1a, 0xbf, 0x1d, 0xa8, 0xc6, 0xee, 0xce, 0xa5, 0xb4, 0x24, 0xbb, 0x2a, 0x98, 0xd2, 0x92, 0xf4,
0x2a, 0x9e, 0x3a, 0x81, 0x3e, 0x11, 0xae, 0xe9, 0xc5, 0xae, 0x42, 0xa2, 0x57, 0x87, 0xd6, 0x23,
0xbb, 0x09, 0xda, 0x5c, 0x39, 0x0c, 0x49, 0xc4, 0x02, 0xd7, 0x2a, 0x26, 0xd2, 0x74, 0xad, 0x3a,
0xcc, 0x48, 0x6d, 0x42, 0x9e, 0xdd, 0x86, 0x43, 0x6a, 0xca, 0xbd, 0x57, 0xe1, 0xaa, 0x5c, 0x53,
0xbe, 0xfd, 0x88, 0x5f, 0x14, 0x63, 0x95, 0xb2, 0xc3, 0xed, 0x94, 0x4a, 0x63, 0x57, 0xa1, 0x46,
0xad, 0x54, 0x83, 0x3c, 0xbb, 0xad, 0x91, 0x52, 0x69, 0xec, 0x06, 0x52, 0x73, 0x38, 0x0e, 0xdb,
0xf3, 0x4e, 0xa0, 0x0d, 0xc8, 0xd1, 0x4c, 0x00, 0x74, 0x66, 0xd8, 0x45, 0x86, 0x61, 0x35, 0xc6,
0xee, 0x3a, 0xa8, 0x13, 0xe8, 0x2e, 0xe4, 0x68, 0x50, 0x35, 0xa5, 0x46, 0xf1, 0x36, 0x42, 0x73,
0x28, 0x4a, 0xc8, 0xa2, 0x09, 0x15, 0x31, 0x05, 0x39, 0x65, 0xc9, 0x92, 0x24, 0x69, 0x37, 0x47,
0xc1, 0x0c, 0x5b, 0x61, 0xd3, 0xa8, 0x9f, 0x15, 0x91, 0x3e, 0x8d, 0x06, 0x32, 0x2e, 0xd2, 0xa7,
0xd1, 0x60, 0x92, 0x85, 0x3a, 0x81, 0x7e, 0x4b, 0x81, 0x46, 0x5a, 0x5e, 0x2c, 0x4a, 0xf5, 0x80,
0x86, 0x25, 0xf7, 0x36, 0x2f, 0x1d, 0x92, 0x2a, 0xe2, 0xe5, 0x63, 0x1a, 0x8b, 0x1d, 0xc8, 0x84,
0xbd, 0x90, 0x56, 0x5f, 0x4a, 0x76, 0x67, 0xf3, 0x95, 0xd1, 0x09, 0xa2, 0xb6, 0xb7, 0xa0, 0x2c,
0xc4, 0x81, 0x53, 0x2c, 0xef, 0x60, 0x00, 0x3b, 0x65, 0x54, 0x25, 0x21, 0x65, 0xa6, 0xde, 0x34,
0x7d, 0x32, 0x45, 0x19, 0xc5, 0x6c, 0xcc, 0x14, 0xf5, 0x8e, 0x65, 0x5f, 0xaa, 0x13, 0x08, 0x43,
0x45, 0xcc, 0xa5, 0x4c, 0xd1, 0x46, 0x49, 0x1a, 0x66, 0xf3, 0xa5, 0x11, 0x30, 0xa3, 0x66, 0x74,
0x80, 0x7e, 0x2e, 0x63, 0xca, 0x5a, 0x37, 0x90, 0x4e, 0xd9, 0x7c, 0xf1, 0x40, 0x3c, 0x71, 0xd9,
0x17, 0xb2, 0x13, 0x53, 0xa4, 0x3f, 0x98, 0xbf, 0x38, 0xc2, 0x5e, 0x64, 0x30, 0x03, 0x2e, 0x65,
0x2f, 0x92, 0x9a, 0x6c, 0xd7, 0xbc, 0x30, 0x32, 0x7e, 0xd4, 0x9f, 0x87, 0x50, 0x4f, 0x66, 0x0c,
0xa6, 0xec, 0x71, 0x53, 0xf2, 0x16, 0x9b, 0x2f, 0x8f, 0x88, 0x2d, 0xae, 0x87, 0x27, 0x06, 0x79,
0xfa, 0x25, 0x2b, 0xd8, 0xa1, 0xc9, 0x6a, 0xa3, 0xf4, 0x5a, 0xcc, 0x8b, 0x1b, 0xa5, 0xd7, 0xb1,
0x2c, 0x38, 0xbe, 0x78, 0xd1, 0x0c, 0x90, 0xb4, 0xc5, 0x4b, 0xcc, 0xbf, 0x4a, 0x59, 0x67, 0xe2,
0xd9, 0x4a, 0xcc, 0xfd, 0x8c, 0x67, 0x96, 0xa0, 0x73, 0x23, 0xa5, 0x9f, 0x0c, 0x73, 0x3f, 0xe5,
0xa9, 0x2a, 0x6c, 0xeb, 0x96, 0x48, 0x9c, 0x49, 0xd9, 0x4a, 0xc9, 0x33, 0x6f, 0x52, 0xb6, 0x6e,
0x29, 0xb9, 0x38, 0x74, 0x62, 0xd5, 0x93, 0x59, 0x08, 0xc3, 0xcf, 0x42, 0x92, 0xd1, 0xe9, 0x83,
0x8f, 0x2b, 0xea, 0xc9, 0x90, 0x7f, 0x4a, 0x03, 0x29, 0x99, 0x01, 0x23, 0x34, 0x90, 0x0c, 0x9c,
0xa7, 0x34, 0x90, 0x12, 0x5f, 0x1f, 0xc1, 0x77, 0x8d, 0x05, 0xb1, 0x53, 0x96, 0x42, 0x59, 0xa0,
0x3b, 0x65, 0x29, 0x94, 0xc6, 0xdf, 0x99, 0x47, 0xdf, 0x0f, 0x50, 0xa7, 0x58, 0xb9, 0x81, 0x08,
0xf6, 0x41, 0xec, 0xdf, 0x85, 0x62, 0x18, 0x75, 0x46, 0xcf, 0xa7, 0xba, 0x88, 0x87, 0xa8, 0xf0,
0x3e, 0x4c, 0x25, 0x4e, 0xf0, 0x52, 0x54, 0x54, 0x1e, 0x89, 0x3e, 0x78, 0x3c, 0xa1, 0x1f, 0xdb,
0x4c, 0x11, 0xc2, 0x40, 0x7c, 0x38, 0xc5, 0xd4, 0x0f, 0x06, 0x49, 0xc5, 0x06, 0x08, 0x63, 0x43,
0x1b, 0x10, 0xc2, 0x9a, 0x43, 0x1b, 0x10, 0x63, 0x7b, 0x7c, 0xbf, 0x1a, 0x0b, 0xe4, 0xa4, 0xed,
0x57, 0x65, 0x61, 0xb4, 0xb4, 0xfd, 0xaa, 0x34, 0x32, 0xc4, 0xd4, 0x3f, 0x79, 0x1a, 0x9a, 0xa2,
0xfe, 0x29, 0xc7, 0xd0, 0x07, 0x8d, 0xc7, 0x16, 0x94, 0x85, 0xb3, 0x74, 0x34, 0x4c, 0x0e, 0xe2,
0x81, 0x7f, 0x8a, 0x5f, 0x22, 0x39, 0x96, 0x57, 0x27, 0x56, 0x7a, 0x50, 0xd9, 0xf0, 0xdc, 0x47,
0xe1, 0xe3, 0x9f, 0x5f, 0x92, 0x57, 0x71, 0xa5, 0x05, 0x35, 0x86, 0xa0, 0xe3, 0x47, 0x81, 0xee,
0x6e, 0x7d, 0x84, 0x4e, 0x2e, 0xb3, 0x7f, 0x6d, 0x58, 0x0e, 0xff, 0xb5, 0x61, 0xf9, 0xba, 0x65,
0xe3, 0xbb, 0x3c, 0xfb, 0xf5, 0x3f, 0x0b, 0x43, 0xee, 0x5e, 0x46, 0xe7, 0xe3, 0x1a, 0xff, 0xe3,
0x88, 0x77, 0x1f, 0x05, 0x77, 0xb7, 0x3e, 0x5a, 0x35, 0x3e, 0x7f, 0xab, 0x00, 0xb9, 0x95, 0xe5,
0x57, 0x97, 0x5f, 0x81, 0x9a, 0x15, 0xa1, 0xb7, 0xbd, 0x6e, 0x6b, 0xb5, 0xcc, 0x88, 0x36, 0x48,
0x3d, 0x1b, 0xca, 0x2f, 0x5f, 0x6c, 0x5b, 0xc1, 0x4e, 0x6f, 0x8b, 0x0c, 0xc1, 0x05, 0x86, 0xf6,
0xb2, 0xe5, 0xf2, 0x5f, 0x17, 0x2c, 0x27, 0xc0, 0x9e, 0x63, 0xd8, 0xec, 0x0f, 0x25, 0x38, 0xb4,
0xbb, 0xf5, 0x87, 0x8a, 0xb2, 0x95, 0xa7, 0xa0, 0x8b, 0x3f, 0x0f, 0x00, 0x00, 0xff, 0xff, 0xe9,
0xd6, 0x0b, 0x41, 0xb2, 0x62, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
@ -53,8 +54,8 @@ const (
var checkPendingTasksInterval = 60 * 1000
// ExpireOldTasksInterval is the default interval to loop through all in memory tasks and expire old ones.
// default 10*60*1000 milliseconds (10 minutes)
var expireOldTasksInterval = 10 * 60 * 1000
// default 2*60*1000 milliseconds (2 minutes)
var expireOldTasksInterval = 2 * 60 * 1000
// import task state
type importTaskState struct {
@ -134,7 +135,7 @@ func (m *importManager) sendOutTasksLoop(wg *sync.WaitGroup) {
}
// expireOldTasksLoop starts a loop that checks and expires old tasks every `ImportTaskExpiration` seconds.
func (m *importManager) expireOldTasksLoop(wg *sync.WaitGroup) {
func (m *importManager) expireOldTasksLoop(wg *sync.WaitGroup, releaseLockFunc func(context.Context, []int64) error) {
defer wg.Done()
ticker := time.NewTicker(time.Duration(expireOldTasksInterval) * time.Millisecond)
defer ticker.Stop()
@ -146,7 +147,7 @@ func (m *importManager) expireOldTasksLoop(wg *sync.WaitGroup) {
case <-ticker.C:
log.Debug("(in loop) starting expiring old tasks...",
zap.Duration("cleaning up interval", time.Duration(expireOldTasksInterval)*time.Millisecond))
m.expireOldTasks()
m.expireOldTasks(releaseLockFunc)
}
}
}
@ -321,8 +322,8 @@ func (m *importManager) importJob(ctx context.Context, req *milvuspb.ImportReque
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
HeuristicDataQueryable: false,
HeuristicDataIndexed: false,
DataQueryable: false,
DataIndexed: false,
}
resp.Tasks = append(resp.Tasks, newTask.GetId())
taskList[i] = newTask.GetId()
@ -351,8 +352,8 @@ func (m *importManager) importJob(ctx context.Context, req *milvuspb.ImportReque
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
HeuristicDataQueryable: false,
HeuristicDataIndexed: false,
DataQueryable: false,
DataIndexed: false,
}
resp.Tasks = append(resp.Tasks, newTask.GetId())
log.Info("new task created as pending task", zap.Int64("task ID", newTask.GetId()))
@ -379,7 +380,7 @@ func (m *importManager) setTaskDataQueryable(taskID int64) {
m.workingLock.Lock()
defer m.workingLock.Unlock()
if v, ok := m.workingTasks[taskID]; ok {
v.HeuristicDataQueryable = true
v.DataQueryable = true
} else {
log.Error("task ID not found", zap.Int64("task ID", taskID))
}
@ -390,7 +391,7 @@ func (m *importManager) setTaskDataIndexed(taskID int64) {
m.workingLock.Lock()
defer m.workingLock.Unlock()
if v, ok := m.workingTasks[taskID]; ok {
v.HeuristicDataIndexed = true
v.DataIndexed = true
} else {
log.Error("task ID not found", zap.Int64("task ID", taskID))
}
@ -447,6 +448,29 @@ func (m *importManager) getCollectionPartitionName(task *datapb.ImportTaskInfo,
}
}
// appendTaskSegments updates the task's segment lists by adding `segIDs` to it.
func (m *importManager) appendTaskSegments(taskID int64, segIDs []int64) error {
log.Debug("import manager appending task segments",
zap.Int64("task ID", taskID),
zap.Int64s("segment ID", segIDs))
var v *datapb.ImportTaskInfo
m.workingLock.Lock()
ok := false
if v, ok = m.workingTasks[taskID]; ok {
v.State.Segments = append(v.GetState().GetSegments(), segIDs...)
// Update task in task store.
m.updateImportTaskStore(v)
}
m.workingLock.Unlock()
if !ok {
log.Debug("import manager appending task segments failed", zap.Int64("task ID", taskID))
return errors.New("failed to update import task, ID not found: " + strconv.FormatInt(taskID, 10))
}
return nil
}
// getTaskState looks for task with the given ID and returns its import state.
func (m *importManager) getTaskState(tID int64) *milvuspb.GetImportStateResponse {
resp := &milvuspb.GetImportStateResponse{
@ -470,8 +494,8 @@ func (m *importManager) getTaskState(tID int64) *milvuspb.GetImportStateResponse
resp.Id = tID
resp.State = commonpb.ImportState_ImportPending
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{Key: Files, Value: strings.Join(t.GetFiles(), ",")})
resp.HeuristicDataQueryable = t.GetHeuristicDataQueryable()
resp.HeuristicDataIndexed = t.GetHeuristicDataIndexed()
resp.DataQueryable = t.GetDataQueryable()
resp.DataIndexed = t.GetDataIndexed()
m.getCollectionPartitionName(t, resp)
found = true
break
@ -499,8 +523,8 @@ func (m *importManager) getTaskState(tID int64) *milvuspb.GetImportStateResponse
Key: FailedReason,
Value: v.GetState().GetErrorMessage(),
})
resp.HeuristicDataQueryable = v.GetHeuristicDataQueryable()
resp.HeuristicDataIndexed = v.GetHeuristicDataIndexed()
resp.DataQueryable = v.GetDataQueryable()
resp.DataIndexed = v.GetDataIndexed()
m.getCollectionPartitionName(v, resp)
}
}()
@ -583,17 +607,27 @@ func (m *importManager) updateImportTaskStore(ti *datapb.ImportTaskInfo) error {
}
// expireOldTasks marks expires tasks as failed.
func (m *importManager) expireOldTasks() {
func (m *importManager) expireOldTasks(releaseLockFunc func(context.Context, []int64) error) {
// Expire old pending tasks, if any.
func() {
m.pendingLock.Lock()
defer m.pendingLock.Unlock()
for _, t := range m.pendingTasks {
if taskExpired(t) {
// Mark this expired task as failed.
log.Info("a pending task has expired", zap.Int64("task ID", t.GetId()))
t.State.StateCode = commonpb.ImportState_ImportFailed
t.State.ErrorMessage = taskExpiredMsgPrefix +
(time.Duration(Params.RootCoordCfg.ImportTaskExpiration*1000) * time.Millisecond).String()
log.Info("releasing seg ref locks on expired import task",
zap.Int64s("segment IDs", t.GetState().GetSegments()))
err := retry.Do(m.ctx, func() error {
return releaseLockFunc(m.ctx, t.GetState().GetSegments())
}, retry.Attempts(100))
if err != nil {
log.Error("failed to release lock, about to panic!")
panic(err)
}
m.updateImportTaskStore(t)
}
}
@ -603,12 +637,21 @@ func (m *importManager) expireOldTasks() {
m.workingLock.Lock()
defer m.workingLock.Unlock()
for _, v := range m.workingTasks {
// Mark this expired task as failed.
if taskExpired(v) {
// Mark this expired task as failed.
log.Info("a working task has expired", zap.Int64("task ID", v.GetId()))
v.State.StateCode = commonpb.ImportState_ImportFailed
v.State.ErrorMessage = taskExpiredMsgPrefix +
(time.Duration(Params.RootCoordCfg.ImportTaskExpiration*1000) * time.Millisecond).String()
log.Info("releasing seg ref locks on expired import task",
zap.Int64s("segment IDs", v.GetState().GetSegments()))
err := retry.Do(m.ctx, func() error {
return releaseLockFunc(m.ctx, v.GetState().GetSegments())
}, retry.Attempts(100))
if err != nil {
log.Error("failed to release lock, about to panic!")
panic(err)
}
m.updateImportTaskStore(v)
}
}
@ -632,11 +675,11 @@ func (m *importManager) listAllTasks() []*milvuspb.GetImportStateResponse {
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Infos: make([]*commonpb.KeyValuePair, 0),
Id: t.GetId(),
State: commonpb.ImportState_ImportPending,
HeuristicDataQueryable: t.GetHeuristicDataQueryable(),
HeuristicDataIndexed: t.GetHeuristicDataIndexed(),
Infos: make([]*commonpb.KeyValuePair, 0),
Id: t.GetId(),
State: commonpb.ImportState_ImportPending,
DataQueryable: t.GetDataQueryable(),
DataIndexed: t.GetDataIndexed(),
}
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{Key: Files, Value: strings.Join(t.GetFiles(), ",")})
m.getCollectionPartitionName(t, resp)
@ -653,13 +696,13 @@ func (m *importManager) listAllTasks() []*milvuspb.GetImportStateResponse {
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Infos: make([]*commonpb.KeyValuePair, 0),
Id: v.GetId(),
State: v.GetState().GetStateCode(),
RowCount: v.GetState().GetRowCount(),
IdList: v.GetState().GetRowIds(),
HeuristicDataQueryable: v.GetHeuristicDataQueryable(),
HeuristicDataIndexed: v.GetHeuristicDataIndexed(),
Infos: make([]*commonpb.KeyValuePair, 0),
Id: v.GetId(),
State: v.GetState().GetStateCode(),
RowCount: v.GetState().GetRowCount(),
IdList: v.GetState().GetRowIds(),
DataQueryable: v.GetDataQueryable(),
DataIndexed: v.GetDataIndexed(),
}
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{Key: Files, Value: strings.Join(v.GetFiles(), ",")})
resp.Infos = append(resp.Infos, &commonpb.KeyValuePair{
@ -681,7 +724,10 @@ func BuildImportTaskKey(taskID int64) string {
return fmt.Sprintf("%s%s%d", Params.RootCoordCfg.ImportTaskSubPath, delimiter, taskID)
}
// taskExpired returns true if the task has already expired.
// taskExpired returns true if the task is considered expired.
func taskExpired(ti *datapb.ImportTaskInfo) bool {
return Params.RootCoordCfg.ImportTaskExpiration <= float64(time.Now().Unix()-ti.GetCreateTs())
return ti.GetState().GetStateCode() != commonpb.ImportState_ImportFailed &&
ti.GetState().GetStateCode() != commonpb.ImportState_ImportPersisted &&
ti.GetState().GetStateCode() != commonpb.ImportState_ImportCompleted &&
Params.RootCoordCfg.ImportTaskExpiration <= float64(time.Now().Unix()-ti.GetCreateTs())
}

View File

@ -93,7 +93,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
mgr.init(ctx)
var wgLoop sync.WaitGroup
wgLoop.Add(2)
mgr.expireOldTasksLoop(&wgLoop)
mgr.expireOldTasksLoop(&wgLoop, func(ctx context.Context, int64s []int64) error {
return nil
})
mgr.sendOutTasksLoop(&wgLoop)
wgLoop.Wait()
})
@ -108,7 +110,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
mgr.init(context.TODO())
var wgLoop sync.WaitGroup
wgLoop.Add(2)
mgr.expireOldTasksLoop(&wgLoop)
mgr.expireOldTasksLoop(&wgLoop, func(ctx context.Context, int64s []int64) error {
return nil
})
mgr.sendOutTasksLoop(&wgLoop)
wgLoop.Wait()
})
@ -130,7 +134,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
mgr.loadFromTaskStore()
var wgLoop sync.WaitGroup
wgLoop.Add(2)
mgr.expireOldTasksLoop(&wgLoop)
mgr.expireOldTasksLoop(&wgLoop, func(ctx context.Context, int64s []int64) error {
return nil
})
mgr.sendOutTasksLoop(&wgLoop)
wgLoop.Wait()
})
@ -145,7 +151,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
mgr.init(ctx)
var wgLoop sync.WaitGroup
wgLoop.Add(2)
mgr.expireOldTasksLoop(&wgLoop)
mgr.expireOldTasksLoop(&wgLoop, func(ctx context.Context, int64s []int64) error {
return nil
})
mgr.sendOutTasksLoop(&wgLoop)
time.Sleep(500 * time.Millisecond)
wgLoop.Wait()

View File

@ -154,6 +154,12 @@ type Core struct {
// Seals segments in collection cID, so they can get flushed later.
CallFlushOnCollection func(ctx context.Context, cID int64, segIDs []int64) error
// CallAddSegRefLock triggers AcquireSegmentLock method on DataCoord.
CallAddSegRefLock func(ctx context.Context, segIDs []int64) (retErr error)
// CallReleaseSegRefLock triggers ReleaseSegmentLock method on DataCoord.
CallReleaseSegRefLock func(ctx context.Context, segIDs []int64) (retErr error)
//Proxy manager
proxyManager *proxyManager
@ -290,6 +296,12 @@ func (c *Core) checkInit() error {
if c.CallImportService == nil {
return fmt.Errorf("callImportService is nil")
}
if c.CallAddSegRefLock == nil {
return fmt.Errorf("callAddSegRefLock is nil")
}
if c.CallReleaseSegRefLock == nil {
return fmt.Errorf("callReleaseSegRefLock is nil")
}
return nil
}
@ -685,10 +697,56 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
return nil
}
c.CallAddSegRefLock = func(ctx context.Context, segIDs []int64) (retErr error) {
defer func() {
if err := recover(); err != nil {
retErr = fmt.Errorf("add seg ref lock panic, msg = %v", err)
}
}()
<-initCh
log.Info("acquiring seg lock",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", c.session.ServerID))
resp, _ := s.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{
SegmentIDs: segIDs,
NodeID: c.session.ServerID,
})
if resp.GetErrorCode() != commonpb.ErrorCode_Success {
return fmt.Errorf("failed to acquire segment lock %s", resp.GetReason())
}
log.Info("acquire seg lock succeed",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", c.session.ServerID))
return nil
}
c.CallReleaseSegRefLock = func(ctx context.Context, segIDs []int64) (retErr error) {
defer func() {
if err := recover(); err != nil {
retErr = fmt.Errorf("release seg ref lock panic, msg = %v", err)
}
}()
<-initCh
log.Info("releasing seg lock",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", c.session.ServerID))
resp, _ := s.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{
SegmentIDs: segIDs,
NodeID: c.session.ServerID,
})
if resp.GetErrorCode() != commonpb.ErrorCode_Success {
return fmt.Errorf("failed to release segment lock %s", resp.GetReason())
}
log.Info("release seg lock succeed",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", c.session.ServerID))
return nil
}
return nil
}
// SetIndexCoord set indexcoord
// SetIndexCoord sets IndexCoord.
func (c *Core) SetIndexCoord(s types.IndexCoord) error {
initCh := make(chan struct{})
go func() {
@ -1266,7 +1324,9 @@ func (c *Core) Start() error {
return err
}
log.Debug(typeutil.RootCoordRole, zap.Int64("node id", c.session.ServerID))
log.Debug("starting service",
zap.String("service role", typeutil.RootCoordRole),
zap.Int64("node id", c.session.ServerID))
c.startOnce.Do(func() {
if err := c.proxyManager.WatchProxy(); err != nil {
@ -1283,7 +1343,7 @@ func (c *Core) Start() error {
go c.tsLoop()
go c.chanTimeTick.startWatch(&c.wg)
go c.checkFlushedSegmentsLoop()
go c.importManager.expireOldTasksLoop(&c.wg)
go c.importManager.expireOldTasksLoop(&c.wg, c.CallReleaseSegRefLock)
go c.importManager.sendOutTasksLoop(&c.wg)
Params.RootCoordCfg.CreatedTime = time.Now()
Params.RootCoordCfg.UpdatedTime = time.Now()
@ -2343,6 +2403,25 @@ func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (
if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
}
// Special case for ImportState_ImportAllocSegment state, where we shall only add segment ref lock and do no other
// operations.
// TODO: This is inelegant and must get re-structured.
if ir.GetState() == commonpb.ImportState_ImportAllocSegment {
// Lock the segments, so we don't lose track of them when compaction happens.
// Note that these locks will be unlocked in c.postImportPersistLoop() -> checkSegmentLoadedLoop().
if err := c.CallAddSegRefLock(ctx, ir.GetSegments()); err != nil {
log.Error("failed to acquire segment ref lock", zap.Error(err))
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: fmt.Sprintf("failed to acquire segment ref lock %s", err.Error()),
}, nil
}
// Update task store with new segments.
c.importManager.appendTaskSegments(ir.GetTaskId(), ir.GetSegments())
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
}
// Upon receiving ReportImport request, update the related task's state in task store.
ti, err := c.importManager.updateTaskState(ir)
if err != nil {
@ -2368,6 +2447,15 @@ func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (
// If task failed, send task to idle datanode
if ir.GetState() == commonpb.ImportState_ImportFailed {
// Release segments when task fails.
log.Info("task failed, release segment ref locks")
err := retry.Do(ctx, func() error {
return c.CallReleaseSegRefLock(ctx, ir.GetSegments())
}, retry.Attempts(100))
if err != nil {
log.Error("failed to release lock, about to panic!")
panic(err)
}
resendTaskFunc()
}
@ -2408,10 +2496,12 @@ func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (
}, nil
}
// CountCompleteIndex checks indexing status of the given segments, and returns the # of segments that has complete index.
// CountCompleteIndex checks indexing status of the given segments.
// It returns an error if error occurs. It also returns a boolean indicating whether indexing is done (or if no index
// is needed).
func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, collectionID UniqueID,
allSegmentIDs []UniqueID) (int, error) {
// Note: Index name is always Params.CommonCfg.DefaultIndexName in current Milvus design as of today.
allSegmentIDs []UniqueID) (bool, error) {
// Note: Index name is always Params.CommonCfg.DefaultIndexName in current Milvus designs as of today.
indexName := Params.CommonCfg.DefaultIndexName
// Retrieve index status and detailed index information.
@ -2424,30 +2514,38 @@ func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, co
}
indexDescriptionResp, err := c.DescribeIndex(ctx, describeIndexReq)
if err != nil {
return 0, err
return false, err
}
log.Debug("got index description", zap.String("index_description", indexDescriptionResp.String()))
if len(indexDescriptionResp.GetIndexDescriptions()) == 0 {
log.Info("no index needed for collection, consider indexing done",
zap.Int64("collection ID", collectionID))
return true, nil
}
log.Debug("got index description",
zap.Any("index description", indexDescriptionResp))
// Check if the target index name exists.
matchIndexID := int64(-1)
foundIndexID := false
for _, desc := range indexDescriptionResp.IndexDescriptions {
if desc.IndexName == indexName {
matchIndexID = desc.IndexID
for _, desc := range indexDescriptionResp.GetIndexDescriptions() {
if desc.GetIndexName() == indexName {
matchIndexID = desc.GetIndexID()
foundIndexID = true
break
}
}
if !foundIndexID {
return 0, fmt.Errorf("no index is created")
return false, fmt.Errorf("no index is created")
}
log.Debug("found match index ID", zap.Int64("match index ID", matchIndexID))
log.Debug("found match index ID",
zap.Int64("match index ID", matchIndexID))
getIndexStatesRequest := &indexpb.GetIndexStatesRequest{
IndexBuildIDs: make([]UniqueID, 0),
}
// Fetch index build IDs from segments.
var seg2Check []UniqueID
for _, segmentID := range allSegmentIDs {
describeSegmentRequest := &milvuspb.DescribeSegmentRequest{
Base: &commonpb.MsgBase{
@ -2456,31 +2554,35 @@ func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, co
CollectionID: collectionID,
SegmentID: segmentID,
}
segmentDesc, err := c.DescribeSegment(ctx, describeSegmentRequest)
if err != nil {
log.Error("Failed to describe segment",
segmentDesc, _ := c.DescribeSegment(ctx, describeSegmentRequest)
if segmentDesc.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
// Describe failed, since the segment could get compacted, simply log and ignore the error.
log.Error("failed to describe segment",
zap.Int64("collection ID", collectionID),
zap.Int64("segment ID", segmentID))
return 0, err
zap.Int64("segment ID", segmentID),
zap.String("error", segmentDesc.GetStatus().GetReason()))
}
if segmentDesc.IndexID == matchIndexID {
if segmentDesc.EnableIndex {
getIndexStatesRequest.IndexBuildIDs = append(getIndexStatesRequest.IndexBuildIDs, segmentDesc.BuildID)
if segmentDesc.GetIndexID() == matchIndexID {
if segmentDesc.GetEnableIndex() {
seg2Check = append(seg2Check, segmentID)
getIndexStatesRequest.IndexBuildIDs = append(getIndexStatesRequest.GetIndexBuildIDs(), segmentDesc.GetBuildID())
}
}
}
log.Debug("proxy GetIndexState", zap.Int("# of IndexBuildIDs", len(getIndexStatesRequest.IndexBuildIDs)), zap.Error(err))
if len(getIndexStatesRequest.IndexBuildIDs) == 0 {
log.Info("empty index build IDs returned",
if len(getIndexStatesRequest.GetIndexBuildIDs()) == 0 {
log.Info("none index build IDs returned, perhaps no index is needed",
zap.String("collection name", collectionName),
zap.Int64("collection ID", collectionID))
return 0, nil
return true, nil
}
log.Debug("working on GetIndexState",
zap.Int("# of IndexBuildIDs", len(getIndexStatesRequest.GetIndexBuildIDs())))
states, err := c.CallGetIndexStatesService(ctx, getIndexStatesRequest.GetIndexBuildIDs())
if err != nil {
log.Error("failed to get index state in checkSegmentIndexStates", zap.Error(err))
return 0, err
return false, err
}
// Count the # of segments with finished index.
@ -2491,25 +2593,30 @@ func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, co
}
}
log.Info("segment indexing state checked",
zap.Int("# of checked segment", len(states)),
zap.Int64s("segments checked", seg2Check),
zap.Int("# of checked segment", len(seg2Check)),
zap.Int("# of segments with complete index", ct),
zap.String("collection name", collectionName),
zap.Int64("collection ID", collectionID),
)
return ct, nil
return len(seg2Check) == ct, nil
}
func (c *Core) postImportPersistLoop(ctx context.Context, taskID int64, colID int64, colName string, segIDs []UniqueID) {
// Loop and check if segments are loaded in queryNodes.
c.wg.Add(1)
c.checkSegmentLoadedLoop(ctx, taskID, colID, segIDs)
go c.checkSegmentLoadedLoop(ctx, taskID, colID, segIDs)
// Check if collection has any indexed fields. If so, start a loop to check segments' index states.
if colMeta, err := c.MetaTable.GetCollectionByID(colID, 0); err != nil {
log.Error("failed to find meta for collection",
zap.Int64("collection ID", colID))
} else if len(colMeta.GetFieldIndexes()) != 0 {
zap.Int64("collection ID", colID),
zap.Error(err))
} else if len(colMeta.GetFieldIndexes()) == 0 {
log.Info("no index field found for collection", zap.Int64("collection ID", colID))
} else {
log.Info("start checking index state", zap.Int64("collection ID", colID))
c.wg.Add(1)
c.checkCompleteIndexLoop(ctx, taskID, colID, colName, segIDs)
go c.checkCompleteIndexLoop(ctx, taskID, colID, colName, segIDs)
}
}
@ -2520,6 +2627,16 @@ func (c *Core) checkSegmentLoadedLoop(ctx context.Context, taskID int64, colID i
defer ticker.Stop()
expireTicker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportSegmentStateWaitLimit*1000) * time.Millisecond)
defer expireTicker.Stop()
defer func() {
log.Info("we are done checking segment loading state, release segment ref locks")
err := retry.Do(ctx, func() error {
return c.CallReleaseSegRefLock(ctx, segIDs)
}, retry.Attempts(100))
if err != nil {
log.Error("failed to release lock, about to panic!")
panic(err)
}
}()
for {
select {
case <-c.ctx.Done():
@ -2527,12 +2644,17 @@ func (c *Core) checkSegmentLoadedLoop(ctx context.Context, taskID int64, colID i
return
case <-ticker.C:
resp, err := c.CallGetSegmentInfoService(ctx, colID, segIDs)
log.Debug("(in check segment loaded loop)",
zap.Int64("task ID", taskID),
zap.Int64("collection ID", colID),
zap.Int64s("segment IDs expected", segIDs),
zap.Int("# of segments found", len(resp.GetInfos())))
if err != nil {
log.Warn("(in check segment loaded loop) failed to call get segment info on queryCoord",
zap.Int64("task ID", taskID),
zap.Int64("collection ID", colID),
zap.Int64s("segment IDs", segIDs))
} else if heuristicSegmentsReady(len(resp.GetInfos()), len(segIDs)) {
} else if len(resp.GetInfos()) == len(segIDs) {
// Check if all segment info are loaded in queryNodes.
log.Info("(in check segment loaded loop) all import data segments loaded in queryNodes",
zap.Int64("task ID", taskID),
@ -2564,11 +2686,14 @@ func (c *Core) checkCompleteIndexLoop(ctx context.Context, taskID int64, colID i
log.Info("(in check complete index loop) context done, exiting checkCompleteIndexLoop")
return
case <-ticker.C:
if ct, err := c.CountCompleteIndex(ctx, colName, colID, segIDs); err == nil && heuristicSegmentsReady(ct, len(segIDs)) {
log.Info("(in check complete index loop) all segment indices are ready!",
if done, err := c.CountCompleteIndex(ctx, colName, colID, segIDs); err == nil && done {
log.Info("(in check complete index loop) indices are built or no index needed",
zap.Int64("task ID", taskID))
c.importManager.setTaskDataIndexed(taskID)
return
} else if err != nil {
log.Error("(in check complete index loop) an error occurs",
zap.Error(err))
}
case <-expireTicker.C:
log.Warn("(in check complete index loop) indexing is taken too long",
@ -2769,10 +2894,3 @@ func (c *Core) ListCredUsers(ctx context.Context, in *milvuspb.ListCredUsersRequ
Usernames: credInfo.Usernames,
}, nil
}
// heuristicSegmentsReady checks and returns if segments are ready based on count in a heuristic way.
// We do this to avoid accidentally compacted segments.
// This is just a temporary solution.
func heuristicSegmentsReady(currCount int, expectedCount int) bool {
return currCount >= expectedCount-2 || float64(currCount)/float64(expectedCount) >= 0.8
}

View File

@ -294,6 +294,20 @@ func (d *dataMock) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb
}, nil
}
func (d *dataMock) AcquireSegmentLock(context.Context, *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
func (d *dataMock) ReleaseSegmentLock(context.Context, *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
type queryMock struct {
types.QueryCoord
collID []typeutil.UniqueID
@ -870,6 +884,7 @@ func TestRootCoord_Base(t *testing.T) {
return localTSO, nil
}
expireOldTasksInterval = 500
err = core.Start()
assert.NoError(t, err)
@ -1366,14 +1381,14 @@ func TestRootCoord_Base(t *testing.T) {
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.NoError(t, err)
// Normal case.
count, err := core.CountCompleteIndex(context.WithValue(ctx, ctxKey{}, ""),
done, err := core.CountCompleteIndex(context.WithValue(ctx, ctxKey{}, ""),
collName, coll.ID, []UniqueID{1000, 1001, 1002})
assert.NoError(t, err)
assert.Equal(t, 3, count)
assert.Equal(t, true, done)
// Case with an empty result.
count, err = core.CountCompleteIndex(ctx, collName, coll.ID, []UniqueID{})
done, err = core.CountCompleteIndex(ctx, collName, coll.ID, []UniqueID{})
assert.NoError(t, err)
assert.Equal(t, 0, count)
assert.Equal(t, true, done)
// Case where GetIndexStates failed with error.
_, err = core.CountCompleteIndex(context.WithValue(ctx, ctxKey{}, returnError),
collName, coll.ID, []UniqueID{1000, 1001, 1002})
@ -1382,6 +1397,10 @@ func TestRootCoord_Base(t *testing.T) {
_, err = core.CountCompleteIndex(context.WithValue(ctx, ctxKey{}, returnUnsuccessfulStatus),
collName, coll.ID, []UniqueID{1000, 1001, 1002})
assert.Error(t, err)
// Case where describing segment fails, which is not considered as an error.
_, err = core.CountCompleteIndex(context.WithValue(ctx, ctxKey{}, ""),
collName, coll.ID, []UniqueID{9000, 9001, 9002})
assert.NoError(t, err)
})
wg.Add(1)
@ -1570,6 +1589,21 @@ func TestRootCoord_Base(t *testing.T) {
time.Sleep(500 * time.Millisecond)
})
wg.Add(1)
t.Run("report import with alloc seg state", func(t *testing.T) {
defer wg.Done()
req := &rootcoordpb.ImportResult{
TaskId: 1,
RowCount: 100,
Segments: []int64{1000, 1001, 1002},
State: commonpb.ImportState_ImportAllocSegment,
}
resp, err := core.ReportImport(context.WithValue(ctx, ctxKey{}, ""), req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
time.Sleep(500 * time.Millisecond)
})
wg.Add(1)
t.Run("report import wait for index", func(t *testing.T) {
defer wg.Done()
@ -3038,6 +3072,18 @@ func TestCheckInit(t *testing.T) {
}
}
err = c.checkInit()
assert.Error(t, err)
c.CallAddSegRefLock = func(context.Context, []int64) error {
return nil
}
err = c.checkInit()
assert.Error(t, err)
c.CallReleaseSegRefLock = func(context.Context, []int64) error {
return nil
}
err = c.checkInit()
assert.NoError(t, err)
err = c.Stop()

View File

@ -411,12 +411,12 @@ func (p *rootCoordConfig) init(base *BaseTable) {
p.DmlChannelNum = p.Base.ParseInt64WithDefault("rootCoord.dmlChannelNum", 256)
p.MaxPartitionNum = p.Base.ParseInt64WithDefault("rootCoord.maxPartitionNum", 4096)
p.MinSegmentSizeToEnableIndex = p.Base.ParseInt64WithDefault("rootCoord.minSegmentSizeToEnableIndex", 1024)
p.ImportTaskExpiration = p.Base.ParseFloatWithDefault("rootCoord.importTaskExpiration", 3600)
p.ImportTaskRetention = p.Base.ParseFloatWithDefault("rootCoord.importTaskRetention", 3600*24)
p.ImportTaskExpiration = p.Base.ParseFloatWithDefault("rootCoord.importTaskExpiration", 15*60)
p.ImportTaskRetention = p.Base.ParseFloatWithDefault("rootCoord.importTaskRetention", 24*60*60)
p.ImportSegmentStateCheckInterval = p.Base.ParseFloatWithDefault("rootCoord.importSegmentStateCheckInterval", 10)
p.ImportSegmentStateWaitLimit = p.Base.ParseFloatWithDefault("rootCoord.importSegmentStateWaitLimit", 60)
p.ImportIndexCheckInterval = p.Base.ParseFloatWithDefault("rootCoord.importIndexCheckInterval", 60*5)
p.ImportIndexWaitLimit = p.Base.ParseFloatWithDefault("rootCoord.importIndexWaitLimit", 60*20)
p.ImportIndexCheckInterval = p.Base.ParseFloatWithDefault("rootCoord.importIndexCheckInterval", 10)
p.ImportIndexWaitLimit = p.Base.ParseFloatWithDefault("rootCoord.importIndexWaitLimit", 10*60)
p.ImportTaskSubPath = "importtask"
}