diff --git a/Makefile b/Makefile index a2ea37880d..ed788b0548 100644 --- a/Makefile +++ b/Makefile @@ -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. diff --git a/configs/milvus.yaml b/configs/milvus.yaml index cf606c7c86..08c2557d31 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -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: diff --git a/internal/core/src/pb/common.pb.cc b/internal/core/src/pb/common.pb.cc index 84a12b2a6e..e53475f54d 100644 --- a/internal/core/src/pb/common.pb.cc +++ b/internal/core/src/pb/common.pb.cc @@ -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; diff --git a/internal/core/src/pb/common.pb.h b/internal/core/src/pb/common.pb.h index 8800a335eb..160a703bf8 100644 --- a/internal/core/src/pb/common.pb.h +++ b/internal/core/src/pb/common.pb.h @@ -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(); diff --git a/internal/datacoord/segment_reference_manager.go b/internal/datacoord/segment_reference_manager.go index cbe6de1566..b124bf1c0c 100644 --- a/internal/datacoord/segment_reference_manager.go +++ b/internal/datacoord/segment_reference_manager.go @@ -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) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 4c69ab21ef..a7b1c5c100 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -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) } } } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 74826de0ba..8df1926f87 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -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) diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 0153ebc412..9d0ed50d86 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -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{ diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index ead8849988..73249fc8dc 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -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, diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index 0ed0094747..66f5a2da58 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -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) diff --git a/internal/proto/common.proto b/internal/proto/common.proto index b2352d1568..bf594f69b1 100644 --- a/internal/proto/common.proto +++ b/internal/proto/common.proto @@ -269,6 +269,7 @@ enum ImportState { ImportParsed = 4; ImportPersisted = 5; ImportCompleted = 6; + ImportAllocSegment = 10; } enum ResourceType { diff --git a/internal/proto/commonpb/common.pb.go b/internal/proto/commonpb/common.pb.go index 748714befd..6fd92d6ec5 100644 --- a/internal/proto/commonpb/common.pb.go +++ b/internal/proto/commonpb/common.pb.go @@ -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, } diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index a08f2e8e60..0a0513a3af 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -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 { diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index b746e8812f..9a583913c5 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -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. diff --git a/internal/proto/milvus.proto b/internal/proto/milvus.proto index 846aa9bcb4..9e2b954cb3 100644 --- a/internal/proto/milvus.proto +++ b/internal/proto/milvus.proto @@ -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 { diff --git a/internal/proto/milvuspb/milvus.pb.go b/internal/proto/milvuspb/milvus.pb.go index 10d6f7b93d..88473801c0 100644 --- a/internal/proto/milvuspb/milvus.pb.go +++ b/internal/proto/milvuspb/milvus.pb.go @@ -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. diff --git a/internal/rootcoord/import_manager.go b/internal/rootcoord/import_manager.go index 2e31dc9fb6..73a0860099 100644 --- a/internal/rootcoord/import_manager.go +++ b/internal/rootcoord/import_manager.go @@ -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()) } diff --git a/internal/rootcoord/import_manager_test.go b/internal/rootcoord/import_manager_test.go index 35a0caa4b4..7e08f68d50 100644 --- a/internal/rootcoord/import_manager_test.go +++ b/internal/rootcoord/import_manager_test.go @@ -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() diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 4438da0793..c5c6dc3831 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -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 -} diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 6d780d5911..7491f44f81 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -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() diff --git a/internal/util/paramtable/component_param.go b/internal/util/paramtable/component_param.go index 429a0478d0..abfec7db58 100644 --- a/internal/util/paramtable/component_param.go +++ b/internal/util/paramtable/component_param.go @@ -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" }