diff --git a/cmd/single/start.go b/cmd/single/start.go index a86fc3772b..d415f82111 100644 --- a/cmd/single/start.go +++ b/cmd/single/start.go @@ -4,9 +4,19 @@ import ( "context" "net/http" "os" + metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" ctrlWebhook "sigs.k8s.io/controller-runtime/pkg/webhook" + _ "github.com/golang/glog" + "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/spf13/cobra" + "golang.org/x/sync/errgroup" + _ "gorm.io/driver/postgres" // Required to import database driver. + "sigs.k8s.io/controller-runtime/pkg/cache" + "sigs.k8s.io/controller-runtime/pkg/manager" + "sigs.k8s.io/controller-runtime/pkg/metrics" + datacatalogConfig "github.com/flyteorg/flyte/datacatalog/pkg/config" datacatalogRepo "github.com/flyteorg/flyte/datacatalog/pkg/repositories" datacatalog "github.com/flyteorg/flyte/datacatalog/pkg/rpc/datacatalogservice" @@ -29,14 +39,6 @@ import ( "github.com/flyteorg/flyte/flytestdlib/promutils" "github.com/flyteorg/flyte/flytestdlib/promutils/labeled" "github.com/flyteorg/flyte/flytestdlib/storage" - _ "github.com/golang/glog" - "github.com/prometheus/client_golang/prometheus/promhttp" - "github.com/spf13/cobra" - "golang.org/x/sync/errgroup" - _ "gorm.io/driver/postgres" // Required to import database driver. - "sigs.k8s.io/controller-runtime/pkg/cache" - "sigs.k8s.io/controller-runtime/pkg/manager" - "sigs.k8s.io/controller-runtime/pkg/metrics" ) const defaultNamespace = "all" @@ -121,7 +123,7 @@ func startPropeller(ctx context.Context, cfg Propeller) error { SyncPeriod: &propellerCfg.DownstreamEval.Duration, DefaultNamespaces: namespaceConfigs, }, - NewCache: executors.NewCache, + NewCache: executors.NewCache, NewClient: executors.BuildNewClientFunc(propellerScope), Metrics: metricsserver.Options{ // Disable metrics serving diff --git a/flyteadmin/flyteadmin_config.yaml b/flyteadmin/flyteadmin_config.yaml index e3d19f7326..443814572b 100644 --- a/flyteadmin/flyteadmin_config.yaml +++ b/flyteadmin/flyteadmin_config.yaml @@ -212,4 +212,4 @@ qualityOfService: staging: MEDIUM # by default production has an UNDEFINED tier when it is omitted from the configuration namespace_mapping: - template: "{{ project }}-{{ domain }}" # Default namespace mapping template. \ No newline at end of file + template: "{{ project }}-{{ domain }}" # Default namespace mapping template. diff --git a/flyteadmin/pkg/manager/impl/node_execution_manager.go b/flyteadmin/pkg/manager/impl/node_execution_manager.go index afe17b6ac6..d618a1784f 100644 --- a/flyteadmin/pkg/manager/impl/node_execution_manager.go +++ b/flyteadmin/pkg/manager/impl/node_execution_manager.go @@ -297,6 +297,31 @@ func (m *NodeExecutionManager) CreateNodeEvent(ctx context.Context, request admi return &admin.NodeExecutionEventResponse{}, nil } +func (m *NodeExecutionManager) GetDynamicNodeWorkflow(ctx context.Context, request admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) { + if err := validation.ValidateNodeExecutionIdentifier(request.Id); err != nil { + logger.Debugf(ctx, "can't get node execution data with invalid identifier [%+v]: %v", request.Id, err) + } + + ctx = getNodeExecutionContext(ctx, request.Id) + nodeExecutionModel, err := util.GetNodeExecutionModel(ctx, m.db, request.Id) + if err != nil { + logger.Errorf(ctx, "failed to get node execution with id [%+v] with err %v", + request.Id, err) + return nil, err + } + + if nodeExecutionModel.DynamicWorkflowRemoteClosureReference == "" { + return &admin.DynamicNodeWorkflowResponse{}, errors.NewFlyteAdminErrorf(codes.NotFound, "node does not contain dynamic workflow") + } + + closure, err := m.fetchDynamicWorkflowClosure(ctx, nodeExecutionModel.DynamicWorkflowRemoteClosureReference) + if err != nil { + return nil, err + } + + return &admin.DynamicNodeWorkflowResponse{CompiledWorkflow: closure}, nil +} + // Handles making additional database calls, if necessary, to populate IsParent & IsDynamic data using the historical pattern of // preloading child node executions. Otherwise, simply calls transform on the input model. func (m *NodeExecutionManager) transformNodeExecutionModel(ctx context.Context, nodeExecutionModel models.NodeExecution, @@ -516,23 +541,15 @@ func (m *NodeExecutionManager) GetNodeExecutionData( } if len(nodeExecutionModel.DynamicWorkflowRemoteClosureReference) > 0 { - closure := &core.CompiledWorkflowClosure{} - err := m.storageClient.ReadProtobuf(ctx, storage.DataReference(nodeExecutionModel.DynamicWorkflowRemoteClosureReference), closure) + closure, err := m.fetchDynamicWorkflowClosure(ctx, nodeExecutionModel.DynamicWorkflowRemoteClosureReference) if err != nil { - return nil, errors.NewFlyteAdminErrorf(codes.Internal, - "Unable to read WorkflowClosure from location %s : %v", nodeExecutionModel.DynamicWorkflowRemoteClosureReference, err) + return nil, err } - if wf := closure.Primary; wf == nil { - return nil, errors.NewFlyteAdminErrorf(codes.Internal, "Empty primary workflow definition in loaded dynamic workflow model.") - } else if template := wf.Template; template == nil { - return nil, errors.NewFlyteAdminErrorf(codes.Internal, "Empty primary workflow template in loaded dynamic workflow model.") - } else { - response.DynamicWorkflow = &admin.DynamicWorkflowNodeMetadata{ - Id: closure.Primary.Template.Id, - CompiledWorkflow: closure, - DynamicJobSpecUri: nodeExecution.Closure.DynamicJobSpecUri, - } + response.DynamicWorkflow = &admin.DynamicWorkflowNodeMetadata{ + Id: closure.Primary.Template.Id, + CompiledWorkflow: closure, + DynamicJobSpecUri: nodeExecution.Closure.DynamicJobSpecUri, } } @@ -546,6 +563,21 @@ func (m *NodeExecutionManager) GetNodeExecutionData( return response, nil } +func (m *NodeExecutionManager) fetchDynamicWorkflowClosure(ctx context.Context, ref string) (*core.CompiledWorkflowClosure, error) { + closure := &core.CompiledWorkflowClosure{} + err := m.storageClient.ReadProtobuf(ctx, storage.DataReference(ref), closure) + if err != nil { + return nil, errors.NewFlyteAdminErrorf(codes.Internal, "Unable to read WorkflowClosure from location %s : %v", ref, err) + } + + if wf := closure.Primary; wf == nil { + return nil, errors.NewFlyteAdminErrorf(codes.Internal, "Empty primary workflow definition in loaded dynamic workflow model.") + } else if template := wf.Template; template == nil { + return nil, errors.NewFlyteAdminErrorf(codes.Internal, "Empty primary workflow template in loaded dynamic workflow model.") + } + return closure, nil +} + func NewNodeExecutionManager(db repoInterfaces.Repository, config runtimeInterfaces.Configuration, storagePrefix []string, storageClient *storage.DataStore, scope promutils.Scope, urlData dataInterfaces.RemoteURLInterface, eventPublisher notificationInterfaces.Publisher, cloudEventPublisher cloudeventInterfaces.Publisher, diff --git a/flyteadmin/pkg/manager/impl/node_execution_manager_test.go b/flyteadmin/pkg/manager/impl/node_execution_manager_test.go index 1f36d28afc..821a3c8ca1 100644 --- a/flyteadmin/pkg/manager/impl/node_execution_manager_test.go +++ b/flyteadmin/pkg/manager/impl/node_execution_manager_test.go @@ -10,7 +10,9 @@ import ( "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" eventWriterMocks "github.com/flyteorg/flyte/flyteadmin/pkg/async/events/mocks" "github.com/flyteorg/flyte/flyteadmin/pkg/common" @@ -1319,3 +1321,148 @@ func TestGetNodeExecutionData(t *testing.T) { }, }, dataResponse)) } + +func Test_GetDynamicNodeWorkflow_Success(t *testing.T) { + repo := repositoryMocks.NewMockRepository() + nodeExecID := core.NodeExecutionIdentifier{ + ExecutionId: &core.WorkflowExecutionIdentifier{ + Project: project, + Domain: domain, + Name: name, + }, + } + repo.NodeExecutionRepo().(*repositoryMocks.MockNodeExecutionRepo). + SetGetCallback(func(ctx context.Context, input interfaces.NodeExecutionResource) (models.NodeExecution, error) { + assert.Equal(t, nodeExecID, input.NodeExecutionIdentifier) + return models.NodeExecution{DynamicWorkflowRemoteClosureReference: remoteClosureIdentifier}, nil + }) + mockStorageClient := commonMocks.GetMockStorageClient() + expectedClosure := testutils.GetWorkflowClosure().CompiledWorkflow + mockStorageClient.ComposedProtobufStore.(*commonMocks.TestDataStore).ReadProtobufCb = func(ctx context.Context, reference storage.DataReference, msg proto.Message) error { + assert.Equal(t, remoteClosureIdentifier, reference.String()) + bytes, err := proto.Marshal(expectedClosure) + require.NoError(t, err) + return proto.Unmarshal(bytes, msg) + } + ctx := context.TODO() + nodeExecManager := NewNodeExecutionManager(repo, + getMockExecutionsConfigProvider(), + storagePrefix, + mockStorageClient, + mockScope.NewTestScope(), + mockNodeExecutionRemoteURL, + nil, nil, + &eventWriterMocks.NodeExecutionEventWriter{}) + expected := &admin.DynamicNodeWorkflowResponse{ + CompiledWorkflow: expectedClosure, + } + + resp, err := nodeExecManager.GetDynamicNodeWorkflow(ctx, admin.GetDynamicNodeWorkflowRequest{Id: &nodeExecID}) + + assert.NoError(t, err) + assert.True(t, proto.Equal(expected, resp)) +} + +func Test_GetDynamicNodeWorkflow_DBError(t *testing.T) { + repo := repositoryMocks.NewMockRepository() + nodeExecID := core.NodeExecutionIdentifier{ + ExecutionId: &core.WorkflowExecutionIdentifier{ + Project: project, + Domain: domain, + Name: name, + }, + } + expectedErr := errors.New("failure") + repo.NodeExecutionRepo().(*repositoryMocks.MockNodeExecutionRepo). + SetGetCallback(func(ctx context.Context, input interfaces.NodeExecutionResource) (models.NodeExecution, error) { + assert.Equal(t, nodeExecID, input.NodeExecutionIdentifier) + return models.NodeExecution{}, expectedErr + }) + mockStorageClient := commonMocks.GetMockStorageClient() + ctx := context.TODO() + nodeExecManager := NewNodeExecutionManager(repo, + getMockExecutionsConfigProvider(), + storagePrefix, + mockStorageClient, + mockScope.NewTestScope(), + mockNodeExecutionRemoteURL, + nil, nil, + &eventWriterMocks.NodeExecutionEventWriter{}) + + resp, err := nodeExecManager.GetDynamicNodeWorkflow(ctx, admin.GetDynamicNodeWorkflowRequest{Id: &nodeExecID}) + + assert.Equal(t, expectedErr, err) + assert.Empty(t, resp) +} + +func Test_GetDynamicNodeWorkflow_NoRemoteReference(t *testing.T) { + repo := repositoryMocks.NewMockRepository() + nodeExecID := core.NodeExecutionIdentifier{ + ExecutionId: &core.WorkflowExecutionIdentifier{ + Project: project, + Domain: domain, + Name: name, + }, + } + repo.NodeExecutionRepo().(*repositoryMocks.MockNodeExecutionRepo). + SetGetCallback(func(ctx context.Context, input interfaces.NodeExecutionResource) (models.NodeExecution, error) { + assert.Equal(t, nodeExecID, input.NodeExecutionIdentifier) + return models.NodeExecution{DynamicWorkflowRemoteClosureReference: ""}, nil + }) + mockStorageClient := commonMocks.GetMockStorageClient() + ctx := context.TODO() + nodeExecManager := NewNodeExecutionManager(repo, + getMockExecutionsConfigProvider(), + storagePrefix, + mockStorageClient, + mockScope.NewTestScope(), + mockNodeExecutionRemoteURL, + nil, nil, + &eventWriterMocks.NodeExecutionEventWriter{}) + + resp, err := nodeExecManager.GetDynamicNodeWorkflow(ctx, admin.GetDynamicNodeWorkflowRequest{Id: &nodeExecID}) + + st, ok := status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.NotFound, st.Code()) + assert.Equal(t, "node does not contain dynamic workflow", st.Message()) + assert.Empty(t, resp) +} + +func Test_GetDynamicNodeWorkflow_StorageError(t *testing.T) { + repo := repositoryMocks.NewMockRepository() + nodeExecID := core.NodeExecutionIdentifier{ + ExecutionId: &core.WorkflowExecutionIdentifier{ + Project: project, + Domain: domain, + Name: name, + }, + } + repo.NodeExecutionRepo().(*repositoryMocks.MockNodeExecutionRepo). + SetGetCallback(func(ctx context.Context, input interfaces.NodeExecutionResource) (models.NodeExecution, error) { + assert.Equal(t, nodeExecID, input.NodeExecutionIdentifier) + return models.NodeExecution{DynamicWorkflowRemoteClosureReference: remoteClosureIdentifier}, nil + }) + mockStorageClient := commonMocks.GetMockStorageClient() + mockStorageClient.ComposedProtobufStore.(*commonMocks.TestDataStore).ReadProtobufCb = func(ctx context.Context, reference storage.DataReference, msg proto.Message) error { + assert.Equal(t, remoteClosureIdentifier, reference.String()) + return errors.New("failure") + } + ctx := context.TODO() + nodeExecManager := NewNodeExecutionManager(repo, + getMockExecutionsConfigProvider(), + storagePrefix, + mockStorageClient, + mockScope.NewTestScope(), + mockNodeExecutionRemoteURL, + nil, nil, + &eventWriterMocks.NodeExecutionEventWriter{}) + + resp, err := nodeExecManager.GetDynamicNodeWorkflow(ctx, admin.GetDynamicNodeWorkflowRequest{Id: &nodeExecID}) + + st, ok := status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.Internal, st.Code()) + assert.Equal(t, "Unable to read WorkflowClosure from location s3://flyte/metadata/admin/remote closure id : failure", st.Message()) + assert.Empty(t, resp) +} diff --git a/flyteadmin/pkg/manager/interfaces/node_execution.go b/flyteadmin/pkg/manager/interfaces/node_execution.go index 3aa949ec85..97f2a1a166 100644 --- a/flyteadmin/pkg/manager/interfaces/node_execution.go +++ b/flyteadmin/pkg/manager/interfaces/node_execution.go @@ -15,4 +15,5 @@ type NodeExecutionInterface interface { ListNodeExecutionsForTask(ctx context.Context, request admin.NodeExecutionForTaskListRequest) (*admin.NodeExecutionList, error) GetNodeExecutionData( ctx context.Context, request admin.NodeExecutionGetDataRequest) (*admin.NodeExecutionGetDataResponse, error) + GetDynamicNodeWorkflow(ctx context.Context, request admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) } diff --git a/flyteadmin/pkg/manager/mocks/node_execution.go b/flyteadmin/pkg/manager/mocks/node_execution.go index 313566fc3d..83295e423a 100644 --- a/flyteadmin/pkg/manager/mocks/node_execution.go +++ b/flyteadmin/pkg/manager/mocks/node_execution.go @@ -84,3 +84,7 @@ func (m *MockNodeExecutionManager) GetNodeExecutionData( } return nil, nil } + +func (m *MockNodeExecutionManager) GetDynamicNodeWorkflow(ctx context.Context, request admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) { + return nil, nil +} diff --git a/flyteadmin/pkg/rpc/adminservice/metrics.go b/flyteadmin/pkg/rpc/adminservice/metrics.go index 3409dfd18d..6e5060cab5 100644 --- a/flyteadmin/pkg/rpc/adminservice/metrics.go +++ b/flyteadmin/pkg/rpc/adminservice/metrics.go @@ -46,12 +46,13 @@ type namedEntityEndpointMetrics struct { type nodeExecutionEndpointMetrics struct { scope promutils.Scope - createEvent util.RequestMetrics - get util.RequestMetrics - getData util.RequestMetrics - getMetrics util.RequestMetrics - list util.RequestMetrics - listChildren util.RequestMetrics + createEvent util.RequestMetrics + get util.RequestMetrics + getData util.RequestMetrics + getMetrics util.RequestMetrics + list util.RequestMetrics + listChildren util.RequestMetrics + getDynamicNodeWorkflow util.RequestMetrics } type projectEndpointMetrics struct { @@ -161,13 +162,14 @@ func InitMetrics(adminScope promutils.Scope) AdminMetrics { update: util.NewRequestMetrics(adminScope, "update_named_entity"), }, nodeExecutionEndpointMetrics: nodeExecutionEndpointMetrics{ - scope: adminScope, - createEvent: util.NewRequestMetrics(adminScope, "create_node_execution_event"), - get: util.NewRequestMetrics(adminScope, "get_node_execution"), - getData: util.NewRequestMetrics(adminScope, "get_node_execution_data"), - getMetrics: util.NewRequestMetrics(adminScope, "get_node_execution_metrics"), - list: util.NewRequestMetrics(adminScope, "list_node_execution"), - listChildren: util.NewRequestMetrics(adminScope, "list_children_node_executions"), + scope: adminScope, + createEvent: util.NewRequestMetrics(adminScope, "create_node_execution_event"), + get: util.NewRequestMetrics(adminScope, "get_node_execution"), + getData: util.NewRequestMetrics(adminScope, "get_node_execution_data"), + getMetrics: util.NewRequestMetrics(adminScope, "get_node_execution_metrics"), + list: util.NewRequestMetrics(adminScope, "list_node_execution"), + listChildren: util.NewRequestMetrics(adminScope, "list_children_node_executions"), + getDynamicNodeWorkflow: util.NewRequestMetrics(adminScope, "get_dynamic_node_workflow"), }, projectEndpointMetrics: projectEndpointMetrics{ scope: adminScope, diff --git a/flyteadmin/pkg/rpc/adminservice/node_execution.go b/flyteadmin/pkg/rpc/adminservice/node_execution.go index 82ae3e1b24..cf17e3ff70 100644 --- a/flyteadmin/pkg/rpc/adminservice/node_execution.go +++ b/flyteadmin/pkg/rpc/adminservice/node_execution.go @@ -48,6 +48,24 @@ func (m *AdminService) GetNodeExecution( return response, nil } +func (m *AdminService) GetDynamicNodeWorkflow(ctx context.Context, request *admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) { + defer m.interceptPanic(ctx, request) + if request == nil { + return nil, status.Errorf(codes.InvalidArgument, "Incorrect request, nil requests not allowed") + } + + var response *admin.DynamicNodeWorkflowResponse + var err error + m.Metrics.nodeExecutionEndpointMetrics.getDynamicNodeWorkflow.Time(func() { + response, err = m.NodeExecutionManager.GetDynamicNodeWorkflow(ctx, *request) + }) + if err != nil { + return nil, util.TransformAndRecordError(err, &m.Metrics.workflowEndpointMetrics.get) + } + m.Metrics.nodeExecutionEndpointMetrics.getDynamicNodeWorkflow.Success() + return response, nil +} + func (m *AdminService) ListNodeExecutions( ctx context.Context, request *admin.NodeExecutionListRequest) (*admin.NodeExecutionList, error) { defer m.interceptPanic(ctx, request) diff --git a/flyteadmin/tests/node_execution_test.go b/flyteadmin/tests/node_execution_test.go index 6b34749e3c..60787e430b 100644 --- a/flyteadmin/tests/node_execution_test.go +++ b/flyteadmin/tests/node_execution_test.go @@ -5,12 +5,17 @@ package tests import ( "context" + "fmt" + "io" + "net/http" "testing" "time" "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "google.golang.org/protobuf/types/known/timestamppb" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" @@ -478,3 +483,120 @@ func TestCreateChildNodeExecutionForTaskExecution(t *testing.T) { assert.True(t, proto.Equal(taskExecutionIdentifier, taskExecutionResp.Id)) assert.True(t, taskExecutionResp.IsParent) } + +func TestGetDynamicNodeWorkflow(t *testing.T) { + ctx := context.Background() + truncateAllTablesForTestingOnly() + populateWorkflowExecutionForTestingOnly(project, domain, name) + client, conn := GetTestAdminServiceClient() + defer conn.Close() + + occurredAt := time.Now() + occurredAtProto := timestamppb.New(occurredAt) + + createTaskAndNodeExecution(ctx, t, client, conn, occurredAtProto) + + _, err := client.CreateTaskEvent(ctx, &admin.TaskExecutionEventRequest{ + RequestId: "request id", + Event: &event.TaskExecutionEvent{ + TaskId: taskIdentifier, + ParentNodeExecutionId: nodeExecutionId, + Phase: core.TaskExecution_RUNNING, + RetryAttempt: 1, + OccurredAt: occurredAtProto, + }, + }) + require.NoError(t, err) + + dynamicWfId := core.Identifier{ + ResourceType: core.ResourceType_WORKFLOW, + Project: "admintests", + Domain: "development", + Name: "name", + Version: "version", + } + dynamicWf := &core.CompiledWorkflowClosure{ + Primary: &core.CompiledWorkflow{ + Template: &core.WorkflowTemplate{ + Id: &dynamicWfId, + Interface: &core.TypedInterface{}, + Nodes: []*core.Node{ + { + Id: "I'm a node", + Target: &core.Node_TaskNode{ + TaskNode: &core.TaskNode{ + Reference: &core.TaskNode_ReferenceId{ + ReferenceId: taskIdentifier, + }, + }, + }, + }, + }, + }, + }, + } + childOccurredAt := occurredAt.Add(time.Minute) + childOccurredAtProto := timestamppb.New(childOccurredAt) + childNodeExecutionID := &core.NodeExecutionIdentifier{ + NodeId: "child_node", + ExecutionId: &core.WorkflowExecutionIdentifier{ + Project: project, + Domain: domain, + Name: name, + }, + } + _, err = client.CreateNodeEvent(ctx, &admin.NodeExecutionEventRequest{ + RequestId: "request id", + Event: &event.NodeExecutionEvent{ + Id: childNodeExecutionID, + Phase: core.NodeExecution_RUNNING, + InputValue: &event.NodeExecutionEvent_InputUri{ + InputUri: inputURI, + }, + OccurredAt: childOccurredAtProto, + ParentTaskMetadata: &event.ParentTaskExecutionMetadata{ + Id: taskExecutionIdentifier, + }, + IsDynamic: true, + IsParent: true, + TargetMetadata: &event.NodeExecutionEvent_TaskNodeMetadata{ + TaskNodeMetadata: &event.TaskNodeMetadata{ + DynamicWorkflow: &event.DynamicWorkflowNodeMetadata{ + Id: &dynamicWfId, + CompiledWorkflow: dynamicWf, + DynamicJobSpecUri: "s3://bla-bla", + }, + }, + }, + }, + }) + require.NoError(t, err) + + t.Run("TestGetDynamicNodeWorkflowGrpc", func(t *testing.T) { + resp, err := client.GetDynamicNodeWorkflow(ctx, &admin.GetDynamicNodeWorkflowRequest{ + Id: childNodeExecutionID, + }) + + assert.NoError(t, err) + assert.True(t, proto.Equal(dynamicWf, resp.GetCompiledWorkflow())) + }) + + t.Run("TestGetDynamicNodeWorkflowHttp", func(t *testing.T) { + url := fmt.Sprintf("%s/api/v1/node_executions/project/domain/execution%%20name/child_node/dynamic_workflow", GetTestHostEndpoint()) + getRequest, err := http.NewRequest(http.MethodGet, url, nil) + require.NoError(t, err) + addHTTPRequestHeaders(getRequest) + + httpClient := &http.Client{Timeout: 2 * time.Second} + resp, err := httpClient.Do(getRequest) + require.NoError(t, err) + defer resp.Body.Close() + + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + require.Equal(t, http.StatusOK, resp.StatusCode, "unexpected resp: %s", string(body)) + wfResp := &admin.DynamicNodeWorkflowResponse{} + require.NoError(t, proto.Unmarshal(body, wfResp)) + assert.True(t, proto.Equal(dynamicWf, wfResp.GetCompiledWorkflow())) + }) +} diff --git a/flyteadmin/tests/shared.go b/flyteadmin/tests/shared.go index 5c81522972..4654fa4c09 100644 --- a/flyteadmin/tests/shared.go +++ b/flyteadmin/tests/shared.go @@ -10,6 +10,7 @@ import ( "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/flyteorg/flyte/flyteadmin/pkg/manager/impl/testutils" "github.com/flyteorg/flyte/flyteidl/clients/go/coreutils" @@ -48,7 +49,7 @@ func insertTasksForTests(t *testing.T, client service.AdminServiceClient) { } _, err := client.CreateTask(ctx, &req) - assert.Nil(t, err) + require.NoError(t, err) } } } @@ -105,7 +106,7 @@ func insertWorkflowsForTests(t *testing.T, client service.AdminServiceClient) { } _, err := client.CreateWorkflow(ctx, &req) - assert.Nil(t, err, "Failed to create workflow test data with err %v", err) + require.NoError(t, err, "Failed to create workflow test data with err %v", err) } } } diff --git a/flyteadmin/tests/task_execution_test.go b/flyteadmin/tests/task_execution_test.go index f0fad9d7f4..e380104684 100644 --- a/flyteadmin/tests/task_execution_test.go +++ b/flyteadmin/tests/task_execution_test.go @@ -14,6 +14,7 @@ import ( ptypesStruct "github.com/golang/protobuf/ptypes/struct" "github.com/golang/protobuf/ptypes/timestamp" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "google.golang.org/grpc" "github.com/flyteorg/flyte/flyteadmin/pkg/manager/impl/testutils" @@ -51,7 +52,7 @@ func createTaskAndNodeExecution( Id: taskIdentifier, Spec: testutils.GetValidTaskRequest().Spec, }) - assert.Nil(t, err) + require.NoError(t, err) _, err = client.CreateNodeEvent(ctx, &admin.NodeExecutionEventRequest{ RequestId: "request id", @@ -64,7 +65,7 @@ func createTaskAndNodeExecution( OccurredAt: occurredAtProto, }, }) - assert.Nil(t, err) + require.NoError(t, err) } func TestCreateTaskExecution(t *testing.T) { diff --git a/flyteidl/clients/go/admin/mocks/AdminServiceClient.go b/flyteidl/clients/go/admin/mocks/AdminServiceClient.go index 38100b9b71..3d4d3039b4 100644 --- a/flyteidl/clients/go/admin/mocks/AdminServiceClient.go +++ b/flyteidl/clients/go/admin/mocks/AdminServiceClient.go @@ -593,6 +593,54 @@ func (_m *AdminServiceClient) GetDescriptionEntity(ctx context.Context, in *admi return r0, r1 } +type AdminServiceClient_GetDynamicNodeWorkflow struct { + *mock.Call +} + +func (_m AdminServiceClient_GetDynamicNodeWorkflow) Return(_a0 *admin.DynamicNodeWorkflowResponse, _a1 error) *AdminServiceClient_GetDynamicNodeWorkflow { + return &AdminServiceClient_GetDynamicNodeWorkflow{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *AdminServiceClient) OnGetDynamicNodeWorkflow(ctx context.Context, in *admin.GetDynamicNodeWorkflowRequest, opts ...grpc.CallOption) *AdminServiceClient_GetDynamicNodeWorkflow { + c_call := _m.On("GetDynamicNodeWorkflow", ctx, in, opts) + return &AdminServiceClient_GetDynamicNodeWorkflow{Call: c_call} +} + +func (_m *AdminServiceClient) OnGetDynamicNodeWorkflowMatch(matchers ...interface{}) *AdminServiceClient_GetDynamicNodeWorkflow { + c_call := _m.On("GetDynamicNodeWorkflow", matchers...) + return &AdminServiceClient_GetDynamicNodeWorkflow{Call: c_call} +} + +// GetDynamicNodeWorkflow provides a mock function with given fields: ctx, in, opts +func (_m *AdminServiceClient) GetDynamicNodeWorkflow(ctx context.Context, in *admin.GetDynamicNodeWorkflowRequest, opts ...grpc.CallOption) (*admin.DynamicNodeWorkflowResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + var r0 *admin.DynamicNodeWorkflowResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.GetDynamicNodeWorkflowRequest, ...grpc.CallOption) *admin.DynamicNodeWorkflowResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.DynamicNodeWorkflowResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.GetDynamicNodeWorkflowRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + type AdminServiceClient_GetExecution struct { *mock.Call } diff --git a/flyteidl/clients/go/admin/mocks/AdminServiceServer.go b/flyteidl/clients/go/admin/mocks/AdminServiceServer.go index 00d069d104..37d540f5ae 100644 --- a/flyteidl/clients/go/admin/mocks/AdminServiceServer.go +++ b/flyteidl/clients/go/admin/mocks/AdminServiceServer.go @@ -507,6 +507,47 @@ func (_m *AdminServiceServer) GetDescriptionEntity(_a0 context.Context, _a1 *adm return r0, r1 } +type AdminServiceServer_GetDynamicNodeWorkflow struct { + *mock.Call +} + +func (_m AdminServiceServer_GetDynamicNodeWorkflow) Return(_a0 *admin.DynamicNodeWorkflowResponse, _a1 error) *AdminServiceServer_GetDynamicNodeWorkflow { + return &AdminServiceServer_GetDynamicNodeWorkflow{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *AdminServiceServer) OnGetDynamicNodeWorkflow(_a0 context.Context, _a1 *admin.GetDynamicNodeWorkflowRequest) *AdminServiceServer_GetDynamicNodeWorkflow { + c_call := _m.On("GetDynamicNodeWorkflow", _a0, _a1) + return &AdminServiceServer_GetDynamicNodeWorkflow{Call: c_call} +} + +func (_m *AdminServiceServer) OnGetDynamicNodeWorkflowMatch(matchers ...interface{}) *AdminServiceServer_GetDynamicNodeWorkflow { + c_call := _m.On("GetDynamicNodeWorkflow", matchers...) + return &AdminServiceServer_GetDynamicNodeWorkflow{Call: c_call} +} + +// GetDynamicNodeWorkflow provides a mock function with given fields: _a0, _a1 +func (_m *AdminServiceServer) GetDynamicNodeWorkflow(_a0 context.Context, _a1 *admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) { + ret := _m.Called(_a0, _a1) + + var r0 *admin.DynamicNodeWorkflowResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.GetDynamicNodeWorkflowRequest) *admin.DynamicNodeWorkflowResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.DynamicNodeWorkflowResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.GetDynamicNodeWorkflowRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + type AdminServiceServer_GetExecution struct { *mock.Call } diff --git a/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.cc index 09fc0a5e20..94d514ae08 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.cc +++ b/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.cc @@ -90,6 +90,14 @@ class NodeExecutionGetDataResponseDefaultTypeInternal { public: ::google::protobuf::internal::ExplicitlyConstructed _instance; } _NodeExecutionGetDataResponse_default_instance_; +class GetDynamicNodeWorkflowRequestDefaultTypeInternal { + public: + ::google::protobuf::internal::ExplicitlyConstructed _instance; +} _GetDynamicNodeWorkflowRequest_default_instance_; +class DynamicNodeWorkflowResponseDefaultTypeInternal { + public: + ::google::protobuf::internal::ExplicitlyConstructed _instance; +} _DynamicNodeWorkflowResponse_default_instance_; } // namespace admin } // namespace flyteidl static void InitDefaultsNodeExecutionGetRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { @@ -284,6 +292,36 @@ ::google::protobuf::internal::SCCInfo<4> scc_info_NodeExecutionGetDataResponse_f &scc_info_DynamicWorkflowNodeMetadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base, &scc_info_FlyteURLs_flyteidl_2fadmin_2fcommon_2eproto.base,}}; +static void InitDefaultsGetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::flyteidl::admin::_GetDynamicNodeWorkflowRequest_default_instance_; + new (ptr) ::flyteidl::admin::GetDynamicNodeWorkflowRequest(); + ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); + } + ::flyteidl::admin::GetDynamicNodeWorkflowRequest::InitAsDefaultInstance(); +} + +::google::protobuf::internal::SCCInfo<1> scc_info_GetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsGetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto}, { + &scc_info_NodeExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto.base,}}; + +static void InitDefaultsDynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::flyteidl::admin::_DynamicNodeWorkflowResponse_default_instance_; + new (ptr) ::flyteidl::admin::DynamicNodeWorkflowResponse(); + ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); + } + ::flyteidl::admin::DynamicNodeWorkflowResponse::InitAsDefaultInstance(); +} + +::google::protobuf::internal::SCCInfo<1> scc_info_DynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsDynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto}, { + &scc_info_CompiledWorkflowClosure_flyteidl_2fcore_2fcompiler_2eproto.base,}}; + void InitDefaults_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { ::google::protobuf::internal::InitSCC(&scc_info_NodeExecutionGetRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_NodeExecutionListRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); @@ -297,9 +335,11 @@ void InitDefaults_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { ::google::protobuf::internal::InitSCC(&scc_info_DynamicWorkflowNodeMetadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_NodeExecutionGetDataRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_NodeExecutionGetDataResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + ::google::protobuf::internal::InitSCC(&scc_info_GetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + ::google::protobuf::internal::InitSCC(&scc_info_DynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); } -::google::protobuf::Metadata file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto[12]; +::google::protobuf::Metadata file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto[14]; constexpr ::google::protobuf::EnumDescriptor const** file_level_enum_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto = nullptr; constexpr ::google::protobuf::ServiceDescriptor const** file_level_service_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto = nullptr; @@ -415,6 +455,18 @@ const ::google::protobuf::uint32 TableStruct_flyteidl_2fadmin_2fnode_5fexecution PROTOBUF_FIELD_OFFSET(::flyteidl::admin::NodeExecutionGetDataResponse, full_outputs_), PROTOBUF_FIELD_OFFSET(::flyteidl::admin::NodeExecutionGetDataResponse, dynamic_workflow_), PROTOBUF_FIELD_OFFSET(::flyteidl::admin::NodeExecutionGetDataResponse, flyte_urls_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::flyteidl::admin::GetDynamicNodeWorkflowRequest, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::flyteidl::admin::GetDynamicNodeWorkflowRequest, id_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::flyteidl::admin::DynamicNodeWorkflowResponse, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::flyteidl::admin::DynamicNodeWorkflowResponse, compiled_workflow_), }; static const ::google::protobuf::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { { 0, -1, sizeof(::flyteidl::admin::NodeExecutionGetRequest)}, @@ -429,6 +481,8 @@ static const ::google::protobuf::internal::MigrationSchema schemas[] PROTOBUF_SE { 86, -1, sizeof(::flyteidl::admin::DynamicWorkflowNodeMetadata)}, { 94, -1, sizeof(::flyteidl::admin::NodeExecutionGetDataRequest)}, { 100, -1, sizeof(::flyteidl::admin::NodeExecutionGetDataResponse)}, + { 111, -1, sizeof(::flyteidl::admin::GetDynamicNodeWorkflowRequest)}, + { 117, -1, sizeof(::flyteidl::admin::DynamicNodeWorkflowResponse)}, }; static ::google::protobuf::Message const * const file_default_instances[] = { @@ -444,12 +498,14 @@ static ::google::protobuf::Message const * const file_default_instances[] = { reinterpret_cast(&::flyteidl::admin::_DynamicWorkflowNodeMetadata_default_instance_), reinterpret_cast(&::flyteidl::admin::_NodeExecutionGetDataRequest_default_instance_), reinterpret_cast(&::flyteidl::admin::_NodeExecutionGetDataResponse_default_instance_), + reinterpret_cast(&::flyteidl::admin::_GetDynamicNodeWorkflowRequest_default_instance_), + reinterpret_cast(&::flyteidl::admin::_DynamicNodeWorkflowResponse_default_instance_), }; ::google::protobuf::internal::AssignDescriptorsTable assign_descriptors_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto = { {}, AddDescriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto, "flyteidl/admin/node_execution.proto", schemas, file_default_instances, TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto::offsets, - file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto, 12, file_level_enum_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto, file_level_service_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto, + file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto, 14, file_level_enum_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto, file_level_service_descriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto, }; const char descriptor_table_protodef_flyteidl_2fadmin_2fnode_5fexecution_2eproto[] = @@ -519,14 +575,18 @@ const char descriptor_table_protodef_flyteidl_2fadmin_2fnode_5fexecution_2eproto ".core.LiteralMap\022E\n\020dynamic_workflow\030\020 \001" "(\0132+.flyteidl.admin.DynamicWorkflowNodeM" "etadata\022-\n\nflyte_urls\030\021 \001(\0132\031.flyteidl.a" - "dmin.FlyteURLsB=Z;github.com/flyteorg/fl" - "yte/flyteidl/gen/pb-go/flyteidl/adminb\006p" - "roto3" + "dmin.FlyteURLs\"S\n\035GetDynamicNodeWorkflow" + "Request\0222\n\002id\030\001 \001(\0132&.flyteidl.core.Node" + "ExecutionIdentifier\"`\n\033DynamicNodeWorkfl" + "owResponse\022A\n\021compiled_workflow\030\001 \001(\0132&." + "flyteidl.core.CompiledWorkflowClosureB=Z" + ";github.com/flyteorg/flyte/flyteidl/gen/" + "pb-go/flyteidl/adminb\006proto3" ; ::google::protobuf::internal::DescriptorTable descriptor_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto = { false, InitDefaults_flyteidl_2fadmin_2fnode_5fexecution_2eproto, descriptor_table_protodef_flyteidl_2fadmin_2fnode_5fexecution_2eproto, - "flyteidl/admin/node_execution.proto", &assign_descriptors_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto, 2725, + "flyteidl/admin/node_execution.proto", &assign_descriptors_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto, 2908, }; void AddDescriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto() { @@ -6668,6 +6728,592 @@ ::google::protobuf::Metadata NodeExecutionGetDataResponse::GetMetadata() const { } +// =================================================================== + +void GetDynamicNodeWorkflowRequest::InitAsDefaultInstance() { + ::flyteidl::admin::_GetDynamicNodeWorkflowRequest_default_instance_._instance.get_mutable()->id_ = const_cast< ::flyteidl::core::NodeExecutionIdentifier*>( + ::flyteidl::core::NodeExecutionIdentifier::internal_default_instance()); +} +class GetDynamicNodeWorkflowRequest::HasBitSetters { + public: + static const ::flyteidl::core::NodeExecutionIdentifier& id(const GetDynamicNodeWorkflowRequest* msg); +}; + +const ::flyteidl::core::NodeExecutionIdentifier& +GetDynamicNodeWorkflowRequest::HasBitSetters::id(const GetDynamicNodeWorkflowRequest* msg) { + return *msg->id_; +} +void GetDynamicNodeWorkflowRequest::clear_id() { + if (GetArenaNoVirtual() == nullptr && id_ != nullptr) { + delete id_; + } + id_ = nullptr; +} +#if !defined(_MSC_VER) || _MSC_VER >= 1900 +const int GetDynamicNodeWorkflowRequest::kIdFieldNumber; +#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 + +GetDynamicNodeWorkflowRequest::GetDynamicNodeWorkflowRequest() + : ::google::protobuf::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:flyteidl.admin.GetDynamicNodeWorkflowRequest) +} +GetDynamicNodeWorkflowRequest::GetDynamicNodeWorkflowRequest(const GetDynamicNodeWorkflowRequest& from) + : ::google::protobuf::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + if (from.has_id()) { + id_ = new ::flyteidl::core::NodeExecutionIdentifier(*from.id_); + } else { + id_ = nullptr; + } + // @@protoc_insertion_point(copy_constructor:flyteidl.admin.GetDynamicNodeWorkflowRequest) +} + +void GetDynamicNodeWorkflowRequest::SharedCtor() { + ::google::protobuf::internal::InitSCC( + &scc_info_GetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + id_ = nullptr; +} + +GetDynamicNodeWorkflowRequest::~GetDynamicNodeWorkflowRequest() { + // @@protoc_insertion_point(destructor:flyteidl.admin.GetDynamicNodeWorkflowRequest) + SharedDtor(); +} + +void GetDynamicNodeWorkflowRequest::SharedDtor() { + if (this != internal_default_instance()) delete id_; +} + +void GetDynamicNodeWorkflowRequest::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const GetDynamicNodeWorkflowRequest& GetDynamicNodeWorkflowRequest::default_instance() { + ::google::protobuf::internal::InitSCC(&::scc_info_GetDynamicNodeWorkflowRequest_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + return *internal_default_instance(); +} + + +void GetDynamicNodeWorkflowRequest::Clear() { +// @@protoc_insertion_point(message_clear_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + ::google::protobuf::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaNoVirtual() == nullptr && id_ != nullptr) { + delete id_; + } + id_ = nullptr; + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* GetDynamicNodeWorkflowRequest::_InternalParse(const char* begin, const char* end, void* object, + ::google::protobuf::internal::ParseContext* ctx) { + auto msg = static_cast(object); + ::google::protobuf::int32 size; (void)size; + int depth; (void)depth; + ::google::protobuf::uint32 tag; + ::google::protobuf::internal::ParseFunc parser_till_end; (void)parser_till_end; + auto ptr = begin; + while (ptr < end) { + ptr = ::google::protobuf::io::Parse32(ptr, &tag); + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); + switch (tag >> 3) { + // .flyteidl.core.NodeExecutionIdentifier id = 1; + case 1: { + if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; + ptr = ::google::protobuf::io::ReadSize(ptr, &size); + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); + parser_till_end = ::flyteidl::core::NodeExecutionIdentifier::_InternalParse; + object = msg->mutable_id(); + if (size > end - ptr) goto len_delim_till_end; + ptr += size; + GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( + {parser_till_end, object}, ptr - size, ptr)); + break; + } + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->EndGroup(tag); + return ptr; + } + auto res = UnknownFieldParse(tag, {_InternalParse, msg}, + ptr, end, msg->_internal_metadata_.mutable_unknown_fields(), ctx); + ptr = res.first; + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr != nullptr); + if (res.second) return ptr; + } + } // switch + } // while + return ptr; +len_delim_till_end: + return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, + {parser_till_end, object}, size); +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool GetDynamicNodeWorkflowRequest::MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::google::protobuf::uint32 tag; + // @@protoc_insertion_point(parse_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + for (;;) { + ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // .flyteidl.core.NodeExecutionIdentifier id = 1; + case 1: { + if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, mutable_id())); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::google::protobuf::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:flyteidl.admin.GetDynamicNodeWorkflowRequest) + return true; +failure: + // @@protoc_insertion_point(parse_failure:flyteidl.admin.GetDynamicNodeWorkflowRequest) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void GetDynamicNodeWorkflowRequest::SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .flyteidl.core.NodeExecutionIdentifier id = 1; + if (this->has_id()) { + ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( + 1, HasBitSetters::id(this), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::google::protobuf::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:flyteidl.admin.GetDynamicNodeWorkflowRequest) +} + +::google::protobuf::uint8* GetDynamicNodeWorkflowRequest::InternalSerializeWithCachedSizesToArray( + ::google::protobuf::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .flyteidl.core.NodeExecutionIdentifier id = 1; + if (this->has_id()) { + target = ::google::protobuf::internal::WireFormatLite:: + InternalWriteMessageToArray( + 1, HasBitSetters::id(this), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:flyteidl.admin.GetDynamicNodeWorkflowRequest) + return target; +} + +size_t GetDynamicNodeWorkflowRequest::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::google::protobuf::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .flyteidl.core.NodeExecutionIdentifier id = 1; + if (this->has_id()) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::MessageSize( + *id_); + } + + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void GetDynamicNodeWorkflowRequest::MergeFrom(const ::google::protobuf::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + GOOGLE_DCHECK_NE(&from, this); + const GetDynamicNodeWorkflowRequest* source = + ::google::protobuf::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.admin.GetDynamicNodeWorkflowRequest) + ::google::protobuf::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.admin.GetDynamicNodeWorkflowRequest) + MergeFrom(*source); + } +} + +void GetDynamicNodeWorkflowRequest::MergeFrom(const GetDynamicNodeWorkflowRequest& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.has_id()) { + mutable_id()->::flyteidl::core::NodeExecutionIdentifier::MergeFrom(from.id()); + } +} + +void GetDynamicNodeWorkflowRequest::CopyFrom(const ::google::protobuf::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void GetDynamicNodeWorkflowRequest::CopyFrom(const GetDynamicNodeWorkflowRequest& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.admin.GetDynamicNodeWorkflowRequest) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool GetDynamicNodeWorkflowRequest::IsInitialized() const { + return true; +} + +void GetDynamicNodeWorkflowRequest::Swap(GetDynamicNodeWorkflowRequest* other) { + if (other == this) return; + InternalSwap(other); +} +void GetDynamicNodeWorkflowRequest::InternalSwap(GetDynamicNodeWorkflowRequest* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + swap(id_, other->id_); +} + +::google::protobuf::Metadata GetDynamicNodeWorkflowRequest::GetMetadata() const { + ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto); + return ::file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto[kIndexInFileMessages]; +} + + +// =================================================================== + +void DynamicNodeWorkflowResponse::InitAsDefaultInstance() { + ::flyteidl::admin::_DynamicNodeWorkflowResponse_default_instance_._instance.get_mutable()->compiled_workflow_ = const_cast< ::flyteidl::core::CompiledWorkflowClosure*>( + ::flyteidl::core::CompiledWorkflowClosure::internal_default_instance()); +} +class DynamicNodeWorkflowResponse::HasBitSetters { + public: + static const ::flyteidl::core::CompiledWorkflowClosure& compiled_workflow(const DynamicNodeWorkflowResponse* msg); +}; + +const ::flyteidl::core::CompiledWorkflowClosure& +DynamicNodeWorkflowResponse::HasBitSetters::compiled_workflow(const DynamicNodeWorkflowResponse* msg) { + return *msg->compiled_workflow_; +} +void DynamicNodeWorkflowResponse::clear_compiled_workflow() { + if (GetArenaNoVirtual() == nullptr && compiled_workflow_ != nullptr) { + delete compiled_workflow_; + } + compiled_workflow_ = nullptr; +} +#if !defined(_MSC_VER) || _MSC_VER >= 1900 +const int DynamicNodeWorkflowResponse::kCompiledWorkflowFieldNumber; +#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 + +DynamicNodeWorkflowResponse::DynamicNodeWorkflowResponse() + : ::google::protobuf::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:flyteidl.admin.DynamicNodeWorkflowResponse) +} +DynamicNodeWorkflowResponse::DynamicNodeWorkflowResponse(const DynamicNodeWorkflowResponse& from) + : ::google::protobuf::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + if (from.has_compiled_workflow()) { + compiled_workflow_ = new ::flyteidl::core::CompiledWorkflowClosure(*from.compiled_workflow_); + } else { + compiled_workflow_ = nullptr; + } + // @@protoc_insertion_point(copy_constructor:flyteidl.admin.DynamicNodeWorkflowResponse) +} + +void DynamicNodeWorkflowResponse::SharedCtor() { + ::google::protobuf::internal::InitSCC( + &scc_info_DynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + compiled_workflow_ = nullptr; +} + +DynamicNodeWorkflowResponse::~DynamicNodeWorkflowResponse() { + // @@protoc_insertion_point(destructor:flyteidl.admin.DynamicNodeWorkflowResponse) + SharedDtor(); +} + +void DynamicNodeWorkflowResponse::SharedDtor() { + if (this != internal_default_instance()) delete compiled_workflow_; +} + +void DynamicNodeWorkflowResponse::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const DynamicNodeWorkflowResponse& DynamicNodeWorkflowResponse::default_instance() { + ::google::protobuf::internal::InitSCC(&::scc_info_DynamicNodeWorkflowResponse_flyteidl_2fadmin_2fnode_5fexecution_2eproto.base); + return *internal_default_instance(); +} + + +void DynamicNodeWorkflowResponse::Clear() { +// @@protoc_insertion_point(message_clear_start:flyteidl.admin.DynamicNodeWorkflowResponse) + ::google::protobuf::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaNoVirtual() == nullptr && compiled_workflow_ != nullptr) { + delete compiled_workflow_; + } + compiled_workflow_ = nullptr; + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* DynamicNodeWorkflowResponse::_InternalParse(const char* begin, const char* end, void* object, + ::google::protobuf::internal::ParseContext* ctx) { + auto msg = static_cast(object); + ::google::protobuf::int32 size; (void)size; + int depth; (void)depth; + ::google::protobuf::uint32 tag; + ::google::protobuf::internal::ParseFunc parser_till_end; (void)parser_till_end; + auto ptr = begin; + while (ptr < end) { + ptr = ::google::protobuf::io::Parse32(ptr, &tag); + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); + switch (tag >> 3) { + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + case 1: { + if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; + ptr = ::google::protobuf::io::ReadSize(ptr, &size); + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); + parser_till_end = ::flyteidl::core::CompiledWorkflowClosure::_InternalParse; + object = msg->mutable_compiled_workflow(); + if (size > end - ptr) goto len_delim_till_end; + ptr += size; + GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( + {parser_till_end, object}, ptr - size, ptr)); + break; + } + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->EndGroup(tag); + return ptr; + } + auto res = UnknownFieldParse(tag, {_InternalParse, msg}, + ptr, end, msg->_internal_metadata_.mutable_unknown_fields(), ctx); + ptr = res.first; + GOOGLE_PROTOBUF_PARSER_ASSERT(ptr != nullptr); + if (res.second) return ptr; + } + } // switch + } // while + return ptr; +len_delim_till_end: + return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, + {parser_till_end, object}, size); +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool DynamicNodeWorkflowResponse::MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::google::protobuf::uint32 tag; + // @@protoc_insertion_point(parse_start:flyteidl.admin.DynamicNodeWorkflowResponse) + for (;;) { + ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + case 1: { + if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, mutable_compiled_workflow())); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::google::protobuf::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:flyteidl.admin.DynamicNodeWorkflowResponse) + return true; +failure: + // @@protoc_insertion_point(parse_failure:flyteidl.admin.DynamicNodeWorkflowResponse) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void DynamicNodeWorkflowResponse::SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:flyteidl.admin.DynamicNodeWorkflowResponse) + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + if (this->has_compiled_workflow()) { + ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( + 1, HasBitSetters::compiled_workflow(this), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::google::protobuf::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:flyteidl.admin.DynamicNodeWorkflowResponse) +} + +::google::protobuf::uint8* DynamicNodeWorkflowResponse::InternalSerializeWithCachedSizesToArray( + ::google::protobuf::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:flyteidl.admin.DynamicNodeWorkflowResponse) + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + if (this->has_compiled_workflow()) { + target = ::google::protobuf::internal::WireFormatLite:: + InternalWriteMessageToArray( + 1, HasBitSetters::compiled_workflow(this), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:flyteidl.admin.DynamicNodeWorkflowResponse) + return target; +} + +size_t DynamicNodeWorkflowResponse::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:flyteidl.admin.DynamicNodeWorkflowResponse) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::google::protobuf::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + if (this->has_compiled_workflow()) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::MessageSize( + *compiled_workflow_); + } + + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void DynamicNodeWorkflowResponse::MergeFrom(const ::google::protobuf::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.admin.DynamicNodeWorkflowResponse) + GOOGLE_DCHECK_NE(&from, this); + const DynamicNodeWorkflowResponse* source = + ::google::protobuf::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.admin.DynamicNodeWorkflowResponse) + ::google::protobuf::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.admin.DynamicNodeWorkflowResponse) + MergeFrom(*source); + } +} + +void DynamicNodeWorkflowResponse::MergeFrom(const DynamicNodeWorkflowResponse& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.admin.DynamicNodeWorkflowResponse) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.has_compiled_workflow()) { + mutable_compiled_workflow()->::flyteidl::core::CompiledWorkflowClosure::MergeFrom(from.compiled_workflow()); + } +} + +void DynamicNodeWorkflowResponse::CopyFrom(const ::google::protobuf::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.admin.DynamicNodeWorkflowResponse) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void DynamicNodeWorkflowResponse::CopyFrom(const DynamicNodeWorkflowResponse& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.admin.DynamicNodeWorkflowResponse) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool DynamicNodeWorkflowResponse::IsInitialized() const { + return true; +} + +void DynamicNodeWorkflowResponse::Swap(DynamicNodeWorkflowResponse* other) { + if (other == this) return; + InternalSwap(other); +} +void DynamicNodeWorkflowResponse::InternalSwap(DynamicNodeWorkflowResponse* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + swap(compiled_workflow_, other->compiled_workflow_); +} + +::google::protobuf::Metadata DynamicNodeWorkflowResponse::GetMetadata() const { + ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fadmin_2fnode_5fexecution_2eproto); + return ::file_level_metadata_flyteidl_2fadmin_2fnode_5fexecution_2eproto[kIndexInFileMessages]; +} + + // @@protoc_insertion_point(namespace_scope) } // namespace admin } // namespace flyteidl @@ -6709,6 +7355,12 @@ template<> PROTOBUF_NOINLINE ::flyteidl::admin::NodeExecutionGetDataRequest* Are template<> PROTOBUF_NOINLINE ::flyteidl::admin::NodeExecutionGetDataResponse* Arena::CreateMaybeMessage< ::flyteidl::admin::NodeExecutionGetDataResponse >(Arena* arena) { return Arena::CreateInternal< ::flyteidl::admin::NodeExecutionGetDataResponse >(arena); } +template<> PROTOBUF_NOINLINE ::flyteidl::admin::GetDynamicNodeWorkflowRequest* Arena::CreateMaybeMessage< ::flyteidl::admin::GetDynamicNodeWorkflowRequest >(Arena* arena) { + return Arena::CreateInternal< ::flyteidl::admin::GetDynamicNodeWorkflowRequest >(arena); +} +template<> PROTOBUF_NOINLINE ::flyteidl::admin::DynamicNodeWorkflowResponse* Arena::CreateMaybeMessage< ::flyteidl::admin::DynamicNodeWorkflowResponse >(Arena* arena) { + return Arena::CreateInternal< ::flyteidl::admin::DynamicNodeWorkflowResponse >(arena); +} } // namespace protobuf } // namespace google diff --git a/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.h b/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.h index 88da7e3612..15e691830c 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.h +++ b/flyteidl/gen/pb-cpp/flyteidl/admin/node_execution.pb.h @@ -49,7 +49,7 @@ struct TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto { PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::google::protobuf::internal::AuxillaryParseTableField aux[] PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::google::protobuf::internal::ParseTable schema[12] + static const ::google::protobuf::internal::ParseTable schema[14] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::google::protobuf::internal::FieldMetadata field_metadata[]; static const ::google::protobuf::internal::SerializationTable serialization_table[]; @@ -58,9 +58,15 @@ struct TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto { void AddDescriptors_flyteidl_2fadmin_2fnode_5fexecution_2eproto(); namespace flyteidl { namespace admin { +class DynamicNodeWorkflowResponse; +class DynamicNodeWorkflowResponseDefaultTypeInternal; +extern DynamicNodeWorkflowResponseDefaultTypeInternal _DynamicNodeWorkflowResponse_default_instance_; class DynamicWorkflowNodeMetadata; class DynamicWorkflowNodeMetadataDefaultTypeInternal; extern DynamicWorkflowNodeMetadataDefaultTypeInternal _DynamicWorkflowNodeMetadata_default_instance_; +class GetDynamicNodeWorkflowRequest; +class GetDynamicNodeWorkflowRequestDefaultTypeInternal; +extern GetDynamicNodeWorkflowRequestDefaultTypeInternal _GetDynamicNodeWorkflowRequest_default_instance_; class NodeExecution; class NodeExecutionDefaultTypeInternal; extern NodeExecutionDefaultTypeInternal _NodeExecution_default_instance_; @@ -98,7 +104,9 @@ extern WorkflowNodeMetadataDefaultTypeInternal _WorkflowNodeMetadata_default_ins } // namespace flyteidl namespace google { namespace protobuf { +template<> ::flyteidl::admin::DynamicNodeWorkflowResponse* Arena::CreateMaybeMessage<::flyteidl::admin::DynamicNodeWorkflowResponse>(Arena*); template<> ::flyteidl::admin::DynamicWorkflowNodeMetadata* Arena::CreateMaybeMessage<::flyteidl::admin::DynamicWorkflowNodeMetadata>(Arena*); +template<> ::flyteidl::admin::GetDynamicNodeWorkflowRequest* Arena::CreateMaybeMessage<::flyteidl::admin::GetDynamicNodeWorkflowRequest>(Arena*); template<> ::flyteidl::admin::NodeExecution* Arena::CreateMaybeMessage<::flyteidl::admin::NodeExecution>(Arena*); template<> ::flyteidl::admin::NodeExecutionClosure* Arena::CreateMaybeMessage<::flyteidl::admin::NodeExecutionClosure>(Arena*); template<> ::flyteidl::admin::NodeExecutionForTaskListRequest* Arena::CreateMaybeMessage<::flyteidl::admin::NodeExecutionForTaskListRequest>(Arena*); @@ -1956,6 +1964,236 @@ class NodeExecutionGetDataResponse final : mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto; }; +// ------------------------------------------------------------------- + +class GetDynamicNodeWorkflowRequest final : + public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.admin.GetDynamicNodeWorkflowRequest) */ { + public: + GetDynamicNodeWorkflowRequest(); + virtual ~GetDynamicNodeWorkflowRequest(); + + GetDynamicNodeWorkflowRequest(const GetDynamicNodeWorkflowRequest& from); + + inline GetDynamicNodeWorkflowRequest& operator=(const GetDynamicNodeWorkflowRequest& from) { + CopyFrom(from); + return *this; + } + #if LANG_CXX11 + GetDynamicNodeWorkflowRequest(GetDynamicNodeWorkflowRequest&& from) noexcept + : GetDynamicNodeWorkflowRequest() { + *this = ::std::move(from); + } + + inline GetDynamicNodeWorkflowRequest& operator=(GetDynamicNodeWorkflowRequest&& from) noexcept { + if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) { + if (this != &from) InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + #endif + static const ::google::protobuf::Descriptor* descriptor() { + return default_instance().GetDescriptor(); + } + static const GetDynamicNodeWorkflowRequest& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const GetDynamicNodeWorkflowRequest* internal_default_instance() { + return reinterpret_cast( + &_GetDynamicNodeWorkflowRequest_default_instance_); + } + static constexpr int kIndexInFileMessages = + 12; + + void Swap(GetDynamicNodeWorkflowRequest* other); + friend void swap(GetDynamicNodeWorkflowRequest& a, GetDynamicNodeWorkflowRequest& b) { + a.Swap(&b); + } + + // implements Message ---------------------------------------------- + + inline GetDynamicNodeWorkflowRequest* New() const final { + return CreateMaybeMessage(nullptr); + } + + GetDynamicNodeWorkflowRequest* New(::google::protobuf::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::google::protobuf::Message& from) final; + void MergeFrom(const ::google::protobuf::Message& from) final; + void CopyFrom(const GetDynamicNodeWorkflowRequest& from); + void MergeFrom(const GetDynamicNodeWorkflowRequest& from); + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + #if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + static const char* _InternalParse(const char* begin, const char* end, void* object, ::google::protobuf::internal::ParseContext* ctx); + ::google::protobuf::internal::ParseFunc _ParseFunc() const final { return _InternalParse; } + #else + bool MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) final; + #endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + void SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const final; + ::google::protobuf::uint8* InternalSerializeWithCachedSizesToArray( + ::google::protobuf::uint8* target) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(GetDynamicNodeWorkflowRequest* other); + private: + inline ::google::protobuf::Arena* GetArenaNoVirtual() const { + return nullptr; + } + inline void* MaybeArenaPtr() const { + return nullptr; + } + public: + + ::google::protobuf::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + // .flyteidl.core.NodeExecutionIdentifier id = 1; + bool has_id() const; + void clear_id(); + static const int kIdFieldNumber = 1; + const ::flyteidl::core::NodeExecutionIdentifier& id() const; + ::flyteidl::core::NodeExecutionIdentifier* release_id(); + ::flyteidl::core::NodeExecutionIdentifier* mutable_id(); + void set_allocated_id(::flyteidl::core::NodeExecutionIdentifier* id); + + // @@protoc_insertion_point(class_scope:flyteidl.admin.GetDynamicNodeWorkflowRequest) + private: + class HasBitSetters; + + ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; + ::flyteidl::core::NodeExecutionIdentifier* id_; + mutable ::google::protobuf::internal::CachedSize _cached_size_; + friend struct ::TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto; +}; +// ------------------------------------------------------------------- + +class DynamicNodeWorkflowResponse final : + public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.admin.DynamicNodeWorkflowResponse) */ { + public: + DynamicNodeWorkflowResponse(); + virtual ~DynamicNodeWorkflowResponse(); + + DynamicNodeWorkflowResponse(const DynamicNodeWorkflowResponse& from); + + inline DynamicNodeWorkflowResponse& operator=(const DynamicNodeWorkflowResponse& from) { + CopyFrom(from); + return *this; + } + #if LANG_CXX11 + DynamicNodeWorkflowResponse(DynamicNodeWorkflowResponse&& from) noexcept + : DynamicNodeWorkflowResponse() { + *this = ::std::move(from); + } + + inline DynamicNodeWorkflowResponse& operator=(DynamicNodeWorkflowResponse&& from) noexcept { + if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) { + if (this != &from) InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + #endif + static const ::google::protobuf::Descriptor* descriptor() { + return default_instance().GetDescriptor(); + } + static const DynamicNodeWorkflowResponse& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const DynamicNodeWorkflowResponse* internal_default_instance() { + return reinterpret_cast( + &_DynamicNodeWorkflowResponse_default_instance_); + } + static constexpr int kIndexInFileMessages = + 13; + + void Swap(DynamicNodeWorkflowResponse* other); + friend void swap(DynamicNodeWorkflowResponse& a, DynamicNodeWorkflowResponse& b) { + a.Swap(&b); + } + + // implements Message ---------------------------------------------- + + inline DynamicNodeWorkflowResponse* New() const final { + return CreateMaybeMessage(nullptr); + } + + DynamicNodeWorkflowResponse* New(::google::protobuf::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::google::protobuf::Message& from) final; + void MergeFrom(const ::google::protobuf::Message& from) final; + void CopyFrom(const DynamicNodeWorkflowResponse& from); + void MergeFrom(const DynamicNodeWorkflowResponse& from); + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + #if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + static const char* _InternalParse(const char* begin, const char* end, void* object, ::google::protobuf::internal::ParseContext* ctx); + ::google::protobuf::internal::ParseFunc _ParseFunc() const final { return _InternalParse; } + #else + bool MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) final; + #endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + void SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const final; + ::google::protobuf::uint8* InternalSerializeWithCachedSizesToArray( + ::google::protobuf::uint8* target) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(DynamicNodeWorkflowResponse* other); + private: + inline ::google::protobuf::Arena* GetArenaNoVirtual() const { + return nullptr; + } + inline void* MaybeArenaPtr() const { + return nullptr; + } + public: + + ::google::protobuf::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + // .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + bool has_compiled_workflow() const; + void clear_compiled_workflow(); + static const int kCompiledWorkflowFieldNumber = 1; + const ::flyteidl::core::CompiledWorkflowClosure& compiled_workflow() const; + ::flyteidl::core::CompiledWorkflowClosure* release_compiled_workflow(); + ::flyteidl::core::CompiledWorkflowClosure* mutable_compiled_workflow(); + void set_allocated_compiled_workflow(::flyteidl::core::CompiledWorkflowClosure* compiled_workflow); + + // @@protoc_insertion_point(class_scope:flyteidl.admin.DynamicNodeWorkflowResponse) + private: + class HasBitSetters; + + ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; + ::flyteidl::core::CompiledWorkflowClosure* compiled_workflow_; + mutable ::google::protobuf::internal::CachedSize _cached_size_; + friend struct ::TableStruct_flyteidl_2fadmin_2fnode_5fexecution_2eproto; +}; // =================================================================== @@ -4147,6 +4385,104 @@ inline void NodeExecutionGetDataResponse::set_allocated_flyte_urls(::flyteidl::a // @@protoc_insertion_point(field_set_allocated:flyteidl.admin.NodeExecutionGetDataResponse.flyte_urls) } +// ------------------------------------------------------------------- + +// GetDynamicNodeWorkflowRequest + +// .flyteidl.core.NodeExecutionIdentifier id = 1; +inline bool GetDynamicNodeWorkflowRequest::has_id() const { + return this != internal_default_instance() && id_ != nullptr; +} +inline const ::flyteidl::core::NodeExecutionIdentifier& GetDynamicNodeWorkflowRequest::id() const { + const ::flyteidl::core::NodeExecutionIdentifier* p = id_; + // @@protoc_insertion_point(field_get:flyteidl.admin.GetDynamicNodeWorkflowRequest.id) + return p != nullptr ? *p : *reinterpret_cast( + &::flyteidl::core::_NodeExecutionIdentifier_default_instance_); +} +inline ::flyteidl::core::NodeExecutionIdentifier* GetDynamicNodeWorkflowRequest::release_id() { + // @@protoc_insertion_point(field_release:flyteidl.admin.GetDynamicNodeWorkflowRequest.id) + + ::flyteidl::core::NodeExecutionIdentifier* temp = id_; + id_ = nullptr; + return temp; +} +inline ::flyteidl::core::NodeExecutionIdentifier* GetDynamicNodeWorkflowRequest::mutable_id() { + + if (id_ == nullptr) { + auto* p = CreateMaybeMessage<::flyteidl::core::NodeExecutionIdentifier>(GetArenaNoVirtual()); + id_ = p; + } + // @@protoc_insertion_point(field_mutable:flyteidl.admin.GetDynamicNodeWorkflowRequest.id) + return id_; +} +inline void GetDynamicNodeWorkflowRequest::set_allocated_id(::flyteidl::core::NodeExecutionIdentifier* id) { + ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); + if (message_arena == nullptr) { + delete reinterpret_cast< ::google::protobuf::MessageLite*>(id_); + } + if (id) { + ::google::protobuf::Arena* submessage_arena = nullptr; + if (message_arena != submessage_arena) { + id = ::google::protobuf::internal::GetOwnedMessage( + message_arena, id, submessage_arena); + } + + } else { + + } + id_ = id; + // @@protoc_insertion_point(field_set_allocated:flyteidl.admin.GetDynamicNodeWorkflowRequest.id) +} + +// ------------------------------------------------------------------- + +// DynamicNodeWorkflowResponse + +// .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; +inline bool DynamicNodeWorkflowResponse::has_compiled_workflow() const { + return this != internal_default_instance() && compiled_workflow_ != nullptr; +} +inline const ::flyteidl::core::CompiledWorkflowClosure& DynamicNodeWorkflowResponse::compiled_workflow() const { + const ::flyteidl::core::CompiledWorkflowClosure* p = compiled_workflow_; + // @@protoc_insertion_point(field_get:flyteidl.admin.DynamicNodeWorkflowResponse.compiled_workflow) + return p != nullptr ? *p : *reinterpret_cast( + &::flyteidl::core::_CompiledWorkflowClosure_default_instance_); +} +inline ::flyteidl::core::CompiledWorkflowClosure* DynamicNodeWorkflowResponse::release_compiled_workflow() { + // @@protoc_insertion_point(field_release:flyteidl.admin.DynamicNodeWorkflowResponse.compiled_workflow) + + ::flyteidl::core::CompiledWorkflowClosure* temp = compiled_workflow_; + compiled_workflow_ = nullptr; + return temp; +} +inline ::flyteidl::core::CompiledWorkflowClosure* DynamicNodeWorkflowResponse::mutable_compiled_workflow() { + + if (compiled_workflow_ == nullptr) { + auto* p = CreateMaybeMessage<::flyteidl::core::CompiledWorkflowClosure>(GetArenaNoVirtual()); + compiled_workflow_ = p; + } + // @@protoc_insertion_point(field_mutable:flyteidl.admin.DynamicNodeWorkflowResponse.compiled_workflow) + return compiled_workflow_; +} +inline void DynamicNodeWorkflowResponse::set_allocated_compiled_workflow(::flyteidl::core::CompiledWorkflowClosure* compiled_workflow) { + ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); + if (message_arena == nullptr) { + delete reinterpret_cast< ::google::protobuf::MessageLite*>(compiled_workflow_); + } + if (compiled_workflow) { + ::google::protobuf::Arena* submessage_arena = nullptr; + if (message_arena != submessage_arena) { + compiled_workflow = ::google::protobuf::internal::GetOwnedMessage( + message_arena, compiled_workflow, submessage_arena); + } + + } else { + + } + compiled_workflow_ = compiled_workflow; + // @@protoc_insertion_point(field_set_allocated:flyteidl.admin.DynamicNodeWorkflowResponse.compiled_workflow) +} + #ifdef __GNUC__ #pragma GCC diagnostic pop #endif // __GNUC__ @@ -4172,6 +4508,10 @@ inline void NodeExecutionGetDataResponse::set_allocated_flyte_urls(::flyteidl::a // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) diff --git a/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.cc index cf8c19b8a1..2af2441047 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.cc +++ b/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.cc @@ -44,6 +44,7 @@ static const char* AdminService_method_names[] = { "/flyteidl.service.AdminService/ListExecutions", "/flyteidl.service.AdminService/TerminateExecution", "/flyteidl.service.AdminService/GetNodeExecution", + "/flyteidl.service.AdminService/GetDynamicNodeWorkflow", "/flyteidl.service.AdminService/ListNodeExecutions", "/flyteidl.service.AdminService/ListNodeExecutionsForTask", "/flyteidl.service.AdminService/GetNodeExecutionData", @@ -106,35 +107,36 @@ AdminService::Stub::Stub(const std::shared_ptr< ::grpc::ChannelInterface>& chann , rpcmethod_ListExecutions_(AdminService_method_names[21], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) , rpcmethod_TerminateExecution_(AdminService_method_names[22], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) , rpcmethod_GetNodeExecution_(AdminService_method_names[23], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListNodeExecutions_(AdminService_method_names[24], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListNodeExecutionsForTask_(AdminService_method_names[25], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetNodeExecutionData_(AdminService_method_names[26], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_RegisterProject_(AdminService_method_names[27], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_UpdateProject_(AdminService_method_names[28], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListProjects_(AdminService_method_names[29], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_CreateWorkflowEvent_(AdminService_method_names[30], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_CreateNodeEvent_(AdminService_method_names[31], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_CreateTaskEvent_(AdminService_method_names[32], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetTaskExecution_(AdminService_method_names[33], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListTaskExecutions_(AdminService_method_names[34], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetTaskExecutionData_(AdminService_method_names[35], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_UpdateProjectDomainAttributes_(AdminService_method_names[36], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetProjectDomainAttributes_(AdminService_method_names[37], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_DeleteProjectDomainAttributes_(AdminService_method_names[38], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_UpdateProjectAttributes_(AdminService_method_names[39], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetProjectAttributes_(AdminService_method_names[40], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_DeleteProjectAttributes_(AdminService_method_names[41], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_UpdateWorkflowAttributes_(AdminService_method_names[42], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetWorkflowAttributes_(AdminService_method_names[43], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_DeleteWorkflowAttributes_(AdminService_method_names[44], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListMatchableAttributes_(AdminService_method_names[45], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListNamedEntities_(AdminService_method_names[46], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetNamedEntity_(AdminService_method_names[47], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_UpdateNamedEntity_(AdminService_method_names[48], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetVersion_(AdminService_method_names[49], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetDescriptionEntity_(AdminService_method_names[50], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListDescriptionEntities_(AdminService_method_names[51], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetExecutionMetrics_(AdminService_method_names[52], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetDynamicNodeWorkflow_(AdminService_method_names[24], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListNodeExecutions_(AdminService_method_names[25], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListNodeExecutionsForTask_(AdminService_method_names[26], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetNodeExecutionData_(AdminService_method_names[27], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_RegisterProject_(AdminService_method_names[28], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_UpdateProject_(AdminService_method_names[29], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListProjects_(AdminService_method_names[30], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_CreateWorkflowEvent_(AdminService_method_names[31], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_CreateNodeEvent_(AdminService_method_names[32], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_CreateTaskEvent_(AdminService_method_names[33], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetTaskExecution_(AdminService_method_names[34], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListTaskExecutions_(AdminService_method_names[35], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetTaskExecutionData_(AdminService_method_names[36], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_UpdateProjectDomainAttributes_(AdminService_method_names[37], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetProjectDomainAttributes_(AdminService_method_names[38], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_DeleteProjectDomainAttributes_(AdminService_method_names[39], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_UpdateProjectAttributes_(AdminService_method_names[40], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetProjectAttributes_(AdminService_method_names[41], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_DeleteProjectAttributes_(AdminService_method_names[42], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_UpdateWorkflowAttributes_(AdminService_method_names[43], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetWorkflowAttributes_(AdminService_method_names[44], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_DeleteWorkflowAttributes_(AdminService_method_names[45], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListMatchableAttributes_(AdminService_method_names[46], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListNamedEntities_(AdminService_method_names[47], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetNamedEntity_(AdminService_method_names[48], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_UpdateNamedEntity_(AdminService_method_names[49], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetVersion_(AdminService_method_names[50], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetDescriptionEntity_(AdminService_method_names[51], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_ListDescriptionEntities_(AdminService_method_names[52], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) + , rpcmethod_GetExecutionMetrics_(AdminService_method_names[53], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) {} ::grpc::Status AdminService::Stub::CreateTask(::grpc::ClientContext* context, const ::flyteidl::admin::TaskCreateRequest& request, ::flyteidl::admin::TaskCreateResponse* response) { @@ -809,6 +811,34 @@ ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecution>* AdminServi return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::admin::NodeExecution>::Create(channel_.get(), cq, rpcmethod_GetNodeExecution_, context, request, false); } +::grpc::Status AdminService::Stub::GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) { + return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_GetDynamicNodeWorkflow_, context, request, response); +} + +void AdminService::Stub::experimental_async::GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function f) { + ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_GetDynamicNodeWorkflow_, context, request, response, std::move(f)); +} + +void AdminService::Stub::experimental_async::GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function f) { + ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_GetDynamicNodeWorkflow_, context, request, response, std::move(f)); +} + +void AdminService::Stub::experimental_async::GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { + ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_GetDynamicNodeWorkflow_, context, request, response, reactor); +} + +void AdminService::Stub::experimental_async::GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { + ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_GetDynamicNodeWorkflow_, context, request, response, reactor); +} + +::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>* AdminService::Stub::AsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::admin::DynamicNodeWorkflowResponse>::Create(channel_.get(), cq, rpcmethod_GetDynamicNodeWorkflow_, context, request, true); +} + +::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>* AdminService::Stub::PrepareAsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::admin::DynamicNodeWorkflowResponse>::Create(channel_.get(), cq, rpcmethod_GetDynamicNodeWorkflow_, context, request, false); +} + ::grpc::Status AdminService::Stub::ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::flyteidl::admin::NodeExecutionList* response) { return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_ListNodeExecutions_, context, request, response); } @@ -1745,145 +1775,150 @@ AdminService::Service::Service() { AddMethod(new ::grpc::internal::RpcServiceMethod( AdminService_method_names[24], ::grpc::internal::RpcMethod::NORMAL_RPC, + new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::GetDynamicNodeWorkflowRequest, ::flyteidl::admin::DynamicNodeWorkflowResponse>( + std::mem_fn(&AdminService::Service::GetDynamicNodeWorkflow), this))); + AddMethod(new ::grpc::internal::RpcServiceMethod( + AdminService_method_names[25], + ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NodeExecutionListRequest, ::flyteidl::admin::NodeExecutionList>( std::mem_fn(&AdminService::Service::ListNodeExecutions), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[25], + AdminService_method_names[26], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NodeExecutionForTaskListRequest, ::flyteidl::admin::NodeExecutionList>( std::mem_fn(&AdminService::Service::ListNodeExecutionsForTask), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[26], + AdminService_method_names[27], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NodeExecutionGetDataRequest, ::flyteidl::admin::NodeExecutionGetDataResponse>( std::mem_fn(&AdminService::Service::GetNodeExecutionData), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[27], + AdminService_method_names[28], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectRegisterRequest, ::flyteidl::admin::ProjectRegisterResponse>( std::mem_fn(&AdminService::Service::RegisterProject), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[28], + AdminService_method_names[29], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::Project, ::flyteidl::admin::ProjectUpdateResponse>( std::mem_fn(&AdminService::Service::UpdateProject), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[29], + AdminService_method_names[30], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectListRequest, ::flyteidl::admin::Projects>( std::mem_fn(&AdminService::Service::ListProjects), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[30], + AdminService_method_names[31], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::WorkflowExecutionEventRequest, ::flyteidl::admin::WorkflowExecutionEventResponse>( std::mem_fn(&AdminService::Service::CreateWorkflowEvent), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[31], + AdminService_method_names[32], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NodeExecutionEventRequest, ::flyteidl::admin::NodeExecutionEventResponse>( std::mem_fn(&AdminService::Service::CreateNodeEvent), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[32], + AdminService_method_names[33], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::TaskExecutionEventRequest, ::flyteidl::admin::TaskExecutionEventResponse>( std::mem_fn(&AdminService::Service::CreateTaskEvent), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[33], + AdminService_method_names[34], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::TaskExecutionGetRequest, ::flyteidl::admin::TaskExecution>( std::mem_fn(&AdminService::Service::GetTaskExecution), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[34], + AdminService_method_names[35], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::TaskExecutionListRequest, ::flyteidl::admin::TaskExecutionList>( std::mem_fn(&AdminService::Service::ListTaskExecutions), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[35], + AdminService_method_names[36], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::TaskExecutionGetDataRequest, ::flyteidl::admin::TaskExecutionGetDataResponse>( std::mem_fn(&AdminService::Service::GetTaskExecutionData), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[36], + AdminService_method_names[37], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectDomainAttributesUpdateRequest, ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>( std::mem_fn(&AdminService::Service::UpdateProjectDomainAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[37], + AdminService_method_names[38], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectDomainAttributesGetRequest, ::flyteidl::admin::ProjectDomainAttributesGetResponse>( std::mem_fn(&AdminService::Service::GetProjectDomainAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[38], + AdminService_method_names[39], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectDomainAttributesDeleteRequest, ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>( std::mem_fn(&AdminService::Service::DeleteProjectDomainAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[39], + AdminService_method_names[40], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectAttributesUpdateRequest, ::flyteidl::admin::ProjectAttributesUpdateResponse>( std::mem_fn(&AdminService::Service::UpdateProjectAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[40], + AdminService_method_names[41], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectAttributesGetRequest, ::flyteidl::admin::ProjectAttributesGetResponse>( std::mem_fn(&AdminService::Service::GetProjectAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[41], + AdminService_method_names[42], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ProjectAttributesDeleteRequest, ::flyteidl::admin::ProjectAttributesDeleteResponse>( std::mem_fn(&AdminService::Service::DeleteProjectAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[42], + AdminService_method_names[43], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::WorkflowAttributesUpdateRequest, ::flyteidl::admin::WorkflowAttributesUpdateResponse>( std::mem_fn(&AdminService::Service::UpdateWorkflowAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[43], + AdminService_method_names[44], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::WorkflowAttributesGetRequest, ::flyteidl::admin::WorkflowAttributesGetResponse>( std::mem_fn(&AdminService::Service::GetWorkflowAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[44], + AdminService_method_names[45], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::WorkflowAttributesDeleteRequest, ::flyteidl::admin::WorkflowAttributesDeleteResponse>( std::mem_fn(&AdminService::Service::DeleteWorkflowAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[45], + AdminService_method_names[46], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ListMatchableAttributesRequest, ::flyteidl::admin::ListMatchableAttributesResponse>( std::mem_fn(&AdminService::Service::ListMatchableAttributes), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[46], + AdminService_method_names[47], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NamedEntityListRequest, ::flyteidl::admin::NamedEntityList>( std::mem_fn(&AdminService::Service::ListNamedEntities), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[47], + AdminService_method_names[48], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NamedEntityGetRequest, ::flyteidl::admin::NamedEntity>( std::mem_fn(&AdminService::Service::GetNamedEntity), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[48], + AdminService_method_names[49], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::NamedEntityUpdateRequest, ::flyteidl::admin::NamedEntityUpdateResponse>( std::mem_fn(&AdminService::Service::UpdateNamedEntity), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[49], + AdminService_method_names[50], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::GetVersionRequest, ::flyteidl::admin::GetVersionResponse>( std::mem_fn(&AdminService::Service::GetVersion), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[50], + AdminService_method_names[51], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::ObjectGetRequest, ::flyteidl::admin::DescriptionEntity>( std::mem_fn(&AdminService::Service::GetDescriptionEntity), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[51], + AdminService_method_names[52], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::DescriptionEntityListRequest, ::flyteidl::admin::DescriptionEntityList>( std::mem_fn(&AdminService::Service::ListDescriptionEntities), this))); AddMethod(new ::grpc::internal::RpcServiceMethod( - AdminService_method_names[52], + AdminService_method_names[53], ::grpc::internal::RpcMethod::NORMAL_RPC, new ::grpc::internal::RpcMethodHandler< AdminService::Service, ::flyteidl::admin::WorkflowExecutionGetMetricsRequest, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>( std::mem_fn(&AdminService::Service::GetExecutionMetrics), this))); @@ -2060,6 +2095,13 @@ ::grpc::Status AdminService::Service::GetNodeExecution(::grpc::ServerContext* co return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } +::grpc::Status AdminService::Service::GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) { + (void) context; + (void) request; + (void) response; + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); +} + ::grpc::Status AdminService::Service::ListNodeExecutions(::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, ::flyteidl::admin::NodeExecutionList* response) { (void) context; (void) request; diff --git a/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.h b/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.h index 830119f9ad..102f1003b4 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.h +++ b/flyteidl/gen/pb-cpp/flyteidl/service/admin.grpc.pb.h @@ -246,6 +246,14 @@ class AdminService final { std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecution>> PrepareAsyncGetNodeExecution(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) { return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecution>>(PrepareAsyncGetNodeExecutionRaw(context, request, cq)); } + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + virtual ::grpc::Status GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) = 0; + std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>> AsyncGetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>>(AsyncGetDynamicNodeWorkflowRaw(context, request, cq)); + } + std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>> PrepareAsyncGetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>>(PrepareAsyncGetDynamicNodeWorkflowRaw(context, request, cq)); + } // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. virtual ::grpc::Status ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::flyteidl::admin::NodeExecutionList* response) = 0; std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecutionList>> AsyncListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) { @@ -606,6 +614,11 @@ class AdminService final { virtual void GetNodeExecution(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecution* response, std::function) = 0; virtual void GetNodeExecution(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest* request, ::flyteidl::admin::NodeExecution* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; virtual void GetNodeExecution(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecution* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + virtual void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function) = 0; + virtual void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function) = 0; + virtual void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; + virtual void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. virtual void ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, ::flyteidl::admin::NodeExecutionList* response, std::function) = 0; virtual void ListNodeExecutions(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecutionList* response, std::function) = 0; @@ -803,6 +816,8 @@ class AdminService final { virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::ExecutionTerminateResponse>* PrepareAsyncTerminateExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::ExecutionTerminateRequest& request, ::grpc::CompletionQueue* cq) = 0; virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecution>* AsyncGetNodeExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) = 0; virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecution>* PrepareAsyncGetNodeExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) = 0; + virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>* AsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) = 0; + virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::DynamicNodeWorkflowResponse>* PrepareAsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) = 0; virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecutionList>* AsyncListNodeExecutionsRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) = 0; virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecutionList>* PrepareAsyncListNodeExecutionsRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) = 0; virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::admin::NodeExecutionList>* AsyncListNodeExecutionsForTaskRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionForTaskListRequest& request, ::grpc::CompletionQueue* cq) = 0; @@ -1033,6 +1048,13 @@ class AdminService final { std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecution>> PrepareAsyncGetNodeExecution(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) { return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecution>>(PrepareAsyncGetNodeExecutionRaw(context, request, cq)); } + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override; + std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>> AsyncGetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>>(AsyncGetDynamicNodeWorkflowRaw(context, request, cq)); + } + std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>> PrepareAsyncGetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) { + return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>>(PrepareAsyncGetDynamicNodeWorkflowRaw(context, request, cq)); + } ::grpc::Status ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::flyteidl::admin::NodeExecutionList* response) override; std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecutionList>> AsyncListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) { return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecutionList>>(AsyncListNodeExecutionsRaw(context, request, cq)); @@ -1335,6 +1357,10 @@ class AdminService final { void GetNodeExecution(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecution* response, std::function) override; void GetNodeExecution(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest* request, ::flyteidl::admin::NodeExecution* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; void GetNodeExecution(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecution* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; + void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function) override; + void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, std::function) override; + void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; + void GetDynamicNodeWorkflow(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; void ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, ::flyteidl::admin::NodeExecutionList* response, std::function) override; void ListNodeExecutions(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::admin::NodeExecutionList* response, std::function) override; void ListNodeExecutions(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, ::flyteidl::admin::NodeExecutionList* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; @@ -1510,6 +1536,8 @@ class AdminService final { ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::ExecutionTerminateResponse>* PrepareAsyncTerminateExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::ExecutionTerminateRequest& request, ::grpc::CompletionQueue* cq) override; ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecution>* AsyncGetNodeExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) override; ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecution>* PrepareAsyncGetNodeExecutionRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionGetRequest& request, ::grpc::CompletionQueue* cq) override; + ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>* AsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) override; + ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::DynamicNodeWorkflowResponse>* PrepareAsyncGetDynamicNodeWorkflowRaw(::grpc::ClientContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest& request, ::grpc::CompletionQueue* cq) override; ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecutionList>* AsyncListNodeExecutionsRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) override; ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecutionList>* PrepareAsyncListNodeExecutionsRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionListRequest& request, ::grpc::CompletionQueue* cq) override; ::grpc::ClientAsyncResponseReader< ::flyteidl::admin::NodeExecutionList>* AsyncListNodeExecutionsForTaskRaw(::grpc::ClientContext* context, const ::flyteidl::admin::NodeExecutionForTaskListRequest& request, ::grpc::CompletionQueue* cq) override; @@ -1592,6 +1620,7 @@ class AdminService final { const ::grpc::internal::RpcMethod rpcmethod_ListExecutions_; const ::grpc::internal::RpcMethod rpcmethod_TerminateExecution_; const ::grpc::internal::RpcMethod rpcmethod_GetNodeExecution_; + const ::grpc::internal::RpcMethod rpcmethod_GetDynamicNodeWorkflow_; const ::grpc::internal::RpcMethod rpcmethod_ListNodeExecutions_; const ::grpc::internal::RpcMethod rpcmethod_ListNodeExecutionsForTask_; const ::grpc::internal::RpcMethod rpcmethod_GetNodeExecutionData_; @@ -1680,6 +1709,8 @@ class AdminService final { virtual ::grpc::Status TerminateExecution(::grpc::ServerContext* context, const ::flyteidl::admin::ExecutionTerminateRequest* request, ::flyteidl::admin::ExecutionTerminateResponse* response); // Fetches a :ref:`ref_flyteidl.admin.NodeExecution`. virtual ::grpc::Status GetNodeExecution(::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionGetRequest* request, ::flyteidl::admin::NodeExecution* response); + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + virtual ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response); // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. virtual ::grpc::Status ListNodeExecutions(::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, ::flyteidl::admin::NodeExecutionList* response); // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution` launched by the reference :ref:`ref_flyteidl.admin.TaskExecution`. @@ -2221,12 +2252,32 @@ class AdminService final { } }; template + class WithAsyncMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + WithAsyncMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::MarkMethodAsync(24); + } + ~WithAsyncMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable synchronous version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + void RequestGetDynamicNodeWorkflow(::grpc::ServerContext* context, ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::DynamicNodeWorkflowResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { + ::grpc::Service::RequestAsyncUnary(24, context, request, response, new_call_cq, notification_cq, tag); + } + }; + template class WithAsyncMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListNodeExecutions() { - ::grpc::Service::MarkMethodAsync(24); + ::grpc::Service::MarkMethodAsync(25); } ~WithAsyncMethod_ListNodeExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -2237,7 +2288,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNodeExecutions(::grpc::ServerContext* context, ::flyteidl::admin::NodeExecutionListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NodeExecutionList>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(24, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(25, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2246,7 +2297,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListNodeExecutionsForTask() { - ::grpc::Service::MarkMethodAsync(25); + ::grpc::Service::MarkMethodAsync(26); } ~WithAsyncMethod_ListNodeExecutionsForTask() override { BaseClassMustBeDerivedFromService(this); @@ -2257,7 +2308,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNodeExecutionsForTask(::grpc::ServerContext* context, ::flyteidl::admin::NodeExecutionForTaskListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NodeExecutionList>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(25, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(26, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2266,7 +2317,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetNodeExecutionData() { - ::grpc::Service::MarkMethodAsync(26); + ::grpc::Service::MarkMethodAsync(27); } ~WithAsyncMethod_GetNodeExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -2277,7 +2328,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetNodeExecutionData(::grpc::ServerContext* context, ::flyteidl::admin::NodeExecutionGetDataRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NodeExecutionGetDataResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(26, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(27, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2286,7 +2337,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_RegisterProject() { - ::grpc::Service::MarkMethodAsync(27); + ::grpc::Service::MarkMethodAsync(28); } ~WithAsyncMethod_RegisterProject() override { BaseClassMustBeDerivedFromService(this); @@ -2297,7 +2348,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestRegisterProject(::grpc::ServerContext* context, ::flyteidl::admin::ProjectRegisterRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectRegisterResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(27, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(28, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2306,7 +2357,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_UpdateProject() { - ::grpc::Service::MarkMethodAsync(28); + ::grpc::Service::MarkMethodAsync(29); } ~WithAsyncMethod_UpdateProject() override { BaseClassMustBeDerivedFromService(this); @@ -2317,7 +2368,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProject(::grpc::ServerContext* context, ::flyteidl::admin::Project* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectUpdateResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(28, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(29, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2326,7 +2377,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListProjects() { - ::grpc::Service::MarkMethodAsync(29); + ::grpc::Service::MarkMethodAsync(30); } ~WithAsyncMethod_ListProjects() override { BaseClassMustBeDerivedFromService(this); @@ -2337,7 +2388,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListProjects(::grpc::ServerContext* context, ::flyteidl::admin::ProjectListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::Projects>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(29, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(30, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2346,7 +2397,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_CreateWorkflowEvent() { - ::grpc::Service::MarkMethodAsync(30); + ::grpc::Service::MarkMethodAsync(31); } ~WithAsyncMethod_CreateWorkflowEvent() override { BaseClassMustBeDerivedFromService(this); @@ -2357,7 +2408,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateWorkflowEvent(::grpc::ServerContext* context, ::flyteidl::admin::WorkflowExecutionEventRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::WorkflowExecutionEventResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(30, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(31, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2366,7 +2417,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_CreateNodeEvent() { - ::grpc::Service::MarkMethodAsync(31); + ::grpc::Service::MarkMethodAsync(32); } ~WithAsyncMethod_CreateNodeEvent() override { BaseClassMustBeDerivedFromService(this); @@ -2377,7 +2428,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateNodeEvent(::grpc::ServerContext* context, ::flyteidl::admin::NodeExecutionEventRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NodeExecutionEventResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(31, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(32, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2386,7 +2437,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_CreateTaskEvent() { - ::grpc::Service::MarkMethodAsync(32); + ::grpc::Service::MarkMethodAsync(33); } ~WithAsyncMethod_CreateTaskEvent() override { BaseClassMustBeDerivedFromService(this); @@ -2397,7 +2448,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateTaskEvent(::grpc::ServerContext* context, ::flyteidl::admin::TaskExecutionEventRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::TaskExecutionEventResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(32, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(33, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2406,7 +2457,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetTaskExecution() { - ::grpc::Service::MarkMethodAsync(33); + ::grpc::Service::MarkMethodAsync(34); } ~WithAsyncMethod_GetTaskExecution() override { BaseClassMustBeDerivedFromService(this); @@ -2417,7 +2468,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetTaskExecution(::grpc::ServerContext* context, ::flyteidl::admin::TaskExecutionGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::TaskExecution>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(33, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(34, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2426,7 +2477,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListTaskExecutions() { - ::grpc::Service::MarkMethodAsync(34); + ::grpc::Service::MarkMethodAsync(35); } ~WithAsyncMethod_ListTaskExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -2437,7 +2488,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListTaskExecutions(::grpc::ServerContext* context, ::flyteidl::admin::TaskExecutionListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::TaskExecutionList>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(34, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(35, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2446,7 +2497,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetTaskExecutionData() { - ::grpc::Service::MarkMethodAsync(35); + ::grpc::Service::MarkMethodAsync(36); } ~WithAsyncMethod_GetTaskExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -2457,7 +2508,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetTaskExecutionData(::grpc::ServerContext* context, ::flyteidl::admin::TaskExecutionGetDataRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::TaskExecutionGetDataResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(35, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(36, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2466,7 +2517,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::MarkMethodAsync(36); + ::grpc::Service::MarkMethodAsync(37); } ~WithAsyncMethod_UpdateProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2477,7 +2528,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProjectDomainAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectDomainAttributesUpdateRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(36, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(37, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2486,7 +2537,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetProjectDomainAttributes() { - ::grpc::Service::MarkMethodAsync(37); + ::grpc::Service::MarkMethodAsync(38); } ~WithAsyncMethod_GetProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2497,7 +2548,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetProjectDomainAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectDomainAttributesGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectDomainAttributesGetResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(37, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(38, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2506,7 +2557,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::MarkMethodAsync(38); + ::grpc::Service::MarkMethodAsync(39); } ~WithAsyncMethod_DeleteProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2517,7 +2568,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteProjectDomainAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectDomainAttributesDeleteRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(38, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(39, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2526,7 +2577,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_UpdateProjectAttributes() { - ::grpc::Service::MarkMethodAsync(39); + ::grpc::Service::MarkMethodAsync(40); } ~WithAsyncMethod_UpdateProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2537,7 +2588,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProjectAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectAttributesUpdateRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectAttributesUpdateResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(39, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(40, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2546,7 +2597,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetProjectAttributes() { - ::grpc::Service::MarkMethodAsync(40); + ::grpc::Service::MarkMethodAsync(41); } ~WithAsyncMethod_GetProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2557,7 +2608,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetProjectAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectAttributesGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectAttributesGetResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(40, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(41, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2566,7 +2617,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_DeleteProjectAttributes() { - ::grpc::Service::MarkMethodAsync(41); + ::grpc::Service::MarkMethodAsync(42); } ~WithAsyncMethod_DeleteProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2577,7 +2628,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteProjectAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ProjectAttributesDeleteRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ProjectAttributesDeleteResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(41, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(42, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2586,7 +2637,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_UpdateWorkflowAttributes() { - ::grpc::Service::MarkMethodAsync(42); + ::grpc::Service::MarkMethodAsync(43); } ~WithAsyncMethod_UpdateWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2597,7 +2648,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateWorkflowAttributes(::grpc::ServerContext* context, ::flyteidl::admin::WorkflowAttributesUpdateRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::WorkflowAttributesUpdateResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(42, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(43, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2606,7 +2657,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetWorkflowAttributes() { - ::grpc::Service::MarkMethodAsync(43); + ::grpc::Service::MarkMethodAsync(44); } ~WithAsyncMethod_GetWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2617,7 +2668,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetWorkflowAttributes(::grpc::ServerContext* context, ::flyteidl::admin::WorkflowAttributesGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::WorkflowAttributesGetResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(43, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(44, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2626,7 +2677,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_DeleteWorkflowAttributes() { - ::grpc::Service::MarkMethodAsync(44); + ::grpc::Service::MarkMethodAsync(45); } ~WithAsyncMethod_DeleteWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2637,7 +2688,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteWorkflowAttributes(::grpc::ServerContext* context, ::flyteidl::admin::WorkflowAttributesDeleteRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::WorkflowAttributesDeleteResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(44, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(45, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2646,7 +2697,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListMatchableAttributes() { - ::grpc::Service::MarkMethodAsync(45); + ::grpc::Service::MarkMethodAsync(46); } ~WithAsyncMethod_ListMatchableAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -2657,7 +2708,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListMatchableAttributes(::grpc::ServerContext* context, ::flyteidl::admin::ListMatchableAttributesRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::ListMatchableAttributesResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(45, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(46, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2666,7 +2717,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListNamedEntities() { - ::grpc::Service::MarkMethodAsync(46); + ::grpc::Service::MarkMethodAsync(47); } ~WithAsyncMethod_ListNamedEntities() override { BaseClassMustBeDerivedFromService(this); @@ -2677,7 +2728,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNamedEntities(::grpc::ServerContext* context, ::flyteidl::admin::NamedEntityListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NamedEntityList>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(46, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(47, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2686,7 +2737,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetNamedEntity() { - ::grpc::Service::MarkMethodAsync(47); + ::grpc::Service::MarkMethodAsync(48); } ~WithAsyncMethod_GetNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -2697,7 +2748,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetNamedEntity(::grpc::ServerContext* context, ::flyteidl::admin::NamedEntityGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NamedEntity>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(47, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(48, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2706,7 +2757,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_UpdateNamedEntity() { - ::grpc::Service::MarkMethodAsync(48); + ::grpc::Service::MarkMethodAsync(49); } ~WithAsyncMethod_UpdateNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -2717,7 +2768,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateNamedEntity(::grpc::ServerContext* context, ::flyteidl::admin::NamedEntityUpdateRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::NamedEntityUpdateResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(48, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(49, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2726,7 +2777,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetVersion() { - ::grpc::Service::MarkMethodAsync(49); + ::grpc::Service::MarkMethodAsync(50); } ~WithAsyncMethod_GetVersion() override { BaseClassMustBeDerivedFromService(this); @@ -2737,7 +2788,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetVersion(::grpc::ServerContext* context, ::flyteidl::admin::GetVersionRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::GetVersionResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(49, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(50, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2746,7 +2797,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetDescriptionEntity() { - ::grpc::Service::MarkMethodAsync(50); + ::grpc::Service::MarkMethodAsync(51); } ~WithAsyncMethod_GetDescriptionEntity() override { BaseClassMustBeDerivedFromService(this); @@ -2757,7 +2808,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetDescriptionEntity(::grpc::ServerContext* context, ::flyteidl::admin::ObjectGetRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::DescriptionEntity>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(50, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(51, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2766,7 +2817,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_ListDescriptionEntities() { - ::grpc::Service::MarkMethodAsync(51); + ::grpc::Service::MarkMethodAsync(52); } ~WithAsyncMethod_ListDescriptionEntities() override { BaseClassMustBeDerivedFromService(this); @@ -2777,7 +2828,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListDescriptionEntities(::grpc::ServerContext* context, ::flyteidl::admin::DescriptionEntityListRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::DescriptionEntityList>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(51, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(52, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -2786,7 +2837,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithAsyncMethod_GetExecutionMetrics() { - ::grpc::Service::MarkMethodAsync(52); + ::grpc::Service::MarkMethodAsync(53); } ~WithAsyncMethod_GetExecutionMetrics() override { BaseClassMustBeDerivedFromService(this); @@ -2797,10 +2848,10 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetExecutionMetrics(::grpc::ServerContext* context, ::flyteidl::admin::WorkflowExecutionGetMetricsRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(52, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(53, context, request, response, new_call_cq, notification_cq, tag); } }; - typedef WithAsyncMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > AsyncService; + typedef WithAsyncMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > AsyncService; template class ExperimentalWithCallbackMethod_CreateTask : public BaseClass { private: @@ -3546,12 +3597,43 @@ class AdminService final { virtual void GetNodeExecution(::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionGetRequest* request, ::flyteidl::admin::NodeExecution* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } }; template + class ExperimentalWithCallbackMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + ExperimentalWithCallbackMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::experimental().MarkMethodCallback(24, + new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::GetDynamicNodeWorkflowRequest, ::flyteidl::admin::DynamicNodeWorkflowResponse>( + [this](::grpc::ServerContext* context, + const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, + ::flyteidl::admin::DynamicNodeWorkflowResponse* response, + ::grpc::experimental::ServerCallbackRpcController* controller) { + return this->GetDynamicNodeWorkflow(context, request, response, controller); + })); + } + void SetMessageAllocatorFor_GetDynamicNodeWorkflow( + ::grpc::experimental::MessageAllocator< ::flyteidl::admin::GetDynamicNodeWorkflowRequest, ::flyteidl::admin::DynamicNodeWorkflowResponse>* allocator) { + static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::GetDynamicNodeWorkflowRequest, ::flyteidl::admin::DynamicNodeWorkflowResponse>*>( + ::grpc::Service::experimental().GetHandler(24)) + ->SetMessageAllocator(allocator); + } + ~ExperimentalWithCallbackMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable synchronous version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + virtual void GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } + }; + template class ExperimentalWithCallbackMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListNodeExecutions() { - ::grpc::Service::experimental().MarkMethodCallback(24, + ::grpc::Service::experimental().MarkMethodCallback(25, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionListRequest, ::flyteidl::admin::NodeExecutionList>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionListRequest* request, @@ -3563,7 +3645,7 @@ class AdminService final { void SetMessageAllocatorFor_ListNodeExecutions( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NodeExecutionListRequest, ::flyteidl::admin::NodeExecutionList>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionListRequest, ::flyteidl::admin::NodeExecutionList>*>( - ::grpc::Service::experimental().GetHandler(24)) + ::grpc::Service::experimental().GetHandler(25)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListNodeExecutions() override { @@ -3582,7 +3664,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListNodeExecutionsForTask() { - ::grpc::Service::experimental().MarkMethodCallback(25, + ::grpc::Service::experimental().MarkMethodCallback(26, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionForTaskListRequest, ::flyteidl::admin::NodeExecutionList>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionForTaskListRequest* request, @@ -3594,7 +3676,7 @@ class AdminService final { void SetMessageAllocatorFor_ListNodeExecutionsForTask( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NodeExecutionForTaskListRequest, ::flyteidl::admin::NodeExecutionList>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionForTaskListRequest, ::flyteidl::admin::NodeExecutionList>*>( - ::grpc::Service::experimental().GetHandler(25)) + ::grpc::Service::experimental().GetHandler(26)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListNodeExecutionsForTask() override { @@ -3613,7 +3695,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetNodeExecutionData() { - ::grpc::Service::experimental().MarkMethodCallback(26, + ::grpc::Service::experimental().MarkMethodCallback(27, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionGetDataRequest, ::flyteidl::admin::NodeExecutionGetDataResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionGetDataRequest* request, @@ -3625,7 +3707,7 @@ class AdminService final { void SetMessageAllocatorFor_GetNodeExecutionData( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NodeExecutionGetDataRequest, ::flyteidl::admin::NodeExecutionGetDataResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionGetDataRequest, ::flyteidl::admin::NodeExecutionGetDataResponse>*>( - ::grpc::Service::experimental().GetHandler(26)) + ::grpc::Service::experimental().GetHandler(27)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetNodeExecutionData() override { @@ -3644,7 +3726,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_RegisterProject() { - ::grpc::Service::experimental().MarkMethodCallback(27, + ::grpc::Service::experimental().MarkMethodCallback(28, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectRegisterRequest, ::flyteidl::admin::ProjectRegisterResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectRegisterRequest* request, @@ -3656,7 +3738,7 @@ class AdminService final { void SetMessageAllocatorFor_RegisterProject( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectRegisterRequest, ::flyteidl::admin::ProjectRegisterResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectRegisterRequest, ::flyteidl::admin::ProjectRegisterResponse>*>( - ::grpc::Service::experimental().GetHandler(27)) + ::grpc::Service::experimental().GetHandler(28)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_RegisterProject() override { @@ -3675,7 +3757,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_UpdateProject() { - ::grpc::Service::experimental().MarkMethodCallback(28, + ::grpc::Service::experimental().MarkMethodCallback(29, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::Project, ::flyteidl::admin::ProjectUpdateResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::Project* request, @@ -3687,7 +3769,7 @@ class AdminService final { void SetMessageAllocatorFor_UpdateProject( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::Project, ::flyteidl::admin::ProjectUpdateResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::Project, ::flyteidl::admin::ProjectUpdateResponse>*>( - ::grpc::Service::experimental().GetHandler(28)) + ::grpc::Service::experimental().GetHandler(29)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_UpdateProject() override { @@ -3706,7 +3788,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListProjects() { - ::grpc::Service::experimental().MarkMethodCallback(29, + ::grpc::Service::experimental().MarkMethodCallback(30, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectListRequest, ::flyteidl::admin::Projects>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectListRequest* request, @@ -3718,7 +3800,7 @@ class AdminService final { void SetMessageAllocatorFor_ListProjects( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectListRequest, ::flyteidl::admin::Projects>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectListRequest, ::flyteidl::admin::Projects>*>( - ::grpc::Service::experimental().GetHandler(29)) + ::grpc::Service::experimental().GetHandler(30)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListProjects() override { @@ -3737,7 +3819,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_CreateWorkflowEvent() { - ::grpc::Service::experimental().MarkMethodCallback(30, + ::grpc::Service::experimental().MarkMethodCallback(31, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowExecutionEventRequest, ::flyteidl::admin::WorkflowExecutionEventResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowExecutionEventRequest* request, @@ -3749,7 +3831,7 @@ class AdminService final { void SetMessageAllocatorFor_CreateWorkflowEvent( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::WorkflowExecutionEventRequest, ::flyteidl::admin::WorkflowExecutionEventResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowExecutionEventRequest, ::flyteidl::admin::WorkflowExecutionEventResponse>*>( - ::grpc::Service::experimental().GetHandler(30)) + ::grpc::Service::experimental().GetHandler(31)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_CreateWorkflowEvent() override { @@ -3768,7 +3850,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_CreateNodeEvent() { - ::grpc::Service::experimental().MarkMethodCallback(31, + ::grpc::Service::experimental().MarkMethodCallback(32, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionEventRequest, ::flyteidl::admin::NodeExecutionEventResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NodeExecutionEventRequest* request, @@ -3780,7 +3862,7 @@ class AdminService final { void SetMessageAllocatorFor_CreateNodeEvent( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NodeExecutionEventRequest, ::flyteidl::admin::NodeExecutionEventResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NodeExecutionEventRequest, ::flyteidl::admin::NodeExecutionEventResponse>*>( - ::grpc::Service::experimental().GetHandler(31)) + ::grpc::Service::experimental().GetHandler(32)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_CreateNodeEvent() override { @@ -3799,7 +3881,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_CreateTaskEvent() { - ::grpc::Service::experimental().MarkMethodCallback(32, + ::grpc::Service::experimental().MarkMethodCallback(33, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionEventRequest, ::flyteidl::admin::TaskExecutionEventResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::TaskExecutionEventRequest* request, @@ -3811,7 +3893,7 @@ class AdminService final { void SetMessageAllocatorFor_CreateTaskEvent( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::TaskExecutionEventRequest, ::flyteidl::admin::TaskExecutionEventResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionEventRequest, ::flyteidl::admin::TaskExecutionEventResponse>*>( - ::grpc::Service::experimental().GetHandler(32)) + ::grpc::Service::experimental().GetHandler(33)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_CreateTaskEvent() override { @@ -3830,7 +3912,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetTaskExecution() { - ::grpc::Service::experimental().MarkMethodCallback(33, + ::grpc::Service::experimental().MarkMethodCallback(34, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionGetRequest, ::flyteidl::admin::TaskExecution>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::TaskExecutionGetRequest* request, @@ -3842,7 +3924,7 @@ class AdminService final { void SetMessageAllocatorFor_GetTaskExecution( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::TaskExecutionGetRequest, ::flyteidl::admin::TaskExecution>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionGetRequest, ::flyteidl::admin::TaskExecution>*>( - ::grpc::Service::experimental().GetHandler(33)) + ::grpc::Service::experimental().GetHandler(34)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetTaskExecution() override { @@ -3861,7 +3943,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListTaskExecutions() { - ::grpc::Service::experimental().MarkMethodCallback(34, + ::grpc::Service::experimental().MarkMethodCallback(35, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionListRequest, ::flyteidl::admin::TaskExecutionList>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::TaskExecutionListRequest* request, @@ -3873,7 +3955,7 @@ class AdminService final { void SetMessageAllocatorFor_ListTaskExecutions( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::TaskExecutionListRequest, ::flyteidl::admin::TaskExecutionList>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionListRequest, ::flyteidl::admin::TaskExecutionList>*>( - ::grpc::Service::experimental().GetHandler(34)) + ::grpc::Service::experimental().GetHandler(35)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListTaskExecutions() override { @@ -3892,7 +3974,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetTaskExecutionData() { - ::grpc::Service::experimental().MarkMethodCallback(35, + ::grpc::Service::experimental().MarkMethodCallback(36, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionGetDataRequest, ::flyteidl::admin::TaskExecutionGetDataResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::TaskExecutionGetDataRequest* request, @@ -3904,7 +3986,7 @@ class AdminService final { void SetMessageAllocatorFor_GetTaskExecutionData( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::TaskExecutionGetDataRequest, ::flyteidl::admin::TaskExecutionGetDataResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::TaskExecutionGetDataRequest, ::flyteidl::admin::TaskExecutionGetDataResponse>*>( - ::grpc::Service::experimental().GetHandler(35)) + ::grpc::Service::experimental().GetHandler(36)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetTaskExecutionData() override { @@ -3923,7 +4005,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(36, + ::grpc::Service::experimental().MarkMethodCallback(37, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesUpdateRequest, ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectDomainAttributesUpdateRequest* request, @@ -3935,7 +4017,7 @@ class AdminService final { void SetMessageAllocatorFor_UpdateProjectDomainAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectDomainAttributesUpdateRequest, ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesUpdateRequest, ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>*>( - ::grpc::Service::experimental().GetHandler(36)) + ::grpc::Service::experimental().GetHandler(37)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_UpdateProjectDomainAttributes() override { @@ -3954,7 +4036,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(37, + ::grpc::Service::experimental().MarkMethodCallback(38, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesGetRequest, ::flyteidl::admin::ProjectDomainAttributesGetResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectDomainAttributesGetRequest* request, @@ -3966,7 +4048,7 @@ class AdminService final { void SetMessageAllocatorFor_GetProjectDomainAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectDomainAttributesGetRequest, ::flyteidl::admin::ProjectDomainAttributesGetResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesGetRequest, ::flyteidl::admin::ProjectDomainAttributesGetResponse>*>( - ::grpc::Service::experimental().GetHandler(37)) + ::grpc::Service::experimental().GetHandler(38)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetProjectDomainAttributes() override { @@ -3985,7 +4067,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(38, + ::grpc::Service::experimental().MarkMethodCallback(39, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesDeleteRequest, ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectDomainAttributesDeleteRequest* request, @@ -3997,7 +4079,7 @@ class AdminService final { void SetMessageAllocatorFor_DeleteProjectDomainAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectDomainAttributesDeleteRequest, ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesDeleteRequest, ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>*>( - ::grpc::Service::experimental().GetHandler(38)) + ::grpc::Service::experimental().GetHandler(39)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_DeleteProjectDomainAttributes() override { @@ -4016,7 +4098,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_UpdateProjectAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(39, + ::grpc::Service::experimental().MarkMethodCallback(40, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesUpdateRequest, ::flyteidl::admin::ProjectAttributesUpdateResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectAttributesUpdateRequest* request, @@ -4028,7 +4110,7 @@ class AdminService final { void SetMessageAllocatorFor_UpdateProjectAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectAttributesUpdateRequest, ::flyteidl::admin::ProjectAttributesUpdateResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesUpdateRequest, ::flyteidl::admin::ProjectAttributesUpdateResponse>*>( - ::grpc::Service::experimental().GetHandler(39)) + ::grpc::Service::experimental().GetHandler(40)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_UpdateProjectAttributes() override { @@ -4047,7 +4129,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetProjectAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(40, + ::grpc::Service::experimental().MarkMethodCallback(41, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesGetRequest, ::flyteidl::admin::ProjectAttributesGetResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectAttributesGetRequest* request, @@ -4059,7 +4141,7 @@ class AdminService final { void SetMessageAllocatorFor_GetProjectAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectAttributesGetRequest, ::flyteidl::admin::ProjectAttributesGetResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesGetRequest, ::flyteidl::admin::ProjectAttributesGetResponse>*>( - ::grpc::Service::experimental().GetHandler(40)) + ::grpc::Service::experimental().GetHandler(41)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetProjectAttributes() override { @@ -4078,7 +4160,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_DeleteProjectAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(41, + ::grpc::Service::experimental().MarkMethodCallback(42, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesDeleteRequest, ::flyteidl::admin::ProjectAttributesDeleteResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ProjectAttributesDeleteRequest* request, @@ -4090,7 +4172,7 @@ class AdminService final { void SetMessageAllocatorFor_DeleteProjectAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ProjectAttributesDeleteRequest, ::flyteidl::admin::ProjectAttributesDeleteResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ProjectAttributesDeleteRequest, ::flyteidl::admin::ProjectAttributesDeleteResponse>*>( - ::grpc::Service::experimental().GetHandler(41)) + ::grpc::Service::experimental().GetHandler(42)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_DeleteProjectAttributes() override { @@ -4109,7 +4191,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_UpdateWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(42, + ::grpc::Service::experimental().MarkMethodCallback(43, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesUpdateRequest, ::flyteidl::admin::WorkflowAttributesUpdateResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowAttributesUpdateRequest* request, @@ -4121,7 +4203,7 @@ class AdminService final { void SetMessageAllocatorFor_UpdateWorkflowAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::WorkflowAttributesUpdateRequest, ::flyteidl::admin::WorkflowAttributesUpdateResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesUpdateRequest, ::flyteidl::admin::WorkflowAttributesUpdateResponse>*>( - ::grpc::Service::experimental().GetHandler(42)) + ::grpc::Service::experimental().GetHandler(43)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_UpdateWorkflowAttributes() override { @@ -4140,7 +4222,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(43, + ::grpc::Service::experimental().MarkMethodCallback(44, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesGetRequest, ::flyteidl::admin::WorkflowAttributesGetResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowAttributesGetRequest* request, @@ -4152,7 +4234,7 @@ class AdminService final { void SetMessageAllocatorFor_GetWorkflowAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::WorkflowAttributesGetRequest, ::flyteidl::admin::WorkflowAttributesGetResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesGetRequest, ::flyteidl::admin::WorkflowAttributesGetResponse>*>( - ::grpc::Service::experimental().GetHandler(43)) + ::grpc::Service::experimental().GetHandler(44)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetWorkflowAttributes() override { @@ -4171,7 +4253,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_DeleteWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(44, + ::grpc::Service::experimental().MarkMethodCallback(45, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesDeleteRequest, ::flyteidl::admin::WorkflowAttributesDeleteResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowAttributesDeleteRequest* request, @@ -4183,7 +4265,7 @@ class AdminService final { void SetMessageAllocatorFor_DeleteWorkflowAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::WorkflowAttributesDeleteRequest, ::flyteidl::admin::WorkflowAttributesDeleteResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowAttributesDeleteRequest, ::flyteidl::admin::WorkflowAttributesDeleteResponse>*>( - ::grpc::Service::experimental().GetHandler(44)) + ::grpc::Service::experimental().GetHandler(45)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_DeleteWorkflowAttributes() override { @@ -4202,7 +4284,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListMatchableAttributes() { - ::grpc::Service::experimental().MarkMethodCallback(45, + ::grpc::Service::experimental().MarkMethodCallback(46, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ListMatchableAttributesRequest, ::flyteidl::admin::ListMatchableAttributesResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ListMatchableAttributesRequest* request, @@ -4214,7 +4296,7 @@ class AdminService final { void SetMessageAllocatorFor_ListMatchableAttributes( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ListMatchableAttributesRequest, ::flyteidl::admin::ListMatchableAttributesResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ListMatchableAttributesRequest, ::flyteidl::admin::ListMatchableAttributesResponse>*>( - ::grpc::Service::experimental().GetHandler(45)) + ::grpc::Service::experimental().GetHandler(46)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListMatchableAttributes() override { @@ -4233,7 +4315,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListNamedEntities() { - ::grpc::Service::experimental().MarkMethodCallback(46, + ::grpc::Service::experimental().MarkMethodCallback(47, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityListRequest, ::flyteidl::admin::NamedEntityList>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NamedEntityListRequest* request, @@ -4245,7 +4327,7 @@ class AdminService final { void SetMessageAllocatorFor_ListNamedEntities( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NamedEntityListRequest, ::flyteidl::admin::NamedEntityList>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityListRequest, ::flyteidl::admin::NamedEntityList>*>( - ::grpc::Service::experimental().GetHandler(46)) + ::grpc::Service::experimental().GetHandler(47)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListNamedEntities() override { @@ -4264,7 +4346,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetNamedEntity() { - ::grpc::Service::experimental().MarkMethodCallback(47, + ::grpc::Service::experimental().MarkMethodCallback(48, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityGetRequest, ::flyteidl::admin::NamedEntity>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NamedEntityGetRequest* request, @@ -4276,7 +4358,7 @@ class AdminService final { void SetMessageAllocatorFor_GetNamedEntity( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NamedEntityGetRequest, ::flyteidl::admin::NamedEntity>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityGetRequest, ::flyteidl::admin::NamedEntity>*>( - ::grpc::Service::experimental().GetHandler(47)) + ::grpc::Service::experimental().GetHandler(48)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetNamedEntity() override { @@ -4295,7 +4377,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_UpdateNamedEntity() { - ::grpc::Service::experimental().MarkMethodCallback(48, + ::grpc::Service::experimental().MarkMethodCallback(49, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityUpdateRequest, ::flyteidl::admin::NamedEntityUpdateResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::NamedEntityUpdateRequest* request, @@ -4307,7 +4389,7 @@ class AdminService final { void SetMessageAllocatorFor_UpdateNamedEntity( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::NamedEntityUpdateRequest, ::flyteidl::admin::NamedEntityUpdateResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::NamedEntityUpdateRequest, ::flyteidl::admin::NamedEntityUpdateResponse>*>( - ::grpc::Service::experimental().GetHandler(48)) + ::grpc::Service::experimental().GetHandler(49)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_UpdateNamedEntity() override { @@ -4326,7 +4408,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetVersion() { - ::grpc::Service::experimental().MarkMethodCallback(49, + ::grpc::Service::experimental().MarkMethodCallback(50, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::GetVersionRequest, ::flyteidl::admin::GetVersionResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::GetVersionRequest* request, @@ -4338,7 +4420,7 @@ class AdminService final { void SetMessageAllocatorFor_GetVersion( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::GetVersionRequest, ::flyteidl::admin::GetVersionResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::GetVersionRequest, ::flyteidl::admin::GetVersionResponse>*>( - ::grpc::Service::experimental().GetHandler(49)) + ::grpc::Service::experimental().GetHandler(50)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetVersion() override { @@ -4357,7 +4439,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetDescriptionEntity() { - ::grpc::Service::experimental().MarkMethodCallback(50, + ::grpc::Service::experimental().MarkMethodCallback(51, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ObjectGetRequest, ::flyteidl::admin::DescriptionEntity>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::ObjectGetRequest* request, @@ -4369,7 +4451,7 @@ class AdminService final { void SetMessageAllocatorFor_GetDescriptionEntity( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::ObjectGetRequest, ::flyteidl::admin::DescriptionEntity>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::ObjectGetRequest, ::flyteidl::admin::DescriptionEntity>*>( - ::grpc::Service::experimental().GetHandler(50)) + ::grpc::Service::experimental().GetHandler(51)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetDescriptionEntity() override { @@ -4388,7 +4470,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_ListDescriptionEntities() { - ::grpc::Service::experimental().MarkMethodCallback(51, + ::grpc::Service::experimental().MarkMethodCallback(52, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::DescriptionEntityListRequest, ::flyteidl::admin::DescriptionEntityList>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::DescriptionEntityListRequest* request, @@ -4400,7 +4482,7 @@ class AdminService final { void SetMessageAllocatorFor_ListDescriptionEntities( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::DescriptionEntityListRequest, ::flyteidl::admin::DescriptionEntityList>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::DescriptionEntityListRequest, ::flyteidl::admin::DescriptionEntityList>*>( - ::grpc::Service::experimental().GetHandler(51)) + ::grpc::Service::experimental().GetHandler(52)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_ListDescriptionEntities() override { @@ -4419,7 +4501,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithCallbackMethod_GetExecutionMetrics() { - ::grpc::Service::experimental().MarkMethodCallback(52, + ::grpc::Service::experimental().MarkMethodCallback(53, new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowExecutionGetMetricsRequest, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>( [this](::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowExecutionGetMetricsRequest* request, @@ -4431,7 +4513,7 @@ class AdminService final { void SetMessageAllocatorFor_GetExecutionMetrics( ::grpc::experimental::MessageAllocator< ::flyteidl::admin::WorkflowExecutionGetMetricsRequest, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>* allocator) { static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::admin::WorkflowExecutionGetMetricsRequest, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>*>( - ::grpc::Service::experimental().GetHandler(52)) + ::grpc::Service::experimental().GetHandler(53)) ->SetMessageAllocator(allocator); } ~ExperimentalWithCallbackMethod_GetExecutionMetrics() override { @@ -4444,7 +4526,7 @@ class AdminService final { } virtual void GetExecutionMetrics(::grpc::ServerContext* context, const ::flyteidl::admin::WorkflowExecutionGetMetricsRequest* request, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } }; - typedef ExperimentalWithCallbackMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > ExperimentalCallbackService; + typedef ExperimentalWithCallbackMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > ExperimentalCallbackService; template class WithGenericMethod_CreateTask : public BaseClass { private: @@ -4854,12 +4936,29 @@ class AdminService final { } }; template + class WithGenericMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + WithGenericMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::MarkMethodGeneric(24); + } + ~WithGenericMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable synchronous version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + }; + template class WithGenericMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListNodeExecutions() { - ::grpc::Service::MarkMethodGeneric(24); + ::grpc::Service::MarkMethodGeneric(25); } ~WithGenericMethod_ListNodeExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -4876,7 +4975,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListNodeExecutionsForTask() { - ::grpc::Service::MarkMethodGeneric(25); + ::grpc::Service::MarkMethodGeneric(26); } ~WithGenericMethod_ListNodeExecutionsForTask() override { BaseClassMustBeDerivedFromService(this); @@ -4893,7 +4992,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetNodeExecutionData() { - ::grpc::Service::MarkMethodGeneric(26); + ::grpc::Service::MarkMethodGeneric(27); } ~WithGenericMethod_GetNodeExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -4910,7 +5009,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_RegisterProject() { - ::grpc::Service::MarkMethodGeneric(27); + ::grpc::Service::MarkMethodGeneric(28); } ~WithGenericMethod_RegisterProject() override { BaseClassMustBeDerivedFromService(this); @@ -4927,7 +5026,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_UpdateProject() { - ::grpc::Service::MarkMethodGeneric(28); + ::grpc::Service::MarkMethodGeneric(29); } ~WithGenericMethod_UpdateProject() override { BaseClassMustBeDerivedFromService(this); @@ -4944,7 +5043,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListProjects() { - ::grpc::Service::MarkMethodGeneric(29); + ::grpc::Service::MarkMethodGeneric(30); } ~WithGenericMethod_ListProjects() override { BaseClassMustBeDerivedFromService(this); @@ -4961,7 +5060,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_CreateWorkflowEvent() { - ::grpc::Service::MarkMethodGeneric(30); + ::grpc::Service::MarkMethodGeneric(31); } ~WithGenericMethod_CreateWorkflowEvent() override { BaseClassMustBeDerivedFromService(this); @@ -4978,7 +5077,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_CreateNodeEvent() { - ::grpc::Service::MarkMethodGeneric(31); + ::grpc::Service::MarkMethodGeneric(32); } ~WithGenericMethod_CreateNodeEvent() override { BaseClassMustBeDerivedFromService(this); @@ -4995,7 +5094,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_CreateTaskEvent() { - ::grpc::Service::MarkMethodGeneric(32); + ::grpc::Service::MarkMethodGeneric(33); } ~WithGenericMethod_CreateTaskEvent() override { BaseClassMustBeDerivedFromService(this); @@ -5012,7 +5111,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetTaskExecution() { - ::grpc::Service::MarkMethodGeneric(33); + ::grpc::Service::MarkMethodGeneric(34); } ~WithGenericMethod_GetTaskExecution() override { BaseClassMustBeDerivedFromService(this); @@ -5029,7 +5128,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListTaskExecutions() { - ::grpc::Service::MarkMethodGeneric(34); + ::grpc::Service::MarkMethodGeneric(35); } ~WithGenericMethod_ListTaskExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -5046,7 +5145,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetTaskExecutionData() { - ::grpc::Service::MarkMethodGeneric(35); + ::grpc::Service::MarkMethodGeneric(36); } ~WithGenericMethod_GetTaskExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -5063,7 +5162,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::MarkMethodGeneric(36); + ::grpc::Service::MarkMethodGeneric(37); } ~WithGenericMethod_UpdateProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5080,7 +5179,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetProjectDomainAttributes() { - ::grpc::Service::MarkMethodGeneric(37); + ::grpc::Service::MarkMethodGeneric(38); } ~WithGenericMethod_GetProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5097,7 +5196,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::MarkMethodGeneric(38); + ::grpc::Service::MarkMethodGeneric(39); } ~WithGenericMethod_DeleteProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5114,7 +5213,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_UpdateProjectAttributes() { - ::grpc::Service::MarkMethodGeneric(39); + ::grpc::Service::MarkMethodGeneric(40); } ~WithGenericMethod_UpdateProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5131,7 +5230,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetProjectAttributes() { - ::grpc::Service::MarkMethodGeneric(40); + ::grpc::Service::MarkMethodGeneric(41); } ~WithGenericMethod_GetProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5148,7 +5247,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_DeleteProjectAttributes() { - ::grpc::Service::MarkMethodGeneric(41); + ::grpc::Service::MarkMethodGeneric(42); } ~WithGenericMethod_DeleteProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5165,7 +5264,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_UpdateWorkflowAttributes() { - ::grpc::Service::MarkMethodGeneric(42); + ::grpc::Service::MarkMethodGeneric(43); } ~WithGenericMethod_UpdateWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5182,7 +5281,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetWorkflowAttributes() { - ::grpc::Service::MarkMethodGeneric(43); + ::grpc::Service::MarkMethodGeneric(44); } ~WithGenericMethod_GetWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5199,7 +5298,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_DeleteWorkflowAttributes() { - ::grpc::Service::MarkMethodGeneric(44); + ::grpc::Service::MarkMethodGeneric(45); } ~WithGenericMethod_DeleteWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5216,7 +5315,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListMatchableAttributes() { - ::grpc::Service::MarkMethodGeneric(45); + ::grpc::Service::MarkMethodGeneric(46); } ~WithGenericMethod_ListMatchableAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -5233,7 +5332,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListNamedEntities() { - ::grpc::Service::MarkMethodGeneric(46); + ::grpc::Service::MarkMethodGeneric(47); } ~WithGenericMethod_ListNamedEntities() override { BaseClassMustBeDerivedFromService(this); @@ -5250,7 +5349,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetNamedEntity() { - ::grpc::Service::MarkMethodGeneric(47); + ::grpc::Service::MarkMethodGeneric(48); } ~WithGenericMethod_GetNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -5267,7 +5366,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_UpdateNamedEntity() { - ::grpc::Service::MarkMethodGeneric(48); + ::grpc::Service::MarkMethodGeneric(49); } ~WithGenericMethod_UpdateNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -5284,7 +5383,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetVersion() { - ::grpc::Service::MarkMethodGeneric(49); + ::grpc::Service::MarkMethodGeneric(50); } ~WithGenericMethod_GetVersion() override { BaseClassMustBeDerivedFromService(this); @@ -5301,7 +5400,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetDescriptionEntity() { - ::grpc::Service::MarkMethodGeneric(50); + ::grpc::Service::MarkMethodGeneric(51); } ~WithGenericMethod_GetDescriptionEntity() override { BaseClassMustBeDerivedFromService(this); @@ -5318,7 +5417,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_ListDescriptionEntities() { - ::grpc::Service::MarkMethodGeneric(51); + ::grpc::Service::MarkMethodGeneric(52); } ~WithGenericMethod_ListDescriptionEntities() override { BaseClassMustBeDerivedFromService(this); @@ -5335,7 +5434,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithGenericMethod_GetExecutionMetrics() { - ::grpc::Service::MarkMethodGeneric(52); + ::grpc::Service::MarkMethodGeneric(53); } ~WithGenericMethod_GetExecutionMetrics() override { BaseClassMustBeDerivedFromService(this); @@ -5827,12 +5926,32 @@ class AdminService final { } }; template + class WithRawMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + WithRawMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::MarkMethodRaw(24); + } + ~WithRawMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable synchronous version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + void RequestGetDynamicNodeWorkflow(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { + ::grpc::Service::RequestAsyncUnary(24, context, request, response, new_call_cq, notification_cq, tag); + } + }; + template class WithRawMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListNodeExecutions() { - ::grpc::Service::MarkMethodRaw(24); + ::grpc::Service::MarkMethodRaw(25); } ~WithRawMethod_ListNodeExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -5843,7 +5962,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNodeExecutions(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(24, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(25, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5852,7 +5971,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListNodeExecutionsForTask() { - ::grpc::Service::MarkMethodRaw(25); + ::grpc::Service::MarkMethodRaw(26); } ~WithRawMethod_ListNodeExecutionsForTask() override { BaseClassMustBeDerivedFromService(this); @@ -5863,7 +5982,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNodeExecutionsForTask(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(25, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(26, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5872,7 +5991,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetNodeExecutionData() { - ::grpc::Service::MarkMethodRaw(26); + ::grpc::Service::MarkMethodRaw(27); } ~WithRawMethod_GetNodeExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -5883,7 +6002,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetNodeExecutionData(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(26, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(27, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5892,7 +6011,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_RegisterProject() { - ::grpc::Service::MarkMethodRaw(27); + ::grpc::Service::MarkMethodRaw(28); } ~WithRawMethod_RegisterProject() override { BaseClassMustBeDerivedFromService(this); @@ -5903,7 +6022,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestRegisterProject(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(27, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(28, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5912,7 +6031,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_UpdateProject() { - ::grpc::Service::MarkMethodRaw(28); + ::grpc::Service::MarkMethodRaw(29); } ~WithRawMethod_UpdateProject() override { BaseClassMustBeDerivedFromService(this); @@ -5923,7 +6042,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProject(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(28, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(29, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5932,7 +6051,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListProjects() { - ::grpc::Service::MarkMethodRaw(29); + ::grpc::Service::MarkMethodRaw(30); } ~WithRawMethod_ListProjects() override { BaseClassMustBeDerivedFromService(this); @@ -5943,7 +6062,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListProjects(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(29, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(30, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5952,7 +6071,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_CreateWorkflowEvent() { - ::grpc::Service::MarkMethodRaw(30); + ::grpc::Service::MarkMethodRaw(31); } ~WithRawMethod_CreateWorkflowEvent() override { BaseClassMustBeDerivedFromService(this); @@ -5963,7 +6082,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateWorkflowEvent(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(30, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(31, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5972,7 +6091,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_CreateNodeEvent() { - ::grpc::Service::MarkMethodRaw(31); + ::grpc::Service::MarkMethodRaw(32); } ~WithRawMethod_CreateNodeEvent() override { BaseClassMustBeDerivedFromService(this); @@ -5983,7 +6102,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateNodeEvent(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(31, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(32, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -5992,7 +6111,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_CreateTaskEvent() { - ::grpc::Service::MarkMethodRaw(32); + ::grpc::Service::MarkMethodRaw(33); } ~WithRawMethod_CreateTaskEvent() override { BaseClassMustBeDerivedFromService(this); @@ -6003,7 +6122,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestCreateTaskEvent(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(32, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(33, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6012,7 +6131,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetTaskExecution() { - ::grpc::Service::MarkMethodRaw(33); + ::grpc::Service::MarkMethodRaw(34); } ~WithRawMethod_GetTaskExecution() override { BaseClassMustBeDerivedFromService(this); @@ -6023,7 +6142,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetTaskExecution(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(33, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(34, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6032,7 +6151,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListTaskExecutions() { - ::grpc::Service::MarkMethodRaw(34); + ::grpc::Service::MarkMethodRaw(35); } ~WithRawMethod_ListTaskExecutions() override { BaseClassMustBeDerivedFromService(this); @@ -6043,7 +6162,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListTaskExecutions(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(34, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(35, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6052,7 +6171,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetTaskExecutionData() { - ::grpc::Service::MarkMethodRaw(35); + ::grpc::Service::MarkMethodRaw(36); } ~WithRawMethod_GetTaskExecutionData() override { BaseClassMustBeDerivedFromService(this); @@ -6063,7 +6182,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetTaskExecutionData(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(35, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(36, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6072,7 +6191,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::MarkMethodRaw(36); + ::grpc::Service::MarkMethodRaw(37); } ~WithRawMethod_UpdateProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6083,7 +6202,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProjectDomainAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(36, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(37, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6092,7 +6211,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetProjectDomainAttributes() { - ::grpc::Service::MarkMethodRaw(37); + ::grpc::Service::MarkMethodRaw(38); } ~WithRawMethod_GetProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6103,7 +6222,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetProjectDomainAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(37, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(38, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6112,7 +6231,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::MarkMethodRaw(38); + ::grpc::Service::MarkMethodRaw(39); } ~WithRawMethod_DeleteProjectDomainAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6123,7 +6242,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteProjectDomainAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(38, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(39, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6132,7 +6251,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_UpdateProjectAttributes() { - ::grpc::Service::MarkMethodRaw(39); + ::grpc::Service::MarkMethodRaw(40); } ~WithRawMethod_UpdateProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6143,7 +6262,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateProjectAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(39, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(40, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6152,7 +6271,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetProjectAttributes() { - ::grpc::Service::MarkMethodRaw(40); + ::grpc::Service::MarkMethodRaw(41); } ~WithRawMethod_GetProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6163,7 +6282,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetProjectAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(40, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(41, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6172,7 +6291,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_DeleteProjectAttributes() { - ::grpc::Service::MarkMethodRaw(41); + ::grpc::Service::MarkMethodRaw(42); } ~WithRawMethod_DeleteProjectAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6183,7 +6302,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteProjectAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(41, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(42, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6192,7 +6311,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_UpdateWorkflowAttributes() { - ::grpc::Service::MarkMethodRaw(42); + ::grpc::Service::MarkMethodRaw(43); } ~WithRawMethod_UpdateWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6203,7 +6322,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateWorkflowAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(42, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(43, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6212,7 +6331,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetWorkflowAttributes() { - ::grpc::Service::MarkMethodRaw(43); + ::grpc::Service::MarkMethodRaw(44); } ~WithRawMethod_GetWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6223,7 +6342,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetWorkflowAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(43, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(44, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6232,7 +6351,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_DeleteWorkflowAttributes() { - ::grpc::Service::MarkMethodRaw(44); + ::grpc::Service::MarkMethodRaw(45); } ~WithRawMethod_DeleteWorkflowAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6243,7 +6362,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestDeleteWorkflowAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(44, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(45, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6252,7 +6371,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListMatchableAttributes() { - ::grpc::Service::MarkMethodRaw(45); + ::grpc::Service::MarkMethodRaw(46); } ~WithRawMethod_ListMatchableAttributes() override { BaseClassMustBeDerivedFromService(this); @@ -6263,7 +6382,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListMatchableAttributes(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(45, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(46, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6272,7 +6391,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListNamedEntities() { - ::grpc::Service::MarkMethodRaw(46); + ::grpc::Service::MarkMethodRaw(47); } ~WithRawMethod_ListNamedEntities() override { BaseClassMustBeDerivedFromService(this); @@ -6283,7 +6402,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListNamedEntities(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(46, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(47, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6292,7 +6411,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetNamedEntity() { - ::grpc::Service::MarkMethodRaw(47); + ::grpc::Service::MarkMethodRaw(48); } ~WithRawMethod_GetNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -6303,7 +6422,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetNamedEntity(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(47, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(48, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6312,7 +6431,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_UpdateNamedEntity() { - ::grpc::Service::MarkMethodRaw(48); + ::grpc::Service::MarkMethodRaw(49); } ~WithRawMethod_UpdateNamedEntity() override { BaseClassMustBeDerivedFromService(this); @@ -6323,7 +6442,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestUpdateNamedEntity(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(48, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(49, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6332,7 +6451,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetVersion() { - ::grpc::Service::MarkMethodRaw(49); + ::grpc::Service::MarkMethodRaw(50); } ~WithRawMethod_GetVersion() override { BaseClassMustBeDerivedFromService(this); @@ -6343,7 +6462,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetVersion(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(49, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(50, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6352,7 +6471,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetDescriptionEntity() { - ::grpc::Service::MarkMethodRaw(50); + ::grpc::Service::MarkMethodRaw(51); } ~WithRawMethod_GetDescriptionEntity() override { BaseClassMustBeDerivedFromService(this); @@ -6363,7 +6482,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetDescriptionEntity(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(50, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(51, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6372,7 +6491,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_ListDescriptionEntities() { - ::grpc::Service::MarkMethodRaw(51); + ::grpc::Service::MarkMethodRaw(52); } ~WithRawMethod_ListDescriptionEntities() override { BaseClassMustBeDerivedFromService(this); @@ -6383,7 +6502,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestListDescriptionEntities(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(51, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(52, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -6392,7 +6511,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithRawMethod_GetExecutionMetrics() { - ::grpc::Service::MarkMethodRaw(52); + ::grpc::Service::MarkMethodRaw(53); } ~WithRawMethod_GetExecutionMetrics() override { BaseClassMustBeDerivedFromService(this); @@ -6403,7 +6522,7 @@ class AdminService final { return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); } void RequestGetExecutionMetrics(::grpc::ServerContext* context, ::grpc::ByteBuffer* request, ::grpc::ServerAsyncResponseWriter< ::grpc::ByteBuffer>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(52, context, request, response, new_call_cq, notification_cq, tag); + ::grpc::Service::RequestAsyncUnary(53, context, request, response, new_call_cq, notification_cq, tag); } }; template @@ -7007,12 +7126,37 @@ class AdminService final { virtual void GetNodeExecution(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } }; template + class ExperimentalWithRawCallbackMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + ExperimentalWithRawCallbackMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::experimental().MarkMethodRawCallback(24, + new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( + [this](::grpc::ServerContext* context, + const ::grpc::ByteBuffer* request, + ::grpc::ByteBuffer* response, + ::grpc::experimental::ServerCallbackRpcController* controller) { + this->GetDynamicNodeWorkflow(context, request, response, controller); + })); + } + ~ExperimentalWithRawCallbackMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable synchronous version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + virtual void GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } + }; + template class ExperimentalWithRawCallbackMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListNodeExecutions() { - ::grpc::Service::experimental().MarkMethodRawCallback(24, + ::grpc::Service::experimental().MarkMethodRawCallback(25, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7037,7 +7181,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListNodeExecutionsForTask() { - ::grpc::Service::experimental().MarkMethodRawCallback(25, + ::grpc::Service::experimental().MarkMethodRawCallback(26, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7062,7 +7206,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetNodeExecutionData() { - ::grpc::Service::experimental().MarkMethodRawCallback(26, + ::grpc::Service::experimental().MarkMethodRawCallback(27, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7087,7 +7231,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_RegisterProject() { - ::grpc::Service::experimental().MarkMethodRawCallback(27, + ::grpc::Service::experimental().MarkMethodRawCallback(28, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7112,7 +7256,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_UpdateProject() { - ::grpc::Service::experimental().MarkMethodRawCallback(28, + ::grpc::Service::experimental().MarkMethodRawCallback(29, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7137,7 +7281,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListProjects() { - ::grpc::Service::experimental().MarkMethodRawCallback(29, + ::grpc::Service::experimental().MarkMethodRawCallback(30, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7162,7 +7306,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_CreateWorkflowEvent() { - ::grpc::Service::experimental().MarkMethodRawCallback(30, + ::grpc::Service::experimental().MarkMethodRawCallback(31, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7187,7 +7331,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_CreateNodeEvent() { - ::grpc::Service::experimental().MarkMethodRawCallback(31, + ::grpc::Service::experimental().MarkMethodRawCallback(32, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7212,7 +7356,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_CreateTaskEvent() { - ::grpc::Service::experimental().MarkMethodRawCallback(32, + ::grpc::Service::experimental().MarkMethodRawCallback(33, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7237,7 +7381,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetTaskExecution() { - ::grpc::Service::experimental().MarkMethodRawCallback(33, + ::grpc::Service::experimental().MarkMethodRawCallback(34, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7262,7 +7406,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListTaskExecutions() { - ::grpc::Service::experimental().MarkMethodRawCallback(34, + ::grpc::Service::experimental().MarkMethodRawCallback(35, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7287,7 +7431,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetTaskExecutionData() { - ::grpc::Service::experimental().MarkMethodRawCallback(35, + ::grpc::Service::experimental().MarkMethodRawCallback(36, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7312,7 +7456,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(36, + ::grpc::Service::experimental().MarkMethodRawCallback(37, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7337,7 +7481,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(37, + ::grpc::Service::experimental().MarkMethodRawCallback(38, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7362,7 +7506,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(38, + ::grpc::Service::experimental().MarkMethodRawCallback(39, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7387,7 +7531,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_UpdateProjectAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(39, + ::grpc::Service::experimental().MarkMethodRawCallback(40, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7412,7 +7556,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetProjectAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(40, + ::grpc::Service::experimental().MarkMethodRawCallback(41, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7437,7 +7581,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_DeleteProjectAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(41, + ::grpc::Service::experimental().MarkMethodRawCallback(42, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7462,7 +7606,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_UpdateWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(42, + ::grpc::Service::experimental().MarkMethodRawCallback(43, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7487,7 +7631,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(43, + ::grpc::Service::experimental().MarkMethodRawCallback(44, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7512,7 +7656,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_DeleteWorkflowAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(44, + ::grpc::Service::experimental().MarkMethodRawCallback(45, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7537,7 +7681,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListMatchableAttributes() { - ::grpc::Service::experimental().MarkMethodRawCallback(45, + ::grpc::Service::experimental().MarkMethodRawCallback(46, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7562,7 +7706,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListNamedEntities() { - ::grpc::Service::experimental().MarkMethodRawCallback(46, + ::grpc::Service::experimental().MarkMethodRawCallback(47, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7587,7 +7731,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetNamedEntity() { - ::grpc::Service::experimental().MarkMethodRawCallback(47, + ::grpc::Service::experimental().MarkMethodRawCallback(48, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7612,7 +7756,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_UpdateNamedEntity() { - ::grpc::Service::experimental().MarkMethodRawCallback(48, + ::grpc::Service::experimental().MarkMethodRawCallback(49, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7637,7 +7781,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetVersion() { - ::grpc::Service::experimental().MarkMethodRawCallback(49, + ::grpc::Service::experimental().MarkMethodRawCallback(50, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7662,7 +7806,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetDescriptionEntity() { - ::grpc::Service::experimental().MarkMethodRawCallback(50, + ::grpc::Service::experimental().MarkMethodRawCallback(51, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7687,7 +7831,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_ListDescriptionEntities() { - ::grpc::Service::experimental().MarkMethodRawCallback(51, + ::grpc::Service::experimental().MarkMethodRawCallback(52, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -7712,7 +7856,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: ExperimentalWithRawCallbackMethod_GetExecutionMetrics() { - ::grpc::Service::experimental().MarkMethodRawCallback(52, + ::grpc::Service::experimental().MarkMethodRawCallback(53, new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( [this](::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, @@ -8212,12 +8356,32 @@ class AdminService final { virtual ::grpc::Status StreamedGetNodeExecution(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::admin::NodeExecutionGetRequest,::flyteidl::admin::NodeExecution>* server_unary_streamer) = 0; }; template + class WithStreamedUnaryMethod_GetDynamicNodeWorkflow : public BaseClass { + private: + void BaseClassMustBeDerivedFromService(const Service *service) {} + public: + WithStreamedUnaryMethod_GetDynamicNodeWorkflow() { + ::grpc::Service::MarkMethodStreamed(24, + new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::GetDynamicNodeWorkflowRequest, ::flyteidl::admin::DynamicNodeWorkflowResponse>(std::bind(&WithStreamedUnaryMethod_GetDynamicNodeWorkflow::StreamedGetDynamicNodeWorkflow, this, std::placeholders::_1, std::placeholders::_2))); + } + ~WithStreamedUnaryMethod_GetDynamicNodeWorkflow() override { + BaseClassMustBeDerivedFromService(this); + } + // disable regular version of this method + ::grpc::Status GetDynamicNodeWorkflow(::grpc::ServerContext* context, const ::flyteidl::admin::GetDynamicNodeWorkflowRequest* request, ::flyteidl::admin::DynamicNodeWorkflowResponse* response) override { + abort(); + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + } + // replace default version of method with streamed unary + virtual ::grpc::Status StreamedGetDynamicNodeWorkflow(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::admin::GetDynamicNodeWorkflowRequest,::flyteidl::admin::DynamicNodeWorkflowResponse>* server_unary_streamer) = 0; + }; + template class WithStreamedUnaryMethod_ListNodeExecutions : public BaseClass { private: void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListNodeExecutions() { - ::grpc::Service::MarkMethodStreamed(24, + ::grpc::Service::MarkMethodStreamed(25, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NodeExecutionListRequest, ::flyteidl::admin::NodeExecutionList>(std::bind(&WithStreamedUnaryMethod_ListNodeExecutions::StreamedListNodeExecutions, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListNodeExecutions() override { @@ -8237,7 +8401,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListNodeExecutionsForTask() { - ::grpc::Service::MarkMethodStreamed(25, + ::grpc::Service::MarkMethodStreamed(26, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NodeExecutionForTaskListRequest, ::flyteidl::admin::NodeExecutionList>(std::bind(&WithStreamedUnaryMethod_ListNodeExecutionsForTask::StreamedListNodeExecutionsForTask, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListNodeExecutionsForTask() override { @@ -8257,7 +8421,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetNodeExecutionData() { - ::grpc::Service::MarkMethodStreamed(26, + ::grpc::Service::MarkMethodStreamed(27, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NodeExecutionGetDataRequest, ::flyteidl::admin::NodeExecutionGetDataResponse>(std::bind(&WithStreamedUnaryMethod_GetNodeExecutionData::StreamedGetNodeExecutionData, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetNodeExecutionData() override { @@ -8277,7 +8441,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_RegisterProject() { - ::grpc::Service::MarkMethodStreamed(27, + ::grpc::Service::MarkMethodStreamed(28, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectRegisterRequest, ::flyteidl::admin::ProjectRegisterResponse>(std::bind(&WithStreamedUnaryMethod_RegisterProject::StreamedRegisterProject, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_RegisterProject() override { @@ -8297,7 +8461,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_UpdateProject() { - ::grpc::Service::MarkMethodStreamed(28, + ::grpc::Service::MarkMethodStreamed(29, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::Project, ::flyteidl::admin::ProjectUpdateResponse>(std::bind(&WithStreamedUnaryMethod_UpdateProject::StreamedUpdateProject, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_UpdateProject() override { @@ -8317,7 +8481,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListProjects() { - ::grpc::Service::MarkMethodStreamed(29, + ::grpc::Service::MarkMethodStreamed(30, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectListRequest, ::flyteidl::admin::Projects>(std::bind(&WithStreamedUnaryMethod_ListProjects::StreamedListProjects, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListProjects() override { @@ -8337,7 +8501,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_CreateWorkflowEvent() { - ::grpc::Service::MarkMethodStreamed(30, + ::grpc::Service::MarkMethodStreamed(31, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::WorkflowExecutionEventRequest, ::flyteidl::admin::WorkflowExecutionEventResponse>(std::bind(&WithStreamedUnaryMethod_CreateWorkflowEvent::StreamedCreateWorkflowEvent, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_CreateWorkflowEvent() override { @@ -8357,7 +8521,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_CreateNodeEvent() { - ::grpc::Service::MarkMethodStreamed(31, + ::grpc::Service::MarkMethodStreamed(32, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NodeExecutionEventRequest, ::flyteidl::admin::NodeExecutionEventResponse>(std::bind(&WithStreamedUnaryMethod_CreateNodeEvent::StreamedCreateNodeEvent, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_CreateNodeEvent() override { @@ -8377,7 +8541,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_CreateTaskEvent() { - ::grpc::Service::MarkMethodStreamed(32, + ::grpc::Service::MarkMethodStreamed(33, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::TaskExecutionEventRequest, ::flyteidl::admin::TaskExecutionEventResponse>(std::bind(&WithStreamedUnaryMethod_CreateTaskEvent::StreamedCreateTaskEvent, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_CreateTaskEvent() override { @@ -8397,7 +8561,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetTaskExecution() { - ::grpc::Service::MarkMethodStreamed(33, + ::grpc::Service::MarkMethodStreamed(34, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::TaskExecutionGetRequest, ::flyteidl::admin::TaskExecution>(std::bind(&WithStreamedUnaryMethod_GetTaskExecution::StreamedGetTaskExecution, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetTaskExecution() override { @@ -8417,7 +8581,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListTaskExecutions() { - ::grpc::Service::MarkMethodStreamed(34, + ::grpc::Service::MarkMethodStreamed(35, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::TaskExecutionListRequest, ::flyteidl::admin::TaskExecutionList>(std::bind(&WithStreamedUnaryMethod_ListTaskExecutions::StreamedListTaskExecutions, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListTaskExecutions() override { @@ -8437,7 +8601,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetTaskExecutionData() { - ::grpc::Service::MarkMethodStreamed(35, + ::grpc::Service::MarkMethodStreamed(36, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::TaskExecutionGetDataRequest, ::flyteidl::admin::TaskExecutionGetDataResponse>(std::bind(&WithStreamedUnaryMethod_GetTaskExecutionData::StreamedGetTaskExecutionData, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetTaskExecutionData() override { @@ -8457,7 +8621,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_UpdateProjectDomainAttributes() { - ::grpc::Service::MarkMethodStreamed(36, + ::grpc::Service::MarkMethodStreamed(37, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesUpdateRequest, ::flyteidl::admin::ProjectDomainAttributesUpdateResponse>(std::bind(&WithStreamedUnaryMethod_UpdateProjectDomainAttributes::StreamedUpdateProjectDomainAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_UpdateProjectDomainAttributes() override { @@ -8477,7 +8641,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetProjectDomainAttributes() { - ::grpc::Service::MarkMethodStreamed(37, + ::grpc::Service::MarkMethodStreamed(38, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesGetRequest, ::flyteidl::admin::ProjectDomainAttributesGetResponse>(std::bind(&WithStreamedUnaryMethod_GetProjectDomainAttributes::StreamedGetProjectDomainAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetProjectDomainAttributes() override { @@ -8497,7 +8661,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_DeleteProjectDomainAttributes() { - ::grpc::Service::MarkMethodStreamed(38, + ::grpc::Service::MarkMethodStreamed(39, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectDomainAttributesDeleteRequest, ::flyteidl::admin::ProjectDomainAttributesDeleteResponse>(std::bind(&WithStreamedUnaryMethod_DeleteProjectDomainAttributes::StreamedDeleteProjectDomainAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_DeleteProjectDomainAttributes() override { @@ -8517,7 +8681,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_UpdateProjectAttributes() { - ::grpc::Service::MarkMethodStreamed(39, + ::grpc::Service::MarkMethodStreamed(40, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectAttributesUpdateRequest, ::flyteidl::admin::ProjectAttributesUpdateResponse>(std::bind(&WithStreamedUnaryMethod_UpdateProjectAttributes::StreamedUpdateProjectAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_UpdateProjectAttributes() override { @@ -8537,7 +8701,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetProjectAttributes() { - ::grpc::Service::MarkMethodStreamed(40, + ::grpc::Service::MarkMethodStreamed(41, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectAttributesGetRequest, ::flyteidl::admin::ProjectAttributesGetResponse>(std::bind(&WithStreamedUnaryMethod_GetProjectAttributes::StreamedGetProjectAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetProjectAttributes() override { @@ -8557,7 +8721,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_DeleteProjectAttributes() { - ::grpc::Service::MarkMethodStreamed(41, + ::grpc::Service::MarkMethodStreamed(42, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ProjectAttributesDeleteRequest, ::flyteidl::admin::ProjectAttributesDeleteResponse>(std::bind(&WithStreamedUnaryMethod_DeleteProjectAttributes::StreamedDeleteProjectAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_DeleteProjectAttributes() override { @@ -8577,7 +8741,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_UpdateWorkflowAttributes() { - ::grpc::Service::MarkMethodStreamed(42, + ::grpc::Service::MarkMethodStreamed(43, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::WorkflowAttributesUpdateRequest, ::flyteidl::admin::WorkflowAttributesUpdateResponse>(std::bind(&WithStreamedUnaryMethod_UpdateWorkflowAttributes::StreamedUpdateWorkflowAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_UpdateWorkflowAttributes() override { @@ -8597,7 +8761,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetWorkflowAttributes() { - ::grpc::Service::MarkMethodStreamed(43, + ::grpc::Service::MarkMethodStreamed(44, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::WorkflowAttributesGetRequest, ::flyteidl::admin::WorkflowAttributesGetResponse>(std::bind(&WithStreamedUnaryMethod_GetWorkflowAttributes::StreamedGetWorkflowAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetWorkflowAttributes() override { @@ -8617,7 +8781,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_DeleteWorkflowAttributes() { - ::grpc::Service::MarkMethodStreamed(44, + ::grpc::Service::MarkMethodStreamed(45, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::WorkflowAttributesDeleteRequest, ::flyteidl::admin::WorkflowAttributesDeleteResponse>(std::bind(&WithStreamedUnaryMethod_DeleteWorkflowAttributes::StreamedDeleteWorkflowAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_DeleteWorkflowAttributes() override { @@ -8637,7 +8801,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListMatchableAttributes() { - ::grpc::Service::MarkMethodStreamed(45, + ::grpc::Service::MarkMethodStreamed(46, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ListMatchableAttributesRequest, ::flyteidl::admin::ListMatchableAttributesResponse>(std::bind(&WithStreamedUnaryMethod_ListMatchableAttributes::StreamedListMatchableAttributes, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListMatchableAttributes() override { @@ -8657,7 +8821,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListNamedEntities() { - ::grpc::Service::MarkMethodStreamed(46, + ::grpc::Service::MarkMethodStreamed(47, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NamedEntityListRequest, ::flyteidl::admin::NamedEntityList>(std::bind(&WithStreamedUnaryMethod_ListNamedEntities::StreamedListNamedEntities, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListNamedEntities() override { @@ -8677,7 +8841,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetNamedEntity() { - ::grpc::Service::MarkMethodStreamed(47, + ::grpc::Service::MarkMethodStreamed(48, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NamedEntityGetRequest, ::flyteidl::admin::NamedEntity>(std::bind(&WithStreamedUnaryMethod_GetNamedEntity::StreamedGetNamedEntity, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetNamedEntity() override { @@ -8697,7 +8861,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_UpdateNamedEntity() { - ::grpc::Service::MarkMethodStreamed(48, + ::grpc::Service::MarkMethodStreamed(49, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::NamedEntityUpdateRequest, ::flyteidl::admin::NamedEntityUpdateResponse>(std::bind(&WithStreamedUnaryMethod_UpdateNamedEntity::StreamedUpdateNamedEntity, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_UpdateNamedEntity() override { @@ -8717,7 +8881,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetVersion() { - ::grpc::Service::MarkMethodStreamed(49, + ::grpc::Service::MarkMethodStreamed(50, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::GetVersionRequest, ::flyteidl::admin::GetVersionResponse>(std::bind(&WithStreamedUnaryMethod_GetVersion::StreamedGetVersion, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetVersion() override { @@ -8737,7 +8901,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetDescriptionEntity() { - ::grpc::Service::MarkMethodStreamed(50, + ::grpc::Service::MarkMethodStreamed(51, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::ObjectGetRequest, ::flyteidl::admin::DescriptionEntity>(std::bind(&WithStreamedUnaryMethod_GetDescriptionEntity::StreamedGetDescriptionEntity, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetDescriptionEntity() override { @@ -8757,7 +8921,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_ListDescriptionEntities() { - ::grpc::Service::MarkMethodStreamed(51, + ::grpc::Service::MarkMethodStreamed(52, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::DescriptionEntityListRequest, ::flyteidl::admin::DescriptionEntityList>(std::bind(&WithStreamedUnaryMethod_ListDescriptionEntities::StreamedListDescriptionEntities, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_ListDescriptionEntities() override { @@ -8777,7 +8941,7 @@ class AdminService final { void BaseClassMustBeDerivedFromService(const Service *service) {} public: WithStreamedUnaryMethod_GetExecutionMetrics() { - ::grpc::Service::MarkMethodStreamed(52, + ::grpc::Service::MarkMethodStreamed(53, new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::admin::WorkflowExecutionGetMetricsRequest, ::flyteidl::admin::WorkflowExecutionGetMetricsResponse>(std::bind(&WithStreamedUnaryMethod_GetExecutionMetrics::StreamedGetExecutionMetrics, this, std::placeholders::_1, std::placeholders::_2))); } ~WithStreamedUnaryMethod_GetExecutionMetrics() override { @@ -8791,9 +8955,9 @@ class AdminService final { // replace default version of method with streamed unary virtual ::grpc::Status StreamedGetExecutionMetrics(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::admin::WorkflowExecutionGetMetricsRequest,::flyteidl::admin::WorkflowExecutionGetMetricsResponse>* server_unary_streamer) = 0; }; - typedef WithStreamedUnaryMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > StreamedUnaryService; + typedef WithStreamedUnaryMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > StreamedUnaryService; typedef Service SplitStreamedService; - typedef WithStreamedUnaryMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > StreamedService; + typedef WithStreamedUnaryMethod_CreateTask > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > StreamedService; }; } // namespace service diff --git a/flyteidl/gen/pb-cpp/flyteidl/service/admin.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/service/admin.pb.cc index 2824f12e1c..b395530a7f 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/service/admin.pb.cc +++ b/flyteidl/gen/pb-cpp/flyteidl/service/admin.pb.cc @@ -52,7 +52,7 @@ const char descriptor_table_protodef_flyteidl_2fservice_2fadmin_2eproto[] = "admin/task_execution.proto\032\034flyteidl/adm" "in/version.proto\032\033flyteidl/admin/common." "proto\032\'flyteidl/admin/description_entity" - ".proto2\227v\n\014AdminService\022\216\001\n\nCreateTask\022!" + ".proto2\270y\n\014AdminService\022\216\001\n\nCreateTask\022!" ".flyteidl.admin.TaskCreateRequest\032\".flyt" "eidl.admin.TaskCreateResponse\"9\202\323\344\223\0023\"\r/" "api/v1/tasks:\001*Z\037\"\032/api/v1/tasks/org/{id" @@ -192,252 +192,262 @@ const char descriptor_table_protodef_flyteidl_2fservice_2fadmin_2eproto[] = "e_id}Z\213\001\022\210\001/api/v1/node_executions/org/{" "id.execution_id.org}/{id.execution_id.pr" "oject}/{id.execution_id.domain}/{id.exec" - "ution_id.name}/{id.node_id}\022\371\002\n\022ListNode" - "Executions\022(.flyteidl.admin.NodeExecutio" - "nListRequest\032!.flyteidl.admin.NodeExecut" - "ionList\"\225\002\202\323\344\223\002\216\002\022s/api/v1/node_executio" - "ns/{workflow_execution_id.project}/{work" - "flow_execution_id.domain}/{workflow_exec" - "ution_id.name}Z\226\001\022\223\001/api/v1/node_executi" - "ons/org/{workflow_execution_id.org}/{wor" - "kflow_execution_id.project}/{workflow_ex" - "ecution_id.domain}/{workflow_execution_i" - "d.name}\022\217\010\n\031ListNodeExecutionsForTask\022/." - "flyteidl.admin.NodeExecutionForTaskListR" - "equest\032!.flyteidl.admin.NodeExecutionLis" - "t\"\235\007\202\323\344\223\002\226\007\022\251\003/api/v1/children/task_exec" - "utions/{task_execution_id.node_execution" - "_id.execution_id.project}/{task_executio" - "n_id.node_execution_id.execution_id.doma" - "in}/{task_execution_id.node_execution_id" - ".execution_id.name}/{task_execution_id.n" - "ode_execution_id.node_id}/{task_executio" - "n_id.task_id.project}/{task_execution_id" - ".task_id.domain}/{task_execution_id.task" - "_id.name}/{task_execution_id.task_id.ver" - "sion}/{task_execution_id.retry_attempt}Z" - "\347\003\022\344\003/api/v1/children/task_executions/or" - "g/{task_execution_id.node_execution_id.e" - "xecution_id.org}/{task_execution_id.node" - "_execution_id.execution_id.project}/{tas" - "k_execution_id.node_execution_id.executi" - "on_id.domain}/{task_execution_id.node_ex" - "ecution_id.execution_id.name}/{task_exec" - "ution_id.node_execution_id.node_id}/{tas" - "k_execution_id.task_id.project}/{task_ex" - "ecution_id.task_id.domain}/{task_executi" - "on_id.task_id.name}/{task_execution_id.t" - "ask_id.version}/{task_execution_id.retry" - "_attempt}\022\203\003\n\024GetNodeExecutionData\022+.fly" - "teidl.admin.NodeExecutionGetDataRequest\032" - ",.flyteidl.admin.NodeExecutionGetDataRes" - "ponse\"\217\002\202\323\344\223\002\210\002\022s/api/v1/data/node_execu" - "tions/{id.execution_id.project}/{id.exec" - "ution_id.domain}/{id.execution_id.name}/" - "{id.node_id}Z\220\001\022\215\001/api/v1/data/node_exec" - "utions/org/{id.execution_id.org}/{id.exe" - "cution_id.project}/{id.execution_id.doma" - "in}/{id.execution_id.name}/{id.node_id}\022" - "\250\001\n\017RegisterProject\022&.flyteidl.admin.Pro" - "jectRegisterRequest\032\'.flyteidl.admin.Pro" - "jectRegisterResponse\"D\202\323\344\223\002>\"\020/api/v1/pr" - "ojects:\001*Z\'\"\"/api/v1/projects/org/{proje" - "ct.org}:\001*\022\227\001\n\rUpdateProject\022\027.flyteidl." - "admin.Project\032%.flyteidl.admin.ProjectUp" - "dateResponse\"F\202\323\344\223\002@\032\025/api/v1/projects/{" - "id}:\001*Z$\032\037/api/v1/projects/org/{org}/{id" - "}:\001*\022\204\001\n\014ListProjects\022\".flyteidl.admin.P" - "rojectListRequest\032\030.flyteidl.admin.Proje" - "cts\"6\202\323\344\223\0020\022\020/api/v1/projectsZ\034\022\032/api/v1" - "/projects/org/{org}\022\325\001\n\023CreateWorkflowEv" - "ent\022-.flyteidl.admin.WorkflowExecutionEv" - "entRequest\032..flyteidl.admin.WorkflowExec" - "utionEventResponse\"_\202\323\344\223\002Y\"\030/api/v1/even" - "ts/workflows:\001*Z:\"5/api/v1/events/org/{e" - "vent.execution_id.org}/workflows:\001*\022\304\001\n\017" - "CreateNodeEvent\022).flyteidl.admin.NodeExe" - "cutionEventRequest\032*.flyteidl.admin.Node" - "ExecutionEventResponse\"Z\202\323\344\223\002T\"\024/api/v1/" - "events/nodes:\001*Z9\"4/api/v1/events/org/{e" - "vent.id.execution_id.org}/nodes:\001*\022\332\001\n\017C" - "reateTaskEvent\022).flyteidl.admin.TaskExec" - "utionEventRequest\032*.flyteidl.admin.TaskE" - "xecutionEventResponse\"p\202\323\344\223\002j\"\024/api/v1/e" - "vents/tasks:\001*ZO\"J/api/v1/events/org/{ev" - "ent.parent_node_execution_id.execution_i" - "d.org}/tasks:\001*\022\313\005\n\020GetTaskExecution\022\'.f" - "lyteidl.admin.TaskExecutionGetRequest\032\035." - "flyteidl.admin.TaskExecution\"\356\004\202\323\344\223\002\347\004\022\231" - "\002/api/v1/task_executions/{id.node_execut" - "ion_id.execution_id.project}/{id.node_ex" - "ecution_id.execution_id.domain}/{id.node" - "_execution_id.execution_id.name}/{id.nod" - "e_execution_id.node_id}/{id.task_id.proj" - "ect}/{id.task_id.domain}/{id.task_id.nam" - "e}/{id.task_id.version}/{id.retry_attemp" - "t}Z\310\002\022\305\002/api/v1/task_executions/org/{id." - "node_execution_id.execution_id.org}/{id." - "node_execution_id.execution_id.project}/" - "{id.node_execution_id.execution_id.domai" - "n}/{id.node_execution_id.execution_id.na" - "me}/{id.node_execution_id.node_id}/{id.t" - "ask_id.project}/{id.task_id.domain}/{id." - "task_id.name}/{id.task_id.version}/{id.r" - "etry_attempt}\022\361\003\n\022ListTaskExecutions\022(.f" - "lyteidl.admin.TaskExecutionListRequest\032!" - ".flyteidl.admin.TaskExecutionList\"\215\003\202\323\344\223" - "\002\206\003\022\252\001/api/v1/task_executions/{node_exec" - "ution_id.execution_id.project}/{node_exe" - "cution_id.execution_id.domain}/{node_exe" - "cution_id.execution_id.name}/{node_execu" - "tion_id.node_id}Z\326\001\022\323\001/api/v1/task_execu" - "tions/org/{node_execution_id.execution_i" - "d.org}/{node_execution_id.execution_id.p" + "ution_id.name}/{id.node_id}\022\236\003\n\026GetDynam" + "icNodeWorkflow\022-.flyteidl.admin.GetDynam" + "icNodeWorkflowRequest\032+.flyteidl.admin.D" + "ynamicNodeWorkflowResponse\"\247\002\202\323\344\223\002\240\002\022\177/a" + "pi/v1/node_executions/{id.execution_id.p" + "roject}/{id.execution_id.domain}/{id.exe" + "cution_id.name}/{id.node_id}/dynamic_wor" + "kflowZ\234\001\022\231\001/api/v1/node_executions/org/{" + "id.execution_id.org}/{id.execution_id.pr" + "oject}/{id.execution_id.domain}/{id.exec" + "ution_id.name}/{id.node_id}/dynamic_work" + "flow\022\371\002\n\022ListNodeExecutions\022(.flyteidl.a" + "dmin.NodeExecutionListRequest\032!.flyteidl" + ".admin.NodeExecutionList\"\225\002\202\323\344\223\002\216\002\022s/api" + "/v1/node_executions/{workflow_execution_" + "id.project}/{workflow_execution_id.domai" + "n}/{workflow_execution_id.name}Z\226\001\022\223\001/ap" + "i/v1/node_executions/org/{workflow_execu" + "tion_id.org}/{workflow_execution_id.proj" + "ect}/{workflow_execution_id.domain}/{wor" + "kflow_execution_id.name}\022\217\010\n\031ListNodeExe" + "cutionsForTask\022/.flyteidl.admin.NodeExec" + "utionForTaskListRequest\032!.flyteidl.admin" + ".NodeExecutionList\"\235\007\202\323\344\223\002\226\007\022\251\003/api/v1/c" + "hildren/task_executions/{task_execution_" + "id.node_execution_id.execution_id.projec" + "t}/{task_execution_id.node_execution_id." + "execution_id.domain}/{task_execution_id." + "node_execution_id.execution_id.name}/{ta" + "sk_execution_id.node_execution_id.node_i" + "d}/{task_execution_id.task_id.project}/{" + "task_execution_id.task_id.domain}/{task_" + "execution_id.task_id.name}/{task_executi" + "on_id.task_id.version}/{task_execution_i" + "d.retry_attempt}Z\347\003\022\344\003/api/v1/children/t" + "ask_executions/org/{task_execution_id.no" + "de_execution_id.execution_id.org}/{task_" + "execution_id.node_execution_id.execution" + "_id.project}/{task_execution_id.node_exe" + "cution_id.execution_id.domain}/{task_exe" + "cution_id.node_execution_id.execution_id" + ".name}/{task_execution_id.node_execution" + "_id.node_id}/{task_execution_id.task_id." + "project}/{task_execution_id.task_id.doma" + "in}/{task_execution_id.task_id.name}/{ta" + "sk_execution_id.task_id.version}/{task_e" + "xecution_id.retry_attempt}\022\203\003\n\024GetNodeEx" + "ecutionData\022+.flyteidl.admin.NodeExecuti" + "onGetDataRequest\032,.flyteidl.admin.NodeEx" + "ecutionGetDataResponse\"\217\002\202\323\344\223\002\210\002\022s/api/v" + "1/data/node_executions/{id.execution_id." + "project}/{id.execution_id.domain}/{id.ex" + "ecution_id.name}/{id.node_id}Z\220\001\022\215\001/api/" + "v1/data/node_executions/org/{id.executio" + "n_id.org}/{id.execution_id.project}/{id." + "execution_id.domain}/{id.execution_id.na" + "me}/{id.node_id}\022\250\001\n\017RegisterProject\022&.f" + "lyteidl.admin.ProjectRegisterRequest\032\'.f" + "lyteidl.admin.ProjectRegisterResponse\"D\202" + "\323\344\223\002>\"\020/api/v1/projects:\001*Z\'\"\"/api/v1/pr" + "ojects/org/{project.org}:\001*\022\227\001\n\rUpdatePr" + "oject\022\027.flyteidl.admin.Project\032%.flyteid" + "l.admin.ProjectUpdateResponse\"F\202\323\344\223\002@\032\025/" + "api/v1/projects/{id}:\001*Z$\032\037/api/v1/proje" + "cts/org/{org}/{id}:\001*\022\204\001\n\014ListProjects\022\"" + ".flyteidl.admin.ProjectListRequest\032\030.fly" + "teidl.admin.Projects\"6\202\323\344\223\0020\022\020/api/v1/pr" + "ojectsZ\034\022\032/api/v1/projects/org/{org}\022\325\001\n" + "\023CreateWorkflowEvent\022-.flyteidl.admin.Wo" + "rkflowExecutionEventRequest\032..flyteidl.a" + "dmin.WorkflowExecutionEventResponse\"_\202\323\344" + "\223\002Y\"\030/api/v1/events/workflows:\001*Z:\"5/api" + "/v1/events/org/{event.execution_id.org}/" + "workflows:\001*\022\304\001\n\017CreateNodeEvent\022).flyte" + "idl.admin.NodeExecutionEventRequest\032*.fl" + "yteidl.admin.NodeExecutionEventResponse\"" + "Z\202\323\344\223\002T\"\024/api/v1/events/nodes:\001*Z9\"4/api" + "/v1/events/org/{event.id.execution_id.or" + "g}/nodes:\001*\022\332\001\n\017CreateTaskEvent\022).flytei" + "dl.admin.TaskExecutionEventRequest\032*.fly" + "teidl.admin.TaskExecutionEventResponse\"p" + "\202\323\344\223\002j\"\024/api/v1/events/tasks:\001*ZO\"J/api/" + "v1/events/org/{event.parent_node_executi" + "on_id.execution_id.org}/tasks:\001*\022\313\005\n\020Get" + "TaskExecution\022\'.flyteidl.admin.TaskExecu" + "tionGetRequest\032\035.flyteidl.admin.TaskExec" + "ution\"\356\004\202\323\344\223\002\347\004\022\231\002/api/v1/task_execution" + "s/{id.node_execution_id.execution_id.pro" + "ject}/{id.node_execution_id.execution_id" + ".domain}/{id.node_execution_id.execution" + "_id.name}/{id.node_execution_id.node_id}" + "/{id.task_id.project}/{id.task_id.domain" + "}/{id.task_id.name}/{id.task_id.version}" + "/{id.retry_attempt}Z\310\002\022\305\002/api/v1/task_ex" + "ecutions/org/{id.node_execution_id.execu" + "tion_id.org}/{id.node_execution_id.execu" + "tion_id.project}/{id.node_execution_id.e" + "xecution_id.domain}/{id.node_execution_i" + "d.execution_id.name}/{id.node_execution_" + "id.node_id}/{id.task_id.project}/{id.tas" + "k_id.domain}/{id.task_id.name}/{id.task_" + "id.version}/{id.retry_attempt}\022\361\003\n\022ListT" + "askExecutions\022(.flyteidl.admin.TaskExecu" + "tionListRequest\032!.flyteidl.admin.TaskExe" + "cutionList\"\215\003\202\323\344\223\002\206\003\022\252\001/api/v1/task_exec" + "utions/{node_execution_id.execution_id.p" "roject}/{node_execution_id.execution_id." "domain}/{node_execution_id.execution_id." - "name}/{node_execution_id.node_id}\022\354\005\n\024Ge" - "tTaskExecutionData\022+.flyteidl.admin.Task" - "ExecutionGetDataRequest\032,.flyteidl.admin" - ".TaskExecutionGetDataResponse\"\370\004\202\323\344\223\002\361\004\022" - "\236\002/api/v1/data/task_executions/{id.node_" - "execution_id.execution_id.project}/{id.n" - "ode_execution_id.execution_id.domain}/{i" - "d.node_execution_id.execution_id.name}/{" - "id.node_execution_id.node_id}/{id.task_i" - "d.project}/{id.task_id.domain}/{id.task_" - "id.name}/{id.task_id.version}/{id.retry_" - "attempt}Z\315\002\022\312\002/api/v1/data/task_executio" - "ns/org/{id.node_execution_id.execution_i" - "d.org}/{id.node_execution_id.execution_i" - "d.project}/{id.node_execution_id.executi" - "on_id.domain}/{id.node_execution_id.exec" - "ution_id.name}/{id.node_execution_id.nod" - "e_id}/{id.task_id.project}/{id.task_id.d" - "omain}/{id.task_id.name}/{id.task_id.ver" - "sion}/{id.retry_attempt}\022\313\002\n\035UpdateProje" - "ctDomainAttributes\0224.flyteidl.admin.Proj" - "ectDomainAttributesUpdateRequest\0325.flyte" - "idl.admin.ProjectDomainAttributesUpdateR" - "esponse\"\274\001\202\323\344\223\002\265\001\032J/api/v1/project_domai" - "n_attributes/{attributes.project}/{attri" - "butes.domain}:\001*Zd\032_/api/v1/project_doma" - "in_attributes/org/{attributes.org}/{attr" - "ibutes.project}/{attributes.domain}:\001*\022\203" - "\002\n\032GetProjectDomainAttributes\0221.flyteidl" - ".admin.ProjectDomainAttributesGetRequest" - "\0322.flyteidl.admin.ProjectDomainAttribute" - "sGetResponse\"~\202\323\344\223\002x\0224/api/v1/project_do" - "main_attributes/{project}/{domain}Z@\022>/a" - "pi/v1/project_domain_attributes/org/{org" - "}/{project}/{domain}\022\223\002\n\035DeleteProjectDo" - "mainAttributes\0224.flyteidl.admin.ProjectD" - "omainAttributesDeleteRequest\0325.flyteidl." - "admin.ProjectDomainAttributesDeleteRespo" - "nse\"\204\001\202\323\344\223\002~*4/api/v1/project_domain_att" - "ributes/{project}/{domain}:\001*ZC*>/api/v1" - "/project_domain_attributes/org/{org}/{pr" - "oject}/{domain}:\001*\022\212\002\n\027UpdateProjectAttr" - "ibutes\022..flyteidl.admin.ProjectAttribute" - "sUpdateRequest\032/.flyteidl.admin.ProjectA" - "ttributesUpdateResponse\"\215\001\202\323\344\223\002\206\001\032//api/" - "v1/project_attributes/{attributes.projec" - "t}:\001*ZP\032K/api/v1/project_domain_attribut" - "es/org/{attributes.org}/{attributes.proj" - "ect}:\001*\022\330\001\n\024GetProjectAttributes\022+.flyte" - "idl.admin.ProjectAttributesGetRequest\032,." - "flyteidl.admin.ProjectAttributesGetRespo" - "nse\"e\202\323\344\223\002_\022$/api/v1/project_attributes/" - "{project}Z7\0225/api/v1/project_domain_attr" - "ibutes/org/{org}/{project}\022\347\001\n\027DeletePro" - "jectAttributes\022..flyteidl.admin.ProjectA" - "ttributesDeleteRequest\032/.flyteidl.admin." - "ProjectAttributesDeleteResponse\"k\202\323\344\223\002e*" - "$/api/v1/project_attributes/{project}:\001*" - "Z:*5/api/v1/project_domain_attributes/or" - "g/{org}/{project}:\001*\022\334\002\n\030UpdateWorkflowA" - "ttributes\022/.flyteidl.admin.WorkflowAttri" - "butesUpdateRequest\0320.flyteidl.admin.Work" - "flowAttributesUpdateResponse\"\334\001\202\323\344\223\002\325\001\032Z" - "/api/v1/workflow_attributes/{attributes." - "project}/{attributes.domain}/{attributes" - ".workflow}:\001*Zt\032o/api/v1/workflow_attrib" - "utes/org/{attributes.org}/{attributes.pr" - "oject}/{attributes.domain}/{attributes.w" - "orkflow}:\001*\022\200\002\n\025GetWorkflowAttributes\022,." - "flyteidl.admin.WorkflowAttributesGetRequ" - "est\032-.flyteidl.admin.WorkflowAttributesG" - "etResponse\"\211\001\202\323\344\223\002\202\001\0229/api/v1/workflow_a" - "ttributes/{project}/{domain}/{workflow}Z" - "E\022C/api/v1/workflow_attributes/org/{org}" - "/{project}/{domain}/{workflow}\022\217\002\n\030Delet" - "eWorkflowAttributes\022/.flyteidl.admin.Wor" - "kflowAttributesDeleteRequest\0320.flyteidl." - "admin.WorkflowAttributesDeleteResponse\"\217" - "\001\202\323\344\223\002\210\001*9/api/v1/workflow_attributes/{p" - "roject}/{domain}/{workflow}:\001*ZH*C/api/v" - "1/workflow_attributes/org/{org}/{project" - "}/{domain}/{workflow}:\001*\022\312\001\n\027ListMatchab" - "leAttributes\022..flyteidl.admin.ListMatcha" - "bleAttributesRequest\032/.flyteidl.admin.Li" - "stMatchableAttributesResponse\"N\202\323\344\223\002H\022\034/" - "api/v1/matchable_attributesZ(\022&/api/v1/m" - "atchable_attributes/org/{org}\022\350\001\n\021ListNa" - "medEntities\022&.flyteidl.admin.NamedEntity" - "ListRequest\032\037.flyteidl.admin.NamedEntity" - "List\"\211\001\202\323\344\223\002\202\001\0229/api/v1/named_entities/{" - "resource_type}/{project}/{domain}ZE\022C/ap" - "i/v1/named_entities/{resource_type}/org/" - "{org}/{project}/{domain}\022\203\002\n\016GetNamedEnt" - "ity\022%.flyteidl.admin.NamedEntityGetReque" - "st\032\033.flyteidl.admin.NamedEntity\"\254\001\202\323\344\223\002\245" - "\001\022I/api/v1/named_entities/{resource_type" - "}/{id.project}/{id.domain}/{id.name}ZX\022V" - "/api/v1/named_entities/{resource_type}/o" - "rg/{id.org}/{id.project}/{id.domain}/{id" - ".name}\022\235\002\n\021UpdateNamedEntity\022(.flyteidl." - "admin.NamedEntityUpdateRequest\032).flyteid" - "l.admin.NamedEntityUpdateResponse\"\262\001\202\323\344\223" - "\002\253\001\032I/api/v1/named_entities/{resource_ty" - "pe}/{id.project}/{id.domain}/{id.name}:\001" - "*Z[\032V/api/v1/named_entities/{resource_ty" - "pe}/org/{id.org}/{id.project}/{id.domain" - "}/{id.name}:\001*\022l\n\nGetVersion\022!.flyteidl." - "admin.GetVersionRequest\032\".flyteidl.admin" - ".GetVersionResponse\"\027\202\323\344\223\002\021\022\017/api/v1/ver" - "sion\022\266\002\n\024GetDescriptionEntity\022 .flyteidl" - ".admin.ObjectGetRequest\032!.flyteidl.admin" - ".DescriptionEntity\"\330\001\202\323\344\223\002\321\001\022_/api/v1/de" - "scription_entities/{id.resource_type}/{i" - "d.project}/{id.domain}/{id.name}/{id.ver" - "sion}Zn\022l/api/v1/description_entities/or" - "g/{id.org}/{id.resource_type}/{id.projec" - "t}/{id.domain}/{id.name}/{id.version}\022\310\003" - "\n\027ListDescriptionEntities\022,.flyteidl.adm" - "in.DescriptionEntityListRequest\032%.flytei" - "dl.admin.DescriptionEntityList\"\327\002\202\323\344\223\002\320\002" - "\022O/api/v1/description_entities/{resource" + "name}/{node_execution_id.node_id}Z\326\001\022\323\001/" + "api/v1/task_executions/org/{node_executi" + "on_id.execution_id.org}/{node_execution_" + "id.execution_id.project}/{node_execution" + "_id.execution_id.domain}/{node_execution" + "_id.execution_id.name}/{node_execution_i" + "d.node_id}\022\354\005\n\024GetTaskExecutionData\022+.fl" + "yteidl.admin.TaskExecutionGetDataRequest" + "\032,.flyteidl.admin.TaskExecutionGetDataRe" + "sponse\"\370\004\202\323\344\223\002\361\004\022\236\002/api/v1/data/task_exe" + "cutions/{id.node_execution_id.execution_" + "id.project}/{id.node_execution_id.execut" + "ion_id.domain}/{id.node_execution_id.exe" + "cution_id.name}/{id.node_execution_id.no" + "de_id}/{id.task_id.project}/{id.task_id." + "domain}/{id.task_id.name}/{id.task_id.ve" + "rsion}/{id.retry_attempt}Z\315\002\022\312\002/api/v1/d" + "ata/task_executions/org/{id.node_executi" + "on_id.execution_id.org}/{id.node_executi" + "on_id.execution_id.project}/{id.node_exe" + "cution_id.execution_id.domain}/{id.node_" + "execution_id.execution_id.name}/{id.node" + "_execution_id.node_id}/{id.task_id.proje" + "ct}/{id.task_id.domain}/{id.task_id.name" + "}/{id.task_id.version}/{id.retry_attempt" + "}\022\313\002\n\035UpdateProjectDomainAttributes\0224.fl" + "yteidl.admin.ProjectDomainAttributesUpda" + "teRequest\0325.flyteidl.admin.ProjectDomain" + "AttributesUpdateResponse\"\274\001\202\323\344\223\002\265\001\032J/api" + "/v1/project_domain_attributes/{attribute" + "s.project}/{attributes.domain}:\001*Zd\032_/ap" + "i/v1/project_domain_attributes/org/{attr" + "ibutes.org}/{attributes.project}/{attrib" + "utes.domain}:\001*\022\203\002\n\032GetProjectDomainAttr" + "ibutes\0221.flyteidl.admin.ProjectDomainAtt" + "ributesGetRequest\0322.flyteidl.admin.Proje" + "ctDomainAttributesGetResponse\"~\202\323\344\223\002x\0224/" + "api/v1/project_domain_attributes/{projec" + "t}/{domain}Z@\022>/api/v1/project_domain_at" + "tributes/org/{org}/{project}/{domain}\022\223\002" + "\n\035DeleteProjectDomainAttributes\0224.flytei" + "dl.admin.ProjectDomainAttributesDeleteRe" + "quest\0325.flyteidl.admin.ProjectDomainAttr" + "ibutesDeleteResponse\"\204\001\202\323\344\223\002~*4/api/v1/p" + "roject_domain_attributes/{project}/{doma" + "in}:\001*ZC*>/api/v1/project_domain_attribu" + "tes/org/{org}/{project}/{domain}:\001*\022\212\002\n\027" + "UpdateProjectAttributes\022..flyteidl.admin" + ".ProjectAttributesUpdateRequest\032/.flytei" + "dl.admin.ProjectAttributesUpdateResponse" + "\"\215\001\202\323\344\223\002\206\001\032//api/v1/project_attributes/{" + "attributes.project}:\001*ZP\032K/api/v1/projec" + "t_domain_attributes/org/{attributes.org}" + "/{attributes.project}:\001*\022\330\001\n\024GetProjectA" + "ttributes\022+.flyteidl.admin.ProjectAttrib" + "utesGetRequest\032,.flyteidl.admin.ProjectA" + "ttributesGetResponse\"e\202\323\344\223\002_\022$/api/v1/pr" + "oject_attributes/{project}Z7\0225/api/v1/pr" + "oject_domain_attributes/org/{org}/{proje" + "ct}\022\347\001\n\027DeleteProjectAttributes\022..flytei" + "dl.admin.ProjectAttributesDeleteRequest\032" + "/.flyteidl.admin.ProjectAttributesDelete" + "Response\"k\202\323\344\223\002e*$/api/v1/project_attrib" + "utes/{project}:\001*Z:*5/api/v1/project_dom" + "ain_attributes/org/{org}/{project}:\001*\022\334\002" + "\n\030UpdateWorkflowAttributes\022/.flyteidl.ad" + "min.WorkflowAttributesUpdateRequest\0320.fl" + "yteidl.admin.WorkflowAttributesUpdateRes" + "ponse\"\334\001\202\323\344\223\002\325\001\032Z/api/v1/workflow_attrib" + "utes/{attributes.project}/{attributes.do" + "main}/{attributes.workflow}:\001*Zt\032o/api/v" + "1/workflow_attributes/org/{attributes.or" + "g}/{attributes.project}/{attributes.doma" + "in}/{attributes.workflow}:\001*\022\200\002\n\025GetWork" + "flowAttributes\022,.flyteidl.admin.Workflow" + "AttributesGetRequest\032-.flyteidl.admin.Wo" + "rkflowAttributesGetResponse\"\211\001\202\323\344\223\002\202\001\0229/" + "api/v1/workflow_attributes/{project}/{do" + "main}/{workflow}ZE\022C/api/v1/workflow_att" + "ributes/org/{org}/{project}/{domain}/{wo" + "rkflow}\022\217\002\n\030DeleteWorkflowAttributes\022/.f" + "lyteidl.admin.WorkflowAttributesDeleteRe" + "quest\0320.flyteidl.admin.WorkflowAttribute" + "sDeleteResponse\"\217\001\202\323\344\223\002\210\001*9/api/v1/workf" + "low_attributes/{project}/{domain}/{workf" + "low}:\001*ZH*C/api/v1/workflow_attributes/o" + "rg/{org}/{project}/{domain}/{workflow}:\001" + "*\022\312\001\n\027ListMatchableAttributes\022..flyteidl" + ".admin.ListMatchableAttributesRequest\032/." + "flyteidl.admin.ListMatchableAttributesRe" + "sponse\"N\202\323\344\223\002H\022\034/api/v1/matchable_attrib" + "utesZ(\022&/api/v1/matchable_attributes/org" + "/{org}\022\350\001\n\021ListNamedEntities\022&.flyteidl." + "admin.NamedEntityListRequest\032\037.flyteidl." + "admin.NamedEntityList\"\211\001\202\323\344\223\002\202\001\0229/api/v1" + "/named_entities/{resource_type}/{project" + "}/{domain}ZE\022C/api/v1/named_entities/{re" + "source_type}/org/{org}/{project}/{domain" + "}\022\203\002\n\016GetNamedEntity\022%.flyteidl.admin.Na" + "medEntityGetRequest\032\033.flyteidl.admin.Nam" + "edEntity\"\254\001\202\323\344\223\002\245\001\022I/api/v1/named_entiti" + "es/{resource_type}/{id.project}/{id.doma" + "in}/{id.name}ZX\022V/api/v1/named_entities/" + "{resource_type}/org/{id.org}/{id.project" + "}/{id.domain}/{id.name}\022\235\002\n\021UpdateNamedE" + "ntity\022(.flyteidl.admin.NamedEntityUpdate" + "Request\032).flyteidl.admin.NamedEntityUpda" + "teResponse\"\262\001\202\323\344\223\002\253\001\032I/api/v1/named_enti" + "ties/{resource_type}/{id.project}/{id.do" + "main}/{id.name}:\001*Z[\032V/api/v1/named_enti" + "ties/{resource_type}/org/{id.org}/{id.pr" + "oject}/{id.domain}/{id.name}:\001*\022l\n\nGetVe" + "rsion\022!.flyteidl.admin.GetVersionRequest" + "\032\".flyteidl.admin.GetVersionResponse\"\027\202\323" + "\344\223\002\021\022\017/api/v1/version\022\266\002\n\024GetDescription" + "Entity\022 .flyteidl.admin.ObjectGetRequest" + "\032!.flyteidl.admin.DescriptionEntity\"\330\001\202\323" + "\344\223\002\321\001\022_/api/v1/description_entities/{id." + "resource_type}/{id.project}/{id.domain}/" + "{id.name}/{id.version}Zn\022l/api/v1/descri" + "ption_entities/org/{id.org}/{id.resource" "_type}/{id.project}/{id.domain}/{id.name" - "}Z^\022\\/api/v1/description_entities/{resou" - "rce_type}/org/{id.org}/{id.project}/{id." - "domain}/{id.name}ZG\022E/api/v1/description" - "_entities/{resource_type}/{id.project}/{" - "id.domain}ZT\022R/api/v1/description_entiti" - "es/{resource_type}/org/{id.org}/{id.proj" - "ect}/{id.domain}\022\225\002\n\023GetExecutionMetrics" - "\0222.flyteidl.admin.WorkflowExecutionGetMe" - "tricsRequest\0323.flyteidl.admin.WorkflowEx" - "ecutionGetMetricsResponse\"\224\001\202\323\344\223\002\215\001\022=/ap" - "i/v1/metrics/executions/{id.project}/{id" - ".domain}/{id.name}ZL\022J/api/v1/metrics/ex" - "ecutions/org/{id.org}/{id.project}/{id.d" - "omain}/{id.name}B\?Z=github.com/flyteorg/" - "flyte/flyteidl/gen/pb-go/flyteidl/servic" - "eb\006proto3" + "}/{id.version}\022\310\003\n\027ListDescriptionEntiti" + "es\022,.flyteidl.admin.DescriptionEntityLis" + "tRequest\032%.flyteidl.admin.DescriptionEnt" + "ityList\"\327\002\202\323\344\223\002\320\002\022O/api/v1/description_e" + "ntities/{resource_type}/{id.project}/{id" + ".domain}/{id.name}Z^\022\\/api/v1/descriptio" + "n_entities/{resource_type}/org/{id.org}/" + "{id.project}/{id.domain}/{id.name}ZG\022E/a" + "pi/v1/description_entities/{resource_typ" + "e}/{id.project}/{id.domain}ZT\022R/api/v1/d" + "escription_entities/{resource_type}/org/" + "{id.org}/{id.project}/{id.domain}\022\225\002\n\023Ge" + "tExecutionMetrics\0222.flyteidl.admin.Workf" + "lowExecutionGetMetricsRequest\0323.flyteidl" + ".admin.WorkflowExecutionGetMetricsRespon" + "se\"\224\001\202\323\344\223\002\215\001\022=/api/v1/metrics/executions" + "/{id.project}/{id.domain}/{id.name}ZL\022J/" + "api/v1/metrics/executions/org/{id.org}/{" + "id.project}/{id.domain}/{id.name}B\?Z=git" + "hub.com/flyteorg/flyte/flyteidl/gen/pb-g" + "o/flyteidl/serviceb\006proto3" ; ::google::protobuf::internal::DescriptorTable descriptor_table_flyteidl_2fservice_2fadmin_2eproto = { false, InitDefaults_flyteidl_2fservice_2fadmin_2eproto, descriptor_table_protodef_flyteidl_2fservice_2fadmin_2eproto, - "flyteidl/service/admin.proto", &assign_descriptors_table_flyteidl_2fservice_2fadmin_2eproto, 15809, + "flyteidl/service/admin.proto", &assign_descriptors_table_flyteidl_2fservice_2fadmin_2eproto, 16226, }; void AddDescriptors_flyteidl_2fservice_2fadmin_2eproto() { diff --git a/flyteidl/gen/pb-go/flyteidl/admin/node_execution.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/node_execution.pb.go index f1e7505a38..0e19a4af16 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/node_execution.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/node_execution.pb.go @@ -950,6 +950,84 @@ func (m *NodeExecutionGetDataResponse) GetFlyteUrls() *FlyteURLs { return nil } +type GetDynamicNodeWorkflowRequest struct { + Id *core.NodeExecutionIdentifier `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetDynamicNodeWorkflowRequest) Reset() { *m = GetDynamicNodeWorkflowRequest{} } +func (m *GetDynamicNodeWorkflowRequest) String() string { return proto.CompactTextString(m) } +func (*GetDynamicNodeWorkflowRequest) ProtoMessage() {} +func (*GetDynamicNodeWorkflowRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_f73b3eae493fd736, []int{12} +} + +func (m *GetDynamicNodeWorkflowRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetDynamicNodeWorkflowRequest.Unmarshal(m, b) +} +func (m *GetDynamicNodeWorkflowRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetDynamicNodeWorkflowRequest.Marshal(b, m, deterministic) +} +func (m *GetDynamicNodeWorkflowRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetDynamicNodeWorkflowRequest.Merge(m, src) +} +func (m *GetDynamicNodeWorkflowRequest) XXX_Size() int { + return xxx_messageInfo_GetDynamicNodeWorkflowRequest.Size(m) +} +func (m *GetDynamicNodeWorkflowRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetDynamicNodeWorkflowRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetDynamicNodeWorkflowRequest proto.InternalMessageInfo + +func (m *GetDynamicNodeWorkflowRequest) GetId() *core.NodeExecutionIdentifier { + if m != nil { + return m.Id + } + return nil +} + +type DynamicNodeWorkflowResponse struct { + CompiledWorkflow *core.CompiledWorkflowClosure `protobuf:"bytes,1,opt,name=compiled_workflow,json=compiledWorkflow,proto3" json:"compiled_workflow,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DynamicNodeWorkflowResponse) Reset() { *m = DynamicNodeWorkflowResponse{} } +func (m *DynamicNodeWorkflowResponse) String() string { return proto.CompactTextString(m) } +func (*DynamicNodeWorkflowResponse) ProtoMessage() {} +func (*DynamicNodeWorkflowResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_f73b3eae493fd736, []int{13} +} + +func (m *DynamicNodeWorkflowResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DynamicNodeWorkflowResponse.Unmarshal(m, b) +} +func (m *DynamicNodeWorkflowResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DynamicNodeWorkflowResponse.Marshal(b, m, deterministic) +} +func (m *DynamicNodeWorkflowResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DynamicNodeWorkflowResponse.Merge(m, src) +} +func (m *DynamicNodeWorkflowResponse) XXX_Size() int { + return xxx_messageInfo_DynamicNodeWorkflowResponse.Size(m) +} +func (m *DynamicNodeWorkflowResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DynamicNodeWorkflowResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DynamicNodeWorkflowResponse proto.InternalMessageInfo + +func (m *DynamicNodeWorkflowResponse) GetCompiledWorkflow() *core.CompiledWorkflowClosure { + if m != nil { + return m.CompiledWorkflow + } + return nil +} + func init() { proto.RegisterType((*NodeExecutionGetRequest)(nil), "flyteidl.admin.NodeExecutionGetRequest") proto.RegisterType((*NodeExecutionListRequest)(nil), "flyteidl.admin.NodeExecutionListRequest") @@ -963,6 +1041,8 @@ func init() { proto.RegisterType((*DynamicWorkflowNodeMetadata)(nil), "flyteidl.admin.DynamicWorkflowNodeMetadata") proto.RegisterType((*NodeExecutionGetDataRequest)(nil), "flyteidl.admin.NodeExecutionGetDataRequest") proto.RegisterType((*NodeExecutionGetDataResponse)(nil), "flyteidl.admin.NodeExecutionGetDataResponse") + proto.RegisterType((*GetDynamicNodeWorkflowRequest)(nil), "flyteidl.admin.GetDynamicNodeWorkflowRequest") + proto.RegisterType((*DynamicNodeWorkflowResponse)(nil), "flyteidl.admin.DynamicNodeWorkflowResponse") } func init() { @@ -970,81 +1050,83 @@ func init() { } var fileDescriptor_f73b3eae493fd736 = []byte{ - // 1208 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x57, 0x6d, 0x6f, 0x1b, 0xc5, - 0x13, 0xef, 0x25, 0x4d, 0x62, 0x8f, 0x13, 0x27, 0xd9, 0xbf, 0xfb, 0xaf, 0xdb, 0xf4, 0xc1, 0x5c, - 0x5b, 0x14, 0x40, 0xb5, 0x25, 0x57, 0x45, 0xe5, 0x99, 0x38, 0xe9, 0x43, 0x20, 0x85, 0xb2, 0xa9, - 0x41, 0x42, 0x88, 0xd3, 0xfa, 0x6e, 0xed, 0x2c, 0x3e, 0xdf, 0x5e, 0x77, 0xf7, 0x14, 0xfc, 0x45, - 0x90, 0xf8, 0x2a, 0xbc, 0xe2, 0x13, 0xf0, 0x35, 0x90, 0xf8, 0x00, 0xbc, 0x46, 0xbb, 0xb7, 0x77, - 0xf6, 0x5d, 0xdc, 0x44, 0x2a, 0x2f, 0x78, 0xe7, 0x9d, 0xf9, 0xcd, 0xcc, 0xce, 0xcc, 0x6f, 0x66, - 0xcf, 0x70, 0x67, 0x18, 0x4e, 0x15, 0x65, 0x41, 0xd8, 0x21, 0xc1, 0x84, 0x45, 0x9d, 0x88, 0x07, - 0xd4, 0xa3, 0x3f, 0x53, 0x3f, 0x51, 0x8c, 0x47, 0xed, 0x58, 0x70, 0xc5, 0x51, 0x3d, 0x03, 0xb5, - 0x0d, 0xe8, 0xfa, 0x4e, 0xc9, 0xc8, 0xe7, 0x93, 0x49, 0x06, 0xbe, 0x7e, 0x33, 0x57, 0xfa, 0x5c, - 0xd0, 0x4e, 0xc9, 0xd7, 0x9c, 0xad, 0x51, 0xfb, 0x44, 0x91, 0x90, 0x8f, 0xac, 0xf2, 0x46, 0x49, - 0xc9, 0x27, 0x31, 0x0b, 0xa9, 0xb0, 0xda, 0x5b, 0x45, 0x2d, 0x0b, 0x68, 0xa4, 0xd8, 0x90, 0xe5, - 0xfa, 0x92, 0x75, 0xc8, 0x14, 0x15, 0x24, 0x94, 0x56, 0x7b, 0x7b, 0xc4, 0xf9, 0x28, 0xa4, 0x1d, - 0x73, 0x1a, 0x24, 0xc3, 0x8e, 0x62, 0x13, 0x2a, 0x15, 0x99, 0xc4, 0x99, 0xfb, 0x32, 0x20, 0x48, - 0x04, 0x99, 0xdd, 0xdc, 0xfd, 0x06, 0xae, 0x7e, 0xc5, 0x03, 0xfa, 0x38, 0x4b, 0xe8, 0x29, 0x55, - 0x98, 0xbe, 0x4a, 0xa8, 0x54, 0xe8, 0x7d, 0x58, 0x62, 0x41, 0xd3, 0x69, 0x39, 0xbb, 0xb5, 0xee, - 0xdb, 0xed, 0xbc, 0x5a, 0xfa, 0x1a, 0xed, 0x82, 0xcd, 0x61, 0x7e, 0x67, 0xbc, 0xc4, 0x02, 0xf7, - 0xd7, 0x25, 0x68, 0x16, 0xf4, 0x47, 0x4c, 0xe6, 0x4e, 0x7f, 0x84, 0x2b, 0xa7, 0x5c, 0x8c, 0x87, - 0x21, 0x3f, 0x9d, 0x75, 0xc4, 0xcb, 0xe3, 0xbc, 0x5b, 0x8a, 0xf3, 0x9d, 0xc5, 0x2e, 0x8a, 0xf5, - 0xbf, 0xd3, 0xb3, 0x4a, 0xd4, 0x80, 0x95, 0x90, 0x4d, 0x98, 0x6a, 0x2e, 0xb5, 0x9c, 0xdd, 0x0d, - 0x9c, 0x1e, 0xb4, 0x54, 0xf1, 0x31, 0x8d, 0x9a, 0xcb, 0x2d, 0x67, 0xb7, 0x8a, 0xd3, 0x03, 0x6a, - 0xc2, 0xda, 0x90, 0x85, 0x8a, 0x0a, 0xd9, 0xbc, 0x6c, 0xe4, 0xd9, 0x11, 0xdd, 0x87, 0x35, 0xc9, - 0x85, 0xf2, 0x06, 0xd3, 0xe6, 0x8a, 0xb9, 0x57, 0xa3, 0x5d, 0x64, 0x4b, 0xfb, 0x98, 0x0b, 0x85, - 0x57, 0x35, 0xa8, 0x37, 0x45, 0xbb, 0xb0, 0x95, 0x44, 0xec, 0x55, 0x42, 0xbd, 0x98, 0x08, 0x1a, - 0x29, 0x9d, 0xcf, 0xaa, 0xf1, 0x58, 0x4f, 0xe5, 0x2f, 0x8c, 0xf8, 0x30, 0x70, 0xff, 0x72, 0xe0, - 0x76, 0xa1, 0x36, 0x4f, 0xb8, 0x78, 0x49, 0xe4, 0x78, 0xbe, 0x44, 0x18, 0xb6, 0x15, 0x91, 0xe3, - 0x45, 0xe5, 0x29, 0xb7, 0x41, 0x9b, 0x2e, 0x2a, 0xcd, 0xa6, 0x2a, 0x2a, 0xfe, 0x93, 0xb2, 0xb8, - 0x7f, 0x3a, 0xb0, 0x51, 0x48, 0xf6, 0x4d, 0x29, 0x85, 0x76, 0xa0, 0xca, 0xa2, 0x38, 0x51, 0x5e, - 0x22, 0x98, 0x49, 0xa1, 0x8a, 0x2b, 0x46, 0xd0, 0x17, 0x0c, 0x7d, 0x0a, 0x6b, 0x7e, 0xc8, 0x65, - 0x22, 0xa8, 0xc9, 0xa3, 0xd6, 0xbd, 0x5b, 0xbe, 0x55, 0xc1, 0xf5, 0x7e, 0x8a, 0xc5, 0x99, 0x11, - 0xda, 0x83, 0xca, 0x84, 0x2a, 0x12, 0x10, 0x45, 0x4c, 0xc2, 0xb5, 0xee, 0xbd, 0x73, 0x1d, 0x3c, - 0xa7, 0x8a, 0x1c, 0x10, 0x45, 0x70, 0x6e, 0xe6, 0xfe, 0xe6, 0xc0, 0x95, 0x85, 0x18, 0x74, 0x1b, - 0x6a, 0x82, 0x2a, 0x31, 0xf5, 0x46, 0x82, 0x27, 0xb1, 0x49, 0xbd, 0x8a, 0xc1, 0x88, 0x9e, 0x6a, - 0x09, 0xba, 0x0b, 0x75, 0x26, 0x33, 0xde, 0xe8, 0x45, 0x65, 0xf2, 0xab, 0xe0, 0x75, 0x26, 0x53, - 0xd6, 0x68, 0xbf, 0xa8, 0x05, 0xeb, 0x32, 0xa6, 0xbe, 0x01, 0x68, 0x3a, 0xa4, 0x0d, 0x03, 0x2d, - 0xd3, 0xfa, 0xc3, 0x00, 0xdd, 0x04, 0x60, 0xd2, 0x0b, 0xa6, 0x11, 0x99, 0x30, 0xdf, 0xe4, 0x51, - 0xc1, 0x55, 0x26, 0x0f, 0x52, 0x01, 0xba, 0x06, 0x15, 0x26, 0x3d, 0x22, 0x04, 0x49, 0x7b, 0x57, - 0xc1, 0x6b, 0x4c, 0xee, 0xe9, 0xa3, 0xfb, 0x0a, 0xb6, 0xcf, 0x8c, 0x2b, 0x7a, 0x02, 0x9b, 0xc5, - 0xad, 0x29, 0x9b, 0x4e, 0x6b, 0x79, 0xb7, 0xd6, 0xbd, 0x79, 0x6e, 0x6d, 0x70, 0x3d, 0x9a, 0x3f, - 0xca, 0x19, 0xc5, 0x96, 0xe6, 0x28, 0xe6, 0xfe, 0xbd, 0x02, 0x8d, 0x45, 0x4d, 0x41, 0x77, 0x00, - 0x78, 0xa2, 0xb2, 0x4e, 0x9b, 0x6a, 0xf5, 0x96, 0x9a, 0xce, 0xb3, 0x4b, 0xb8, 0x9a, 0xca, 0x75, - 0xc3, 0x1f, 0xc2, 0x0a, 0x15, 0x82, 0x0b, 0xe3, 0xb3, 0x70, 0x23, 0x43, 0xa4, 0xdc, 0xe9, 0x63, - 0x0d, 0x7a, 0x76, 0x09, 0xa7, 0x68, 0xf4, 0x39, 0xd4, 0xac, 0x6f, 0xd3, 0x6a, 0x30, 0xc6, 0xd7, - 0x4a, 0xc6, 0x47, 0xe9, 0x7e, 0x7d, 0x4e, 0x62, 0x1b, 0xd7, 0xde, 0xc7, 0x34, 0xf3, 0x11, 0xac, - 0xc4, 0x27, 0x44, 0xa6, 0x3c, 0xab, 0x77, 0xdd, 0xf3, 0x18, 0xdc, 0x7e, 0xa1, 0x91, 0x38, 0x35, - 0x40, 0x1f, 0x00, 0x48, 0x45, 0x84, 0xa2, 0x81, 0x47, 0x94, 0x65, 0xd9, 0xf5, 0x76, 0xba, 0x9b, - 0xdb, 0xd9, 0x6e, 0x6e, 0xbf, 0xcc, 0x96, 0x37, 0xae, 0x5a, 0xf4, 0x9e, 0x42, 0x0f, 0xa1, 0x92, - 0xed, 0x6c, 0x3b, 0x75, 0xd7, 0xce, 0x18, 0x1e, 0x58, 0x00, 0xce, 0xa1, 0x3a, 0xa2, 0x2f, 0x28, - 0xb1, 0x11, 0x57, 0x2f, 0x8e, 0x68, 0xd1, 0x7b, 0x4a, 0x9b, 0x26, 0x71, 0x90, 0x99, 0xae, 0x5d, - 0x6c, 0x6a, 0xd1, 0x7b, 0x0a, 0xfd, 0x00, 0xff, 0xcf, 0xd7, 0xbb, 0xe1, 0x4f, 0x3e, 0x59, 0x95, - 0xc5, 0xa3, 0x99, 0x2d, 0x78, 0x5d, 0xbb, 0xe7, 0x16, 0xfb, 0xcc, 0xc1, 0x8d, 0xd3, 0x05, 0x72, - 0xf4, 0x02, 0x90, 0xd9, 0x8c, 0x45, 0xcf, 0x55, 0xe3, 0xb9, 0x55, 0xf6, 0xac, 0x77, 0x63, 0xc9, - 0xeb, 0x96, 0x2a, 0xc9, 0xf4, 0x58, 0x04, 0xd4, 0x1f, 0x1b, 0xb6, 0xd5, 0xd2, 0x65, 0xa7, 0xcf, - 0x9a, 0x65, 0x1d, 0x68, 0xd8, 0x69, 0xf2, 0x7e, 0xe2, 0x03, 0xcf, 0x8c, 0x9f, 0x86, 0xad, 0x1b, - 0xd8, 0xb6, 0xd5, 0x7d, 0xc1, 0x07, 0xc7, 0x31, 0xf5, 0xfb, 0x82, 0xf5, 0x36, 0x61, 0xc3, 0xf2, - 0x4b, 0x50, 0x99, 0x84, 0xaa, 0xb7, 0x0d, 0x9b, 0x8a, 0x88, 0x11, 0x55, 0xf9, 0x5d, 0xdd, 0x00, - 0x1a, 0x8b, 0x32, 0x46, 0x47, 0x50, 0xa3, 0xb3, 0xdd, 0xf7, 0x06, 0x8f, 0xe1, 0xbc, 0xb9, 0xfb, - 0xbb, 0x03, 0x5b, 0xe5, 0xf4, 0xd1, 0x01, 0xac, 0xfb, 0xc4, 0x3f, 0xa1, 0x9e, 0x54, 0x44, 0x25, - 0xd2, 0xc4, 0xa8, 0x77, 0xdf, 0x2a, 0xc5, 0xd8, 0x4f, 0x3f, 0x5d, 0xf6, 0x35, 0xf2, 0xd8, 0x00, - 0x71, 0xcd, 0x9f, 0x1d, 0xd0, 0x67, 0x50, 0xb3, 0x5f, 0x37, 0xde, 0x98, 0x4e, 0xed, 0x04, 0xde, - 0x5a, 0xec, 0x24, 0x0b, 0x8d, 0xc1, 0x9a, 0x7c, 0x49, 0xa7, 0xe8, 0x1e, 0xd4, 0xfd, 0x13, 0xea, - 0x8f, 0x63, 0xce, 0xa2, 0x74, 0xca, 0xd3, 0x47, 0x66, 0x63, 0x26, 0xed, 0x0b, 0xe6, 0xfe, 0xe1, - 0xc0, 0x8e, 0xdd, 0x5d, 0x0b, 0x0b, 0xf6, 0xce, 0xdc, 0x4b, 0x52, 0x9e, 0xe1, 0xd2, 0xe3, 0x71, - 0x0c, 0xdb, 0xf6, 0x9b, 0x2b, 0xf0, 0x32, 0x5a, 0xd9, 0x8b, 0x97, 0xdf, 0xa0, 0x7d, 0x8b, 0xcb, - 0x42, 0x66, 0x6f, 0xc5, 0x96, 0x5f, 0x52, 0xbc, 0x96, 0x1d, 0xcb, 0xaf, 0x61, 0x87, 0xdb, 0x87, - 0x9d, 0xf2, 0x87, 0x96, 0x79, 0x44, 0xfe, 0xe5, 0xc7, 0xd6, 0x2f, 0xcb, 0x70, 0x63, 0xb1, 0x5f, - 0x19, 0xf3, 0x48, 0x52, 0xf4, 0x00, 0x56, 0xcd, 0x4b, 0x29, 0xad, 0xf3, 0xab, 0xe5, 0x39, 0xe9, - 0x8b, 0xb0, 0x17, 0xf2, 0x81, 0x5e, 0x77, 0xd8, 0x42, 0xd1, 0x43, 0x58, 0x4b, 0xa9, 0x2c, 0x6d, - 0xa1, 0xce, 0xb5, 0xca, 0xb0, 0xe8, 0x43, 0xa8, 0x0d, 0x93, 0x30, 0xf4, 0x6c, 0xc0, 0xe5, 0x0b, - 0x36, 0x2c, 0x06, 0x8d, 0x3e, 0x4c, 0x43, 0x7e, 0x0c, 0xeb, 0xc6, 0x36, 0x8b, 0x7b, 0xf9, 0x22, - 0x63, 0x13, 0xea, 0x6b, 0x1b, 0xf9, 0x5b, 0xd8, 0xca, 0xda, 0x91, 0xb7, 0x78, 0xcb, 0x78, 0x78, - 0xaf, 0x7c, 0xf3, 0x73, 0x58, 0x85, 0x37, 0x83, 0xa2, 0x12, 0x3d, 0x02, 0x30, 0xe6, 0x5e, 0x22, - 0x42, 0xd9, 0xdc, 0x2e, 0xdf, 0x29, 0xf5, 0xf8, 0x44, 0x1f, 0xfb, 0xf8, 0x48, 0xe2, 0xaa, 0xd1, - 0xf4, 0x45, 0x28, 0x7b, 0x9f, 0x7c, 0xff, 0xd1, 0x88, 0xa9, 0x93, 0x64, 0xd0, 0xf6, 0xf9, 0xa4, - 0x63, 0xe4, 0x5c, 0x8c, 0xd2, 0x1f, 0x9d, 0xfc, 0x9b, 0x7e, 0x44, 0xa3, 0x4e, 0x3c, 0xb8, 0x3f, - 0xe2, 0x9d, 0xe2, 0xbf, 0x8f, 0xc1, 0xaa, 0xd9, 0xb3, 0x0f, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, - 0xe3, 0xde, 0x8f, 0xda, 0xcb, 0x0c, 0x00, 0x00, + // 1236 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x57, 0xdf, 0x6e, 0xdb, 0xb6, + 0x17, 0xae, 0x92, 0x26, 0xb1, 0x8f, 0x13, 0x27, 0xe1, 0x2f, 0xfd, 0xd5, 0x6d, 0x9a, 0x36, 0x53, + 0xdb, 0x21, 0xdb, 0x50, 0x1b, 0x48, 0xd1, 0xa1, 0xfb, 0xbf, 0x38, 0xe9, 0x9f, 0x6c, 0xe9, 0xd6, + 0x31, 0xf5, 0x0a, 0x0c, 0xc3, 0x04, 0x5a, 0xa2, 0x1d, 0xce, 0xb2, 0xa8, 0x92, 0x14, 0x32, 0xbf, + 0xc8, 0x80, 0xbd, 0xca, 0xae, 0xf6, 0x04, 0x7b, 0x8d, 0x01, 0x7b, 0x80, 0x5d, 0x0f, 0xa4, 0x28, + 0xd9, 0x52, 0xd5, 0x04, 0x68, 0x2e, 0x76, 0x67, 0x9e, 0xf3, 0x9d, 0xef, 0x90, 0x3c, 0xdf, 0x39, + 0x94, 0xe1, 0xf6, 0x20, 0x9c, 0x28, 0xca, 0x82, 0xb0, 0x43, 0x82, 0x31, 0x8b, 0x3a, 0x11, 0x0f, + 0xa8, 0x47, 0x7f, 0xa1, 0x7e, 0xa2, 0x18, 0x8f, 0xda, 0xb1, 0xe0, 0x8a, 0xa3, 0x66, 0x06, 0x6a, + 0x1b, 0xd0, 0xf5, 0xcd, 0x52, 0x90, 0xcf, 0xc7, 0xe3, 0x0c, 0x7c, 0x7d, 0x2b, 0x77, 0xfa, 0x5c, + 0xd0, 0x4e, 0x89, 0x6b, 0x26, 0xd6, 0xb8, 0x7d, 0xa2, 0x48, 0xc8, 0x87, 0xd6, 0x79, 0xa3, 0xe4, + 0xe4, 0xe3, 0x98, 0x85, 0x54, 0x58, 0xef, 0xcd, 0xa2, 0x97, 0x05, 0x34, 0x52, 0x6c, 0xc0, 0x72, + 0x7f, 0x29, 0x3a, 0x64, 0x8a, 0x0a, 0x12, 0x4a, 0xeb, 0xbd, 0x35, 0xe4, 0x7c, 0x18, 0xd2, 0x8e, + 0x59, 0xf5, 0x93, 0x41, 0x47, 0xb1, 0x31, 0x95, 0x8a, 0x8c, 0xe3, 0x8c, 0xbe, 0x0c, 0x08, 0x12, + 0x41, 0xa6, 0x3b, 0x77, 0xbf, 0x83, 0xab, 0xdf, 0xf0, 0x80, 0x3e, 0xca, 0x0e, 0xf4, 0x84, 0x2a, + 0x4c, 0x5f, 0x25, 0x54, 0x2a, 0xf4, 0x21, 0xcc, 0xb1, 0xa0, 0xe5, 0x6c, 0x3b, 0x3b, 0x8d, 0xdd, + 0x77, 0xdb, 0xf9, 0x6d, 0xe9, 0x6d, 0xb4, 0x0b, 0x31, 0x87, 0xf9, 0x9e, 0xf1, 0x1c, 0x0b, 0xdc, + 0xdf, 0xe6, 0xa0, 0x55, 0xf0, 0x1f, 0x31, 0x99, 0x93, 0xfe, 0x04, 0x57, 0x4e, 0xb9, 0x18, 0x0d, + 0x42, 0x7e, 0x3a, 0xad, 0x88, 0x97, 0xe7, 0x79, 0xbf, 0x94, 0xe7, 0xa5, 0xc5, 0x56, 0xe5, 0xfa, + 0xdf, 0xe9, 0xeb, 0x4e, 0xb4, 0x01, 0x0b, 0x21, 0x1b, 0x33, 0xd5, 0x9a, 0xdb, 0x76, 0x76, 0x56, + 0x70, 0xba, 0xd0, 0x56, 0xc5, 0x47, 0x34, 0x6a, 0xcd, 0x6f, 0x3b, 0x3b, 0x75, 0x9c, 0x2e, 0x50, + 0x0b, 0x96, 0x06, 0x2c, 0x54, 0x54, 0xc8, 0xd6, 0x65, 0x63, 0xcf, 0x96, 0xe8, 0x1e, 0x2c, 0x49, + 0x2e, 0x94, 0xd7, 0x9f, 0xb4, 0x16, 0xcc, 0xbe, 0x36, 0xda, 0x45, 0xb5, 0xb4, 0x8f, 0xb9, 0x50, + 0x78, 0x51, 0x83, 0xba, 0x13, 0xb4, 0x03, 0x6b, 0x49, 0xc4, 0x5e, 0x25, 0xd4, 0x8b, 0x89, 0xa0, + 0x91, 0xd2, 0xe7, 0x59, 0x34, 0x8c, 0xcd, 0xd4, 0xfe, 0xdc, 0x98, 0x0f, 0x03, 0xf7, 0x6f, 0x07, + 0x6e, 0x15, 0xee, 0xe6, 0x31, 0x17, 0x2f, 0x88, 0x1c, 0xcd, 0x5e, 0x11, 0x86, 0x75, 0x45, 0xe4, + 0xa8, 0xea, 0x7a, 0xca, 0x65, 0xd0, 0xa1, 0x55, 0x57, 0xb3, 0xaa, 0x8a, 0x8e, 0xff, 0xe4, 0x5a, + 0xdc, 0xbf, 0x1c, 0x58, 0x29, 0x1c, 0xf6, 0x6d, 0x25, 0x85, 0x36, 0xa1, 0xce, 0xa2, 0x38, 0x51, + 0x5e, 0x22, 0x98, 0x39, 0x42, 0x1d, 0xd7, 0x8c, 0xa1, 0x27, 0x18, 0xfa, 0x1c, 0x96, 0xfc, 0x90, + 0xcb, 0x44, 0x50, 0x73, 0x8e, 0xc6, 0xee, 0x9d, 0xf2, 0xae, 0x0a, 0xd4, 0xfb, 0x29, 0x16, 0x67, + 0x41, 0x68, 0x0f, 0x6a, 0x63, 0xaa, 0x48, 0x40, 0x14, 0x31, 0x07, 0x6e, 0xec, 0xde, 0x3d, 0x93, + 0xe0, 0x19, 0x55, 0xe4, 0x80, 0x28, 0x82, 0xf3, 0x30, 0xf7, 0x77, 0x07, 0xae, 0x54, 0x62, 0xd0, + 0x2d, 0x68, 0x08, 0xaa, 0xc4, 0xc4, 0x1b, 0x0a, 0x9e, 0xc4, 0xe6, 0xe8, 0x75, 0x0c, 0xc6, 0xf4, + 0x44, 0x5b, 0xd0, 0x1d, 0x68, 0x32, 0x99, 0xe9, 0x46, 0x0f, 0x2a, 0x73, 0xbe, 0x1a, 0x5e, 0x66, + 0x32, 0x55, 0x8d, 0xe6, 0x45, 0xdb, 0xb0, 0x2c, 0x63, 0xea, 0x1b, 0x80, 0x96, 0x43, 0x5a, 0x30, + 0xd0, 0x36, 0xed, 0x3f, 0x0c, 0xd0, 0x16, 0x00, 0x93, 0x5e, 0x30, 0x89, 0xc8, 0x98, 0xf9, 0xe6, + 0x1c, 0x35, 0x5c, 0x67, 0xf2, 0x20, 0x35, 0xa0, 0x6b, 0x50, 0x63, 0xd2, 0x23, 0x42, 0x90, 0xb4, + 0x76, 0x35, 0xbc, 0xc4, 0xe4, 0x9e, 0x5e, 0xba, 0xaf, 0x60, 0xfd, 0xb5, 0x76, 0x45, 0x8f, 0x61, + 0xb5, 0x38, 0x35, 0x65, 0xcb, 0xd9, 0x9e, 0xdf, 0x69, 0xec, 0x6e, 0x9d, 0x79, 0x37, 0xb8, 0x19, + 0xcd, 0x2e, 0xe5, 0x54, 0x62, 0x73, 0x33, 0x12, 0x73, 0xff, 0x59, 0x80, 0x8d, 0xaa, 0xa2, 0xa0, + 0xdb, 0x00, 0x3c, 0x51, 0x59, 0xa5, 0xcd, 0x6d, 0x75, 0xe7, 0x5a, 0xce, 0xd3, 0x4b, 0xb8, 0x9e, + 0xda, 0x75, 0xc1, 0x1f, 0xc0, 0x02, 0x15, 0x82, 0x0b, 0xc3, 0x59, 0xd8, 0x91, 0x11, 0x52, 0x4e, + 0xfa, 0x48, 0x83, 0x9e, 0x5e, 0xc2, 0x29, 0x1a, 0x7d, 0x09, 0x0d, 0xcb, 0x6d, 0x4a, 0x0d, 0x26, + 0xf8, 0x5a, 0x29, 0xf8, 0x28, 0x9d, 0xaf, 0xcf, 0x48, 0x6c, 0xf3, 0xda, 0xfd, 0x98, 0x62, 0x3e, + 0x84, 0x85, 0xf8, 0x84, 0xc8, 0x54, 0x67, 0xcd, 0x5d, 0xf7, 0x2c, 0x05, 0xb7, 0x9f, 0x6b, 0x24, + 0x4e, 0x03, 0xd0, 0x47, 0x00, 0x52, 0x11, 0xa1, 0x68, 0xe0, 0x11, 0x65, 0x55, 0x76, 0xbd, 0x9d, + 0xce, 0xe6, 0x76, 0x36, 0x9b, 0xdb, 0x2f, 0xb2, 0xe1, 0x8d, 0xeb, 0x16, 0xbd, 0xa7, 0xd0, 0x03, + 0xa8, 0x65, 0x33, 0xdb, 0x76, 0xdd, 0xb5, 0xd7, 0x02, 0x0f, 0x2c, 0x00, 0xe7, 0x50, 0x9d, 0xd1, + 0x17, 0x94, 0xd8, 0x8c, 0x8b, 0xe7, 0x67, 0xb4, 0xe8, 0x3d, 0xa5, 0x43, 0x93, 0x38, 0xc8, 0x42, + 0x97, 0xce, 0x0f, 0xb5, 0xe8, 0x3d, 0x85, 0x7e, 0x84, 0xff, 0xe7, 0xe3, 0xdd, 0xe8, 0x27, 0xef, + 0xac, 0x5a, 0x75, 0x6b, 0x66, 0x03, 0x5e, 0xdf, 0xdd, 0x33, 0x8b, 0x7d, 0xea, 0xe0, 0x8d, 0xd3, + 0x0a, 0x3b, 0x7a, 0x0e, 0xc8, 0x4c, 0xc6, 0x22, 0x73, 0xdd, 0x30, 0x6f, 0x97, 0x99, 0xf5, 0x6c, + 0x2c, 0xb1, 0xae, 0xa9, 0x92, 0x4d, 0xb7, 0x45, 0x40, 0xfd, 0x91, 0x51, 0x5b, 0x23, 0x1d, 0x76, + 0x7a, 0xad, 0x55, 0xd6, 0x81, 0x0d, 0xdb, 0x4d, 0xde, 0xcf, 0xbc, 0xef, 0x99, 0xf6, 0xd3, 0xb0, + 0x65, 0x03, 0x5b, 0xb7, 0xbe, 0xaf, 0x78, 0xff, 0x38, 0xa6, 0x7e, 0x4f, 0xb0, 0xee, 0x2a, 0xac, + 0x58, 0x7d, 0x09, 0x2a, 0x93, 0x50, 0x75, 0xd7, 0x61, 0x55, 0x11, 0x31, 0xa4, 0x2a, 0xdf, 0xab, + 0x1b, 0xc0, 0x46, 0xd5, 0x89, 0xd1, 0x11, 0x34, 0xe8, 0x74, 0xf6, 0xbd, 0xc5, 0x63, 0x38, 0x1b, + 0xee, 0xfe, 0xe1, 0xc0, 0x5a, 0xf9, 0xf8, 0xe8, 0x00, 0x96, 0x7d, 0xe2, 0x9f, 0x50, 0x4f, 0x2a, + 0xa2, 0x12, 0x69, 0x72, 0x34, 0x77, 0xdf, 0x29, 0xe5, 0xd8, 0x4f, 0x3f, 0x5d, 0xf6, 0x35, 0xf2, + 0xd8, 0x00, 0x71, 0xc3, 0x9f, 0x2e, 0xd0, 0x17, 0xd0, 0xb0, 0x5f, 0x37, 0xde, 0x88, 0x4e, 0x6c, + 0x07, 0xde, 0xac, 0x26, 0xc9, 0x52, 0x63, 0xb0, 0x21, 0x5f, 0xd3, 0x09, 0xba, 0x0b, 0x4d, 0xff, + 0x84, 0xfa, 0xa3, 0x98, 0xb3, 0x28, 0xed, 0xf2, 0xf4, 0x91, 0x59, 0x99, 0x5a, 0x7b, 0x82, 0xb9, + 0x7f, 0x3a, 0xb0, 0x69, 0x67, 0x57, 0xe5, 0x85, 0xbd, 0x37, 0xf3, 0x92, 0x94, 0x7b, 0xb8, 0xf4, + 0x78, 0x1c, 0xc3, 0xba, 0xfd, 0xe6, 0x0a, 0xbc, 0x4c, 0x56, 0x76, 0xe3, 0xe5, 0x37, 0x68, 0xdf, + 0xe2, 0xb2, 0x94, 0xd9, 0x5b, 0xb1, 0xe6, 0x97, 0x1c, 0x6f, 0x54, 0xc7, 0xfc, 0x1b, 0xd4, 0xe1, + 0xf6, 0x60, 0xb3, 0xfc, 0xa1, 0x65, 0x1e, 0x91, 0x0b, 0x7e, 0x6c, 0xfd, 0x3a, 0x0f, 0x37, 0xaa, + 0x79, 0x65, 0xcc, 0x23, 0x49, 0xd1, 0x7d, 0x58, 0x34, 0x2f, 0xa5, 0xb4, 0xe4, 0x57, 0xcb, 0x7d, + 0xd2, 0x13, 0x61, 0x37, 0xe4, 0x7d, 0x3d, 0xee, 0xb0, 0x85, 0xa2, 0x07, 0xb0, 0x94, 0x4a, 0x59, + 0xda, 0x8b, 0x3a, 0x33, 0x2a, 0xc3, 0xa2, 0x8f, 0xa1, 0x31, 0x48, 0xc2, 0xd0, 0xb3, 0x09, 0xe7, + 0xcf, 0x99, 0xb0, 0x18, 0x34, 0xfa, 0x30, 0x4d, 0xf9, 0x29, 0x2c, 0x9b, 0xd8, 0x2c, 0xef, 0xe5, + 0xf3, 0x82, 0x4d, 0xaa, 0x6f, 0x6d, 0xe6, 0xef, 0x61, 0x2d, 0x2b, 0x47, 0x5e, 0xe2, 0x35, 0xc3, + 0xf0, 0x41, 0x79, 0xe7, 0x67, 0xa8, 0x0a, 0xaf, 0x06, 0x45, 0x27, 0x7a, 0x08, 0x60, 0xc2, 0xbd, + 0x44, 0x84, 0xb2, 0xb5, 0x5e, 0xde, 0x53, 0xca, 0xf8, 0x58, 0x2f, 0x7b, 0xf8, 0x48, 0xe2, 0xba, + 0xf1, 0xf4, 0x44, 0x28, 0xdd, 0x97, 0xb0, 0xa5, 0x4b, 0x91, 0xf2, 0xe9, 0x24, 0x19, 0xe7, 0x45, + 0x2b, 0x2e, 0xf2, 0xc6, 0x28, 0xb2, 0xda, 0x7a, 0x57, 0xaa, 0xdd, 0xb9, 0x98, 0xda, 0xbb, 0x9f, + 0xfd, 0xf0, 0xc9, 0x90, 0xa9, 0x93, 0xa4, 0xdf, 0xf6, 0xf9, 0xb8, 0x63, 0x58, 0xb8, 0x18, 0xa6, + 0x3f, 0x3a, 0xf9, 0x1f, 0x94, 0x21, 0x8d, 0x3a, 0x71, 0xff, 0xde, 0x90, 0x77, 0x8a, 0x7f, 0xa5, + 0xfa, 0x8b, 0xe6, 0xd1, 0xb8, 0xff, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x75, 0xb4, 0x2b, 0xe9, + 0x98, 0x0d, 0x00, 0x00, } diff --git a/flyteidl/gen/pb-go/flyteidl/datacatalog/datacatalog.pb.go b/flyteidl/gen/pb-go/flyteidl/datacatalog/datacatalog.pb.go index d1d623e792..b812ac0750 100644 --- a/flyteidl/gen/pb-go/flyteidl/datacatalog/datacatalog.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/datacatalog/datacatalog.pb.go @@ -97,7 +97,6 @@ func (PaginationOptions_SortKey) EnumDescriptor() ([]byte, []int) { return fileDescriptor_275951237ff4368a, []int{36, 1} } -// // Request message for creating a Dataset. type CreateDatasetRequest struct { Dataset *Dataset `protobuf:"bytes,1,opt,name=dataset,proto3" json:"dataset,omitempty"` @@ -138,7 +137,6 @@ func (m *CreateDatasetRequest) GetDataset() *Dataset { return nil } -// // Response message for creating a Dataset type CreateDatasetResponse struct { XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -171,7 +169,6 @@ func (m *CreateDatasetResponse) XXX_DiscardUnknown() { var xxx_messageInfo_CreateDatasetResponse proto.InternalMessageInfo -// // Request message for retrieving a Dataset. The Dataset is retrieved by it's unique identifier // which is a combination of several fields. type GetDatasetRequest struct { @@ -213,7 +210,6 @@ func (m *GetDatasetRequest) GetDataset() *DatasetID { return nil } -// // Response message for retrieving a Dataset. The response will include the metadata for the // Dataset. type GetDatasetResponse struct { @@ -255,7 +251,6 @@ func (m *GetDatasetResponse) GetDataset() *Dataset { return nil } -// // Request message for retrieving an Artifact. Retrieve an artifact based on a query handle that // can be one of artifact_id or tag. The result returned will include the artifact data and metadata // associated with the artifact. @@ -347,7 +342,6 @@ func (*GetArtifactRequest) XXX_OneofWrappers() []interface{} { } } -// // Response message for retrieving an Artifact. The result returned will include the artifact data // and metadata associated with the artifact. type GetArtifactResponse struct { @@ -389,7 +383,6 @@ func (m *GetArtifactResponse) GetArtifact() *Artifact { return nil } -// // Request message for creating an Artifact and its associated artifact Data. type CreateArtifactRequest struct { Artifact *Artifact `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` @@ -430,7 +423,6 @@ func (m *CreateArtifactRequest) GetArtifact() *Artifact { return nil } -// // Response message for creating an Artifact. type CreateArtifactResponse struct { XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -463,7 +455,6 @@ func (m *CreateArtifactResponse) XXX_DiscardUnknown() { var xxx_messageInfo_CreateArtifactResponse proto.InternalMessageInfo -// // Request message for tagging an Artifact. type AddTagRequest struct { Tag *Tag `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"` @@ -504,7 +495,6 @@ func (m *AddTagRequest) GetTag() *Tag { return nil } -// // Response message for tagging an Artifact. type AddTagResponse struct { XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -746,7 +736,6 @@ func (m *ListDatasetsResponse) GetNextToken() string { return "" } -// // Request message for updating an Artifact and overwriting its associated ArtifactData. type UpdateArtifactRequest struct { // ID of dataset the artifact is associated with @@ -858,7 +847,6 @@ func (*UpdateArtifactRequest) XXX_OneofWrappers() []interface{} { } } -// // Response message for updating an Artifact. type UpdateArtifactResponse struct { // The unique ID of the artifact updated @@ -900,7 +888,6 @@ func (m *UpdateArtifactResponse) GetArtifactId() string { return "" } -// // ReservationID message that is composed of several string fields. type ReservationID struct { // The unique ID for the reserved dataset @@ -1210,7 +1197,6 @@ func (m *ReleaseReservationResponse) XXX_DiscardUnknown() { var xxx_messageInfo_ReleaseReservationResponse proto.InternalMessageInfo -// // Dataset message. It is uniquely identified by DatasetID. type Dataset struct { Id *DatasetID `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -1267,7 +1253,6 @@ func (m *Dataset) GetPartitionKeys() []string { return nil } -// // An artifact could have multiple partitions and each partition can have an arbitrary string key/value pair type Partition struct { Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` @@ -1316,7 +1301,6 @@ func (m *Partition) GetValue() string { return "" } -// // DatasetID message that is composed of several string fields. type DatasetID struct { Project string `protobuf:"bytes,1,opt,name=project,proto3" json:"project,omitempty"` @@ -1398,7 +1382,6 @@ func (m *DatasetID) GetOrg() string { return "" } -// // Artifact message. It is composed of several string fields. type Artifact struct { Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -1487,7 +1470,6 @@ func (m *Artifact) GetCreatedAt() *timestamp.Timestamp { return nil } -// // ArtifactData that belongs to an artifact type ArtifactData struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -1536,7 +1518,6 @@ func (m *ArtifactData) GetValue() *core.Literal { return nil } -// // Tag message that is unique to a Dataset. It is associated to a single artifact and // can be retrieved by name later. type Tag struct { @@ -1594,7 +1575,6 @@ func (m *Tag) GetDataset() *DatasetID { return nil } -// // Metadata representation for artifacts and datasets type Metadata struct { KeyMap map[string]string `protobuf:"bytes,1,rep,name=key_map,json=keyMap,proto3" json:"key_map,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/admin.pb.go b/flyteidl/gen/pb-go/flyteidl/service/admin.pb.go index db03215645..a00d043f14 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/admin.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/service/admin.pb.go @@ -29,181 +29,184 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package func init() { proto.RegisterFile("flyteidl/service/admin.proto", fileDescriptor_5cfa31da1d67295d) } var fileDescriptor_5cfa31da1d67295d = []byte{ - // 2771 bytes of a gzipped FileDescriptorProto + // 2832 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5b, 0x6b, 0x8c, 0x15, 0x57, - 0x1d, 0xcf, 0x99, 0xa5, 0xa2, 0xa7, 0x3c, 0x96, 0x7f, 0x21, 0xc0, 0xf0, 0x2a, 0x43, 0x61, 0x97, - 0x0b, 0xec, 0xdd, 0xf2, 0x68, 0x65, 0xb5, 0x2d, 0x5b, 0x16, 0x6e, 0xa1, 0xbc, 0x5c, 0x68, 0xb5, - 0xb7, 0xc6, 0x9b, 0xd9, 0x7b, 0x0f, 0xcb, 0xc0, 0xdd, 0x3b, 0xd7, 0x99, 0xd9, 0x6d, 0x09, 0xd9, - 0xc6, 0x94, 0x8a, 0x58, 0x03, 0x98, 0x20, 0x34, 0x46, 0xdb, 0x0f, 0x8d, 0x4d, 0xea, 0x23, 0x7e, - 0x30, 0xd1, 0xc4, 0x44, 0x13, 0x3f, 0x48, 0xd2, 0xb4, 0x31, 0x69, 0x53, 0xab, 0x8d, 0xf6, 0x93, - 0x21, 0xa1, 0x89, 0x46, 0xad, 0x7e, 0xb0, 0x7e, 0x31, 0x66, 0xce, 0x63, 0xe6, 0xcc, 0xe3, 0xcc, - 0x63, 0x59, 0x9a, 0x26, 0xe5, 0xdb, 0xee, 0x3d, 0xff, 0x73, 0xce, 0xef, 0xf7, 0xfb, 0x3f, 0xce, - 0x39, 0x33, 0x67, 0xf0, 0xca, 0xe3, 0xed, 0xd3, 0x1e, 0xb1, 0x5a, 0xed, 0xaa, 0x4b, 0x9c, 0x29, - 0xab, 0x49, 0xaa, 0x66, 0x6b, 0xc2, 0xea, 0x0c, 0x74, 0x1d, 0xdb, 0xb3, 0xa1, 0x57, 0xb4, 0x0e, - 0xf0, 0x56, 0x7d, 0xe5, 0xb8, 0x6d, 0x8f, 0xb7, 0x49, 0xd5, 0xec, 0x5a, 0x55, 0xb3, 0xd3, 0xb1, - 0x3d, 0xd3, 0xb3, 0xec, 0x8e, 0xcb, 0xec, 0xf5, 0x70, 0x34, 0x3a, 0x4a, 0xb5, 0xeb, 0xd8, 0x27, - 0x49, 0xd3, 0xe3, 0xad, 0x03, 0xe9, 0xad, 0x8d, 0x96, 0x3d, 0x61, 0x5a, 0x9d, 0x86, 0xe9, 0x79, - 0x8e, 0x35, 0x36, 0xe9, 0x11, 0x31, 0x5a, 0x9f, 0xc2, 0x3e, 0x61, 0xb8, 0x3c, 0x66, 0xe8, 0x99, - 0xee, 0x29, 0xde, 0xb4, 0x2a, 0xd6, 0xf4, 0x94, 0xed, 0x9c, 0x3a, 0xde, 0xb6, 0x9f, 0xe2, 0xcd, - 0xfd, 0x8a, 0xe6, 0xe4, 0x1c, 0x77, 0xc7, 0x2c, 0xdb, 0xe6, 0x64, 0xa7, 0x79, 0xa2, 0xd1, 0x6d, - 0x9b, 0x5c, 0x2c, 0x5d, 0x8f, 0x59, 0x90, 0x29, 0xd2, 0x11, 0xd4, 0x57, 0xc7, 0xdb, 0x9e, 0x26, - 0xcd, 0x49, 0x5f, 0x39, 0x05, 0xd5, 0x09, 0xd3, 0x6b, 0x9e, 0x30, 0xc7, 0xda, 0xa4, 0xe1, 0x10, - 0xd7, 0x9e, 0x74, 0x9a, 0x84, 0x1b, 0xae, 0x8b, 0x19, 0x76, 0xec, 0x16, 0x69, 0xc4, 0x47, 0x5b, - 0x97, 0xa2, 0x47, 0xc2, 0x28, 0xee, 0xab, 0x29, 0xe2, 0xb8, 0x61, 0xeb, 0x8a, 0x58, 0x6b, 0xd3, - 0x9e, 0x98, 0x50, 0xa2, 0x6d, 0x11, 0xb7, 0xe9, 0x58, 0x5d, 0x7f, 0xf0, 0x06, 0xe9, 0x78, 0x96, - 0x77, 0x9a, 0x19, 0x6e, 0x7d, 0x61, 0x0a, 0xcf, 0x1b, 0xf6, 0x4d, 0x8e, 0xb2, 0xf0, 0x81, 0x8b, - 0x08, 0xe3, 0xdd, 0x0e, 0x31, 0x3d, 0x72, 0xcc, 0x74, 0x4f, 0xc1, 0xda, 0x20, 0x24, 0x06, 0x58, - 0xd8, 0xf9, 0xbf, 0xb2, 0xf6, 0x51, 0xf2, 0xd5, 0x49, 0xe2, 0x7a, 0xba, 0x91, 0x65, 0xe2, 0x76, - 0xed, 0x8e, 0x4b, 0x8c, 0x9d, 0xcf, 0xbe, 0x7d, 0xfd, 0xb2, 0xb6, 0xcd, 0x98, 0x4f, 0xc3, 0x72, - 0xea, 0x5e, 0x4a, 0xd8, 0x1d, 0x42, 0x95, 0xfa, 0x1a, 0x43, 0x8f, 0xfc, 0x56, 0xb5, 0x9d, 0xf1, - 0xea, 0x19, 0xab, 0x35, 0x60, 0x3b, 0xe3, 0xd3, 0x43, 0xa8, 0x02, 0xef, 0x22, 0x3c, 0xb7, 0x46, - 0x3c, 0x8a, 0xe6, 0xee, 0xf8, 0x54, 0x87, 0xc7, 0xfc, 0x78, 0xab, 0x11, 0x4f, 0x80, 0x59, 0x9c, - 0x06, 0xc6, 0xf8, 0x0e, 0xa2, 0xf3, 0x5f, 0x40, 0xf0, 0x40, 0x74, 0x32, 0x7f, 0x22, 0x1e, 0xb4, - 0xd3, 0xf4, 0x1f, 0x16, 0xe9, 0xec, 0xef, 0x8e, 0x39, 0x41, 0xd8, 0x5f, 0x5c, 0xf9, 0xe9, 0xfa, - 0x01, 0xd8, 0xaf, 0x46, 0x5b, 0x76, 0x34, 0xb8, 0x86, 0xf0, 0x9d, 0x07, 0x2c, 0x97, 0x72, 0xdb, - 0xd7, 0x72, 0x61, 0x30, 0x0e, 0xfe, 0x90, 0x39, 0x41, 0x5a, 0x7b, 0xa8, 0xbf, 0xf6, 0xb5, 0x7c, - 0xbf, 0x1d, 0xb7, 0x88, 0xe3, 0xf7, 0x10, 0x74, 0x37, 0x16, 0xee, 0x61, 0x3c, 0x41, 0x25, 0x38, - 0x0a, 0xeb, 0x64, 0x02, 0x0d, 0xab, 0xe5, 0x56, 0xcf, 0x84, 0x98, 0x39, 0xe0, 0xfa, 0x66, 0xa8, - 0xa4, 0xf0, 0x64, 0x24, 0x13, 0xd6, 0xf0, 0xaa, 0x86, 0x3f, 0x23, 0x78, 0xb8, 0xb0, 0x2e, 0x8e, - 0x69, 0x94, 0x27, 0x88, 0x0c, 0x7c, 0x59, 0x9a, 0x9f, 0x28, 0xce, 0x3f, 0x33, 0x5f, 0xfd, 0x1e, - 0xc1, 0x60, 0x59, 0x5f, 0xd5, 0x1f, 0x8a, 0xfb, 0xb7, 0xa4, 0x7b, 0xea, 0xfd, 0xb0, 0xa1, 0xd8, - 0xa4, 0xf5, 0x1d, 0xb0, 0x6d, 0x06, 0x53, 0xc1, 0xcb, 0x08, 0x2f, 0x60, 0xb9, 0xf1, 0x45, 0x5e, - 0xc8, 0x60, 0x7d, 0x5c, 0x0a, 0xd1, 0x12, 0x4d, 0xb3, 0x0d, 0x79, 0x66, 0x3c, 0xd5, 0x86, 0xa9, - 0x7c, 0x9f, 0x33, 0x16, 0x09, 0x78, 0xa2, 0x62, 0xd2, 0x74, 0x5b, 0x67, 0xac, 0x4e, 0xfc, 0x9e, - 0x48, 0xb9, 0xf7, 0x11, 0xbe, 0xb3, 0x46, 0xbc, 0x00, 0x61, 0x7e, 0xda, 0x2d, 0x53, 0x81, 0x33, - 0xbe, 0xcf, 0xdc, 0x79, 0x05, 0xc1, 0x70, 0x72, 0xe2, 0xb2, 0xe9, 0x77, 0x04, 0x0e, 0x65, 0xa3, - 0x2f, 0x9d, 0x82, 0x6f, 0x21, 0xbc, 0xd0, 0x0f, 0x3b, 0x81, 0xf7, 0x96, 0xa7, 0xa1, 0x49, 0xe5, - 0x78, 0x12, 0xfa, 0xe2, 0x44, 0x54, 0xa9, 0x38, 0x08, 0x03, 0x0a, 0xce, 0xaa, 0x74, 0xfc, 0x8d, - 0x86, 0xe7, 0xcb, 0x9c, 0x0a, 0xa6, 0xe4, 0x4a, 0x95, 0x0f, 0x29, 0xee, 0xbf, 0x32, 0x3f, 0x5e, - 0x47, 0xb0, 0x7d, 0x26, 0x7e, 0xac, 0xef, 0x4e, 0xf3, 0x7f, 0xd9, 0xf4, 0x94, 0xca, 0x52, 0xfe, - 0xe4, 0xf5, 0x9d, 0x70, 0xff, 0x0c, 0xa7, 0x84, 0x9f, 0x22, 0xdc, 0xcb, 0xf2, 0xea, 0x00, 0xdd, - 0x45, 0x1c, 0x69, 0x9b, 0x1d, 0xe8, 0x8b, 0x0b, 0x14, 0xb6, 0x45, 0x53, 0xb5, 0x3f, 0xdf, 0x90, - 0x27, 0x6b, 0x8d, 0x8a, 0x3a, 0x6c, 0x2c, 0x16, 0x40, 0xa5, 0x4d, 0x0b, 0xcd, 0xd7, 0x0d, 0xc6, - 0xda, 0xb4, 0xa6, 0x44, 0xca, 0xfe, 0x0b, 0xe1, 0xf9, 0x35, 0xe2, 0x49, 0x68, 0xf3, 0x93, 0x56, - 0x57, 0xc3, 0x34, 0x5e, 0x66, 0xee, 0xfe, 0x1e, 0x82, 0x91, 0xd4, 0xf9, 0xcb, 0x66, 0xee, 0x51, - 0xf8, 0x42, 0x2e, 0x8f, 0xd2, 0xc9, 0xfb, 0x6f, 0x84, 0xef, 0xaa, 0x11, 0x6f, 0xb8, 0xe9, 0x59, - 0x53, 0x99, 0x8e, 0x8a, 0x5b, 0x14, 0x51, 0xe0, 0x2a, 0x53, 0xe0, 0x12, 0x82, 0x07, 0x05, 0x72, - 0x93, 0x0e, 0xd3, 0x28, 0x29, 0x44, 0xfd, 0x20, 0x3c, 0x9a, 0x35, 0x42, 0x49, 0x09, 0xfc, 0x0d, - 0xd1, 0x12, 0x3f, 0x25, 0xe3, 0xa4, 0x5c, 0xd8, 0x94, 0xc7, 0x5b, 0x4e, 0xf7, 0xd5, 0x6a, 0xee, - 0x34, 0xe1, 0x1d, 0x4a, 0xbf, 0x1d, 0x56, 0x9f, 0x54, 0xf6, 0xc9, 0x7a, 0x35, 0x04, 0x9f, 0xcd, - 0x65, 0xab, 0xaa, 0x5c, 0xef, 0x21, 0xbc, 0xc8, 0x9f, 0x3c, 0x84, 0x72, 0xcb, 0xeb, 0xf1, 0x49, - 0x4a, 0xb3, 0x15, 0x16, 0x16, 0x09, 0xad, 0xaa, 0x24, 0xdf, 0x17, 0xd6, 0xc0, 0xb8, 0x75, 0x26, - 0xbd, 0x37, 0x35, 0xb6, 0xd8, 0xc8, 0x3e, 0x2b, 0x54, 0x9a, 0xf3, 0x7c, 0xf5, 0x1f, 0x16, 0xab, - 0xff, 0x44, 0x61, 0xc5, 0x2b, 0x1b, 0xa4, 0x7b, 0x15, 0x89, 0x5e, 0xb6, 0x44, 0x57, 0x61, 0x4b, - 0x29, 0x08, 0xf5, 0xcf, 0xc3, 0xd0, 0xcc, 0x27, 0x86, 0x0b, 0x1a, 0xee, 0x7d, 0xac, 0xdb, 0x2a, - 0x5c, 0xa8, 0x99, 0x6d, 0x81, 0x42, 0x2d, 0x0c, 0x79, 0xa1, 0x7e, 0x85, 0x29, 0xfc, 0x12, 0xd2, - 0x67, 0xa5, 0x1e, 0xfa, 0xa5, 0xfd, 0xa8, 0x7e, 0x0b, 0x4a, 0xe2, 0x0f, 0x10, 0x5e, 0xc8, 0xd6, - 0x98, 0x3d, 0xe2, 0x34, 0x09, 0x89, 0x9d, 0x63, 0xd0, 0x14, 0x5d, 0xb6, 0xfa, 0x72, 0xed, 0xb8, - 0x18, 0xbb, 0xa8, 0x16, 0x43, 0x06, 0x08, 0xfc, 0xc1, 0xc9, 0x95, 0xae, 0x59, 0x6b, 0xf5, 0x95, - 0xc9, 0x86, 0x30, 0x33, 0xfc, 0xe5, 0xea, 0xd7, 0x08, 0x2f, 0x1a, 0x25, 0x8c, 0x76, 0x08, 0xb4, - 0x5f, 0x09, 0x40, 0xd8, 0x96, 0x86, 0xfa, 0x18, 0x85, 0x7a, 0xd8, 0x58, 0x91, 0x82, 0xc8, 0xe1, - 0x83, 0xfa, 0x98, 0xb7, 0x18, 0xfd, 0x2a, 0xcc, 0xc2, 0x15, 0x92, 0x39, 0xfc, 0x12, 0xe1, 0xde, - 0x51, 0xd2, 0xb4, 0xa7, 0x88, 0x13, 0xc2, 0xef, 0xcb, 0x80, 0x4f, 0x4d, 0x4b, 0xa3, 0x3f, 0x4a, - 0xd1, 0x1f, 0x0c, 0x8f, 0xc8, 0x11, 0xf4, 0x74, 0x4c, 0x1f, 0xfc, 0x66, 0xa3, 0x2f, 0x1f, 0xbc, - 0xb0, 0xf6, 0x8b, 0xec, 0xbc, 0x1a, 0xf1, 0x42, 0xdc, 0x9b, 0x54, 0x1b, 0xbf, 0xc0, 0x44, 0xda, - 0x34, 0x2c, 0x57, 0x62, 0x37, 0xce, 0xb2, 0x1c, 0x99, 0x86, 0x1d, 0x29, 0x68, 0x0a, 0x94, 0x8e, - 0x11, 0x78, 0x38, 0x8f, 0x46, 0x81, 0xe5, 0xf1, 0x1f, 0x08, 0x2f, 0x64, 0xc9, 0x5b, 0x24, 0x03, - 0xa2, 0xf5, 0xa0, 0x2f, 0xd7, 0x8e, 0x3b, 0xe6, 0x79, 0x46, 0xf5, 0x2c, 0xd2, 0x67, 0xc6, 0xd5, - 0xf7, 0x5a, 0x4d, 0x9f, 0x05, 0xba, 0xbe, 0x43, 0xcf, 0x69, 0xb8, 0x57, 0x76, 0xe8, 0x88, 0xe9, - 0x99, 0x50, 0x2d, 0xe2, 0x54, 0xdf, 0x52, 0x70, 0x1f, 0x2c, 0xde, 0x81, 0x8b, 0x70, 0x91, 0x89, - 0xf0, 0x0d, 0x14, 0x56, 0xf0, 0x96, 0xe9, 0x99, 0x25, 0xbd, 0xbe, 0x0f, 0x6a, 0xaa, 0xde, 0x65, - 0x5d, 0x7f, 0x0d, 0xe1, 0x05, 0xfe, 0x7a, 0x18, 0x20, 0x2e, 0xb8, 0xbc, 0xae, 0x52, 0xba, 0x9d, - 0xae, 0xae, 0x36, 0xa5, 0x69, 0xc1, 0xa6, 0x12, 0xae, 0x96, 0xb7, 0x41, 0x65, 0x19, 0xc1, 0x87, - 0x08, 0xc3, 0x31, 0xe2, 0x4c, 0x58, 0x9d, 0x48, 0x14, 0x6f, 0x54, 0xc2, 0x0c, 0x8c, 0x05, 0xa3, - 0x4a, 0x11, 0xd3, 0x78, 0x2c, 0x57, 0x6e, 0x22, 0x96, 0x2b, 0xb3, 0x14, 0xcb, 0x7f, 0x61, 0xb1, - 0x7c, 0xc8, 0x6e, 0x91, 0x8c, 0xc2, 0x1a, 0x69, 0x96, 0x8a, 0xd3, 0xaa, 0x4c, 0x43, 0xe3, 0xeb, - 0x1a, 0x65, 0xfa, 0x3f, 0x04, 0x1d, 0x81, 0x36, 0xfa, 0x74, 0x96, 0xd1, 0x0d, 0xfe, 0x6d, 0xc4, - 0x21, 0x47, 0x5a, 0x64, 0xfc, 0x91, 0x86, 0x70, 0x59, 0xa6, 0xa3, 0x5b, 0xad, 0xe9, 0xfa, 0xb7, - 0x10, 0x9c, 0x47, 0xaa, 0x39, 0x85, 0x4c, 0x91, 0x41, 0x02, 0xcd, 0x66, 0x1f, 0x0d, 0xfc, 0x57, - 0xc3, 0xe0, 0x47, 0x76, 0x44, 0x1d, 0x37, 0xb9, 0xfc, 0x46, 0xda, 0xe5, 0x84, 0x59, 0x9b, 0x6b, - 0x69, 0x5c, 0x61, 0x5a, 0x5f, 0xd4, 0xc0, 0x55, 0x6a, 0x1d, 0x3c, 0xfa, 0x50, 0x70, 0x4c, 0x6f, - 0x0f, 0x98, 0xa6, 0x37, 0xb3, 0xaa, 0x72, 0x15, 0xc1, 0xe5, 0x6c, 0xc1, 0xd3, 0x7b, 0x33, 0xd9, - 0x6f, 0x1d, 0x30, 0xb8, 0xf4, 0x69, 0xbc, 0x3c, 0xa9, 0xfd, 0x5e, 0xdb, 0xa1, 0x0f, 0xb8, 0xab, - 0x99, 0xc2, 0x72, 0xab, 0x92, 0x9e, 0x78, 0x71, 0x2e, 0xf5, 0xc4, 0xd5, 0xb9, 0xf0, 0xc3, 0x1e, - 0xa1, 0x48, 0xf3, 0x84, 0xd5, 0x6e, 0x39, 0x24, 0xfe, 0xe2, 0xc1, 0xad, 0x9e, 0x89, 0xfe, 0xd0, - 0x10, 0x61, 0x13, 0xf9, 0x45, 0xa1, 0x4a, 0xe9, 0xae, 0x81, 0x60, 0xa5, 0x7b, 0xf2, 0xa0, 0x2e, - 0xd2, 0x4f, 0x44, 0x7d, 0x9a, 0x35, 0x7f, 0x04, 0x9e, 0xc9, 0x41, 0xd8, 0x64, 0x80, 0x15, 0x26, - 0x4a, 0x54, 0xc2, 0x40, 0x6c, 0xd3, 0xd3, 0x6c, 0x1c, 0xe2, 0x39, 0xa7, 0x1b, 0xa6, 0xe7, 0x91, - 0x89, 0xae, 0x37, 0x5d, 0xbf, 0xd1, 0x03, 0xd7, 0xf3, 0xdd, 0x45, 0x23, 0xb9, 0xb4, 0x7a, 0x2c, - 0xc8, 0x6f, 0x7b, 0xfa, 0x63, 0xe1, 0x69, 0x38, 0xdb, 0x83, 0x17, 0xc7, 0x97, 0x3c, 0xba, 0x85, - 0xdb, 0x94, 0xb7, 0xec, 0xc9, 0xdb, 0xb7, 0xcd, 0xc5, 0x8c, 0xf9, 0x9a, 0x7f, 0x89, 0x55, 0xe7, - 0xf3, 0x52, 0x75, 0xa6, 0x9b, 0xaf, 0x8f, 0x68, 0x39, 0xfc, 0x36, 0x82, 0x0b, 0x28, 0x73, 0xe2, - 0x8f, 0x7c, 0x4d, 0x7c, 0x15, 0xe1, 0x85, 0xa3, 0x64, 0xdc, 0x72, 0x3d, 0xe2, 0x1c, 0x61, 0x23, - 0x26, 0x8f, 0x0d, 0xbc, 0x41, 0xd8, 0x29, 0x8f, 0x0d, 0x09, 0x3b, 0x2e, 0xfb, 0x08, 0x55, 0xfd, - 0x41, 0xa3, 0x57, 0x70, 0xe7, 0xd8, 0xe9, 0xb1, 0xb9, 0xcf, 0x30, 0xe2, 0x3f, 0x33, 0x19, 0xc4, - 0x3b, 0x7a, 0x71, 0x78, 0x7e, 0x01, 0xe1, 0xf9, 0xec, 0x3c, 0x22, 0x80, 0x2e, 0x55, 0x00, 0xd0, - 0xd7, 0x2b, 0x1a, 0x62, 0xc7, 0x99, 0xbd, 0x14, 0xd7, 0x2e, 0x7d, 0x49, 0x02, 0xc0, 0x19, 0xab, - 0x45, 0x37, 0x78, 0xf7, 0xe8, 0x6b, 0xd2, 0xc1, 0x09, 0x9f, 0x50, 0x64, 0xcf, 0x21, 0x3c, 0xcf, - 0x5f, 0x73, 0xf8, 0x2c, 0x2e, 0x18, 0x8a, 0xf9, 0x33, 0x5f, 0x05, 0x8a, 0xde, 0xc6, 0x7d, 0x14, - 0xd6, 0x20, 0x24, 0xe4, 0xaa, 0xaf, 0x04, 0x5d, 0x0d, 0x07, 0xde, 0x41, 0xf8, 0xae, 0xe8, 0x4b, - 0xb6, 0x3d, 0x53, 0xa4, 0xe3, 0xc1, 0x96, 0xdc, 0x23, 0x0e, 0xb5, 0x13, 0xc0, 0x06, 0x8a, 0x9a, - 0x73, 0x15, 0x1b, 0x14, 0xee, 0x13, 0xc6, 0xb2, 0x60, 0x2b, 0xec, 0x37, 0xbb, 0xd1, 0x17, 0x70, - 0x43, 0xc6, 0x8e, 0x58, 0x33, 0xc5, 0xcd, 0xae, 0x22, 0x24, 0xa3, 0x5d, 0xee, 0x0b, 0xbf, 0x0d, - 0x1e, 0xee, 0xd0, 0xe4, 0xa6, 0x9c, 0x36, 0x66, 0xe6, 0x7d, 0x84, 0x4f, 0xa5, 0x88, 0x29, 0xe7, - 0x52, 0xa7, 0x5c, 0x8e, 0x85, 0x2f, 0x26, 0x38, 0x58, 0x3f, 0x5f, 0x28, 0x8f, 0x9d, 0xc6, 0x76, - 0x35, 0x8f, 0xd4, 0xc4, 0x15, 0x5d, 0xe1, 0x4f, 0x01, 0x0d, 0x7f, 0x37, 0xa3, 0xa0, 0x41, 0x9b, - 0x8a, 0xd1, 0x48, 0x33, 0xe5, 0x34, 0xba, 0x94, 0xc6, 0xc9, 0x04, 0x8d, 0xe0, 0xfa, 0xc1, 0x61, - 0x63, 0xbf, 0x9a, 0x46, 0xd7, 0x74, 0x48, 0xc7, 0x6b, 0x14, 0x59, 0x4d, 0xc5, 0x80, 0xf0, 0xc6, - 0x1d, 0xf4, 0x00, 0x13, 0xc1, 0x94, 0x3c, 0xc0, 0x44, 0x9a, 0xb3, 0x0e, 0x30, 0x11, 0x43, 0xe3, - 0xef, 0x73, 0x28, 0x9f, 0x1b, 0x73, 0xe0, 0xbb, 0x5a, 0xe4, 0x35, 0x7e, 0xac, 0x64, 0x17, 0x5e, - 0xc8, 0x4b, 0xac, 0xdc, 0x85, 0x97, 0xea, 0x9c, 0xb5, 0x39, 0x75, 0x31, 0x4e, 0x5b, 0x7d, 0x93, - 0xcb, 0x6d, 0xea, 0xfa, 0x9a, 0xdc, 0x3a, 0xbd, 0xa6, 0xc1, 0x35, 0xa5, 0x3c, 0x62, 0x61, 0x29, - 0xb4, 0x45, 0xfa, 0xa4, 0x4b, 0x09, 0x1f, 0xf4, 0xb0, 0x93, 0x62, 0x24, 0x0c, 0x53, 0x4e, 0x8a, - 0x91, 0xf6, 0xcc, 0xf3, 0x49, 0xc2, 0xd2, 0xb8, 0xd0, 0x43, 0x83, 0xfa, 0x5c, 0x0f, 0xfc, 0x08, - 0x29, 0x83, 0xba, 0xb0, 0x1b, 0x8a, 0xfa, 0xa0, 0x98, 0x03, 0xd4, 0xea, 0xd7, 0xff, 0x80, 0xe0, - 0x6d, 0x94, 0x19, 0x65, 0x85, 0x42, 0xec, 0x63, 0x47, 0x0c, 0xfe, 0x76, 0x07, 0xdd, 0x8f, 0x46, - 0x1c, 0x95, 0xbe, 0x1f, 0x8d, 0x57, 0xb1, 0xcc, 0xfd, 0x68, 0xba, 0x31, 0xaf, 0xd3, 0x1f, 0xb2, - 0xc2, 0xf6, 0xc1, 0x1c, 0x78, 0x49, 0x8b, 0xec, 0x0b, 0x6f, 0x57, 0xb7, 0x78, 0x75, 0xfb, 0x9d, - 0x06, 0xaf, 0x67, 0x6b, 0x74, 0xbb, 0xc4, 0x95, 0x29, 0x71, 0x6f, 0x68, 0x78, 0x55, 0x64, 0x37, - 0x3d, 0x42, 0x87, 0x1c, 0x0e, 0xae, 0x97, 0xc2, 0x76, 0xc5, 0x06, 0x35, 0x6e, 0x18, 0x7d, 0x97, - 0xb0, 0xa3, 0x64, 0x2f, 0x9e, 0x09, 0xbf, 0x62, 0x4f, 0x63, 0x7f, 0x8e, 0xf4, 0xfd, 0xb1, 0x1d, - 0x6e, 0xf2, 0x16, 0x6e, 0xf5, 0x4c, 0xf4, 0x12, 0x2c, 0x97, 0x47, 0xfa, 0x91, 0xcb, 0xe3, 0xef, - 0x74, 0x5a, 0x7a, 0x23, 0x7f, 0x40, 0x1a, 0x3a, 0x52, 0x7f, 0x16, 0x23, 0xc5, 0x67, 0x81, 0xb3, - 0x1a, 0xd6, 0x6b, 0xc4, 0x53, 0x49, 0x79, 0x6f, 0x41, 0x51, 0xa4, 0x2d, 0xd1, 0xd6, 0x32, 0x5d, - 0xb8, 0x88, 0xcf, 0x50, 0x0d, 0x9f, 0x0e, 0x5f, 0xd3, 0x67, 0x48, 0x98, 0x7c, 0xbd, 0xbf, 0x2b, - 0xbc, 0xf1, 0x91, 0xa3, 0x94, 0xea, 0x45, 0xff, 0x65, 0x0d, 0xaf, 0x1a, 0x21, 0x6d, 0x72, 0xf3, - 0x31, 0xc5, 0x46, 0x29, 0x1b, 0x53, 0xa2, 0x17, 0x97, 0xe3, 0x39, 0x16, 0x53, 0xcf, 0x54, 0x66, - 0xa4, 0x87, 0x1f, 0x3c, 0xbb, 0x2b, 0x37, 0x29, 0x89, 0x1f, 0x1b, 0xcf, 0x6b, 0x78, 0x69, 0x24, - 0xd3, 0x24, 0x3d, 0x06, 0x14, 0xcc, 0x54, 0xd9, 0x55, 0x2d, 0x6c, 0xcf, 0x35, 0xb8, 0xc0, 0x34, - 0x38, 0x87, 0xf4, 0x6a, 0x9c, 0x49, 0x4e, 0x42, 0xf9, 0xfc, 0x8f, 0xe8, 0x8f, 0xce, 0x62, 0xf2, - 0xf0, 0x6b, 0xcd, 0x8b, 0xc3, 0x44, 0x91, 0x94, 0xd8, 0x94, 0xcb, 0x4c, 0x4a, 0x8e, 0xcd, 0xc5, - 0x8c, 0xb9, 0x06, 0x84, 0x4a, 0xd0, 0x80, 0x7b, 0xb2, 0x14, 0x10, 0x20, 0xeb, 0xf7, 0x87, 0xef, - 0x71, 0x4b, 0xf9, 0x1c, 0x6e, 0x20, 0xbc, 0x34, 0x12, 0xfd, 0xa5, 0xfc, 0x1c, 0x8d, 0xf8, 0x6a, - 0x61, 0x7b, 0xce, 0xf1, 0x14, 0xe5, 0x48, 0x2a, 0x85, 0x38, 0xd2, 0x03, 0x79, 0x65, 0x66, 0x34, - 0xe9, 0xab, 0x74, 0x0d, 0x2f, 0x63, 0x71, 0x26, 0x1e, 0x0d, 0x48, 0x54, 0x95, 0x6f, 0x60, 0x55, - 0x31, 0x3d, 0x58, 0xbc, 0x03, 0x27, 0xfb, 0x1e, 0x0b, 0xea, 0x77, 0x90, 0x5e, 0x4f, 0x5c, 0x27, - 0x9d, 0xc1, 0x32, 0x11, 0xf9, 0x4d, 0x0c, 0x44, 0x45, 0xf2, 0x74, 0x3b, 0x6b, 0x82, 0x9b, 0x5a, - 0x36, 0x94, 0xb3, 0xc2, 0xd7, 0x34, 0xbc, 0x44, 0xba, 0x87, 0x2c, 0x69, 0xbb, 0x39, 0x5f, 0x2a, - 0x29, 0x4b, 0xb6, 0x14, 0xb4, 0xe6, 0xaa, 0x7e, 0x93, 0xa9, 0xfa, 0x2c, 0x82, 0x9d, 0x99, 0xaa, - 0x26, 0x8a, 0x5c, 0xf8, 0xe6, 0x67, 0xba, 0xbe, 0x07, 0x76, 0xe7, 0x2a, 0xa6, 0xa8, 0x95, 0xd2, - 0x30, 0x70, 0x49, 0xc3, 0xcb, 0x58, 0x84, 0xcf, 0x2c, 0xc2, 0xa2, 0xd9, 0x34, 0x58, 0xbc, 0x83, - 0x78, 0x50, 0xcc, 0xb4, 0x38, 0x8f, 0x2a, 0x33, 0xd7, 0xc2, 0x0f, 0xa0, 0x47, 0x2a, 0xb3, 0x21, - 0x87, 0x1f, 0x14, 0xaf, 0x23, 0xbc, 0xd4, 0x3f, 0x37, 0x1e, 0x14, 0x1f, 0xe0, 0x64, 0x55, 0x17, - 0x85, 0xa1, 0xb2, 0xba, 0x28, 0xed, 0xb9, 0x1c, 0x87, 0xa8, 0x1a, 0x8f, 0x40, 0x70, 0xc1, 0x29, - 0xfc, 0x0c, 0x28, 0x64, 0x23, 0x7f, 0x45, 0x90, 0xd6, 0x2e, 0x3d, 0xa9, 0x7c, 0x9f, 0x5f, 0x78, - 0x0c, 0xef, 0x27, 0x5a, 0xc4, 0x4d, 0x3e, 0x77, 0x96, 0xae, 0x2f, 0xca, 0x87, 0xeb, 0x35, 0x39, - 0x76, 0x69, 0x91, 0xec, 0x6f, 0x8e, 0x5b, 0xec, 0x1b, 0x20, 0xcb, 0x77, 0x9c, 0xf8, 0x7a, 0xa9, - 0xe1, 0x9d, 0xee, 0x92, 0x34, 0xf9, 0xe5, 0x48, 0xce, 0xe9, 0x9c, 0xb9, 0x27, 0x3a, 0xab, 0xe1, - 0x05, 0x35, 0x22, 0x31, 0x3d, 0x9d, 0xfc, 0xf2, 0x41, 0x6a, 0x94, 0xd2, 0x77, 0x45, 0x86, 0x99, - 0xf1, 0x13, 0x46, 0xf1, 0x15, 0x04, 0xfb, 0x8a, 0x52, 0xcc, 0xbf, 0x93, 0xf2, 0x25, 0x78, 0xbc, - 0x0c, 0xe5, 0x12, 0x57, 0x54, 0x5e, 0xd4, 0xf0, 0x22, 0x56, 0xc4, 0x65, 0x21, 0xfa, 0x33, 0x18, - 0x46, 0xd7, 0x88, 0x8d, 0x05, 0x2c, 0x79, 0xac, 0xfe, 0x8c, 0x29, 0xf3, 0x63, 0xa4, 0xcf, 0x9e, - 0x32, 0x7e, 0x2a, 0x3f, 0xa9, 0xdf, 0x22, 0x71, 0xfc, 0xec, 0x6e, 0x63, 0x5c, 0x23, 0xde, 0xe3, - 0xec, 0xbc, 0x96, 0xfc, 0xfa, 0x2c, 0x6c, 0x53, 0x7e, 0x7d, 0x26, 0x9b, 0x70, 0x25, 0x96, 0x52, - 0x21, 0x16, 0xc1, 0x42, 0x81, 0x9b, 0x9f, 0x07, 0xe1, 0x17, 0x1a, 0xdd, 0x84, 0x8d, 0x84, 0x5f, - 0xc7, 0x71, 0x87, 0xe4, 0x5f, 0x9e, 0x4f, 0x40, 0x4b, 0x0c, 0x62, 0xbc, 0xcb, 0x1c, 0xf0, 0x16, - 0x82, 0xe0, 0xe4, 0x95, 0xf8, 0x0e, 0x8f, 0xca, 0x46, 0x0f, 0xa5, 0x25, 0x3d, 0x11, 0xbd, 0x5e, - 0xdf, 0x81, 0x76, 0xe6, 0x14, 0x09, 0x4f, 0xdc, 0xdc, 0x7c, 0xf0, 0x5a, 0x0f, 0x2b, 0xc2, 0x71, - 0xd2, 0x56, 0xda, 0xda, 0x9c, 0x50, 0x46, 0xae, 0x61, 0xeb, 0x0b, 0x59, 0x1b, 0x7f, 0x64, 0x2f, - 0x2c, 0xdf, 0xd4, 0xe0, 0x70, 0xb6, 0x96, 0xe5, 0x93, 0xfd, 0x2b, 0xf0, 0xe5, 0x52, 0x43, 0x96, - 0xbd, 0x11, 0x5d, 0x83, 0x3d, 0xb3, 0x02, 0xb9, 0x7e, 0x0c, 0x46, 0x67, 0x1f, 0x28, 0x5c, 0xd1, - 0xe8, 0x47, 0x14, 0xc1, 0xa3, 0xb9, 0x83, 0xc4, 0x73, 0xac, 0xa6, 0x0b, 0x5b, 0x8b, 0xdc, 0x07, - 0xe4, 0xc6, 0xc2, 0x99, 0xdb, 0x4a, 0xf5, 0xe1, 0xd9, 0x99, 0xf2, 0x71, 0xe6, 0x04, 0x33, 0x29, - 0x79, 0x93, 0x50, 0xfa, 0x38, 0x33, 0x65, 0x80, 0x92, 0x6e, 0x7b, 0xf8, 0xa1, 0xfa, 0x03, 0xe3, - 0x96, 0x77, 0x62, 0x72, 0x6c, 0xa0, 0x69, 0x4f, 0x54, 0x29, 0x2f, 0x7f, 0x08, 0xfa, 0x47, 0x35, - 0xf8, 0xbc, 0x76, 0x9c, 0x74, 0xaa, 0xdd, 0xb1, 0x2d, 0xe3, 0x76, 0x35, 0xfe, 0x99, 0xf6, 0xd8, - 0xa7, 0xe8, 0x17, 0xb6, 0xdb, 0xfe, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x62, 0x8b, 0x12, 0xc0, 0xc1, - 0x3d, 0x00, 0x00, + 0x1d, 0xcf, 0x99, 0xa5, 0xa2, 0xa7, 0x3c, 0x96, 0x7f, 0x41, 0x60, 0x78, 0x95, 0xa1, 0xb0, 0xcb, + 0x05, 0xf6, 0x6e, 0x79, 0xb4, 0xb2, 0xda, 0x96, 0x2d, 0x0b, 0xb7, 0x50, 0x5e, 0x2e, 0xb4, 0xda, + 0x5b, 0xe3, 0xcd, 0xec, 0xbd, 0x87, 0x65, 0xe0, 0xde, 0x3b, 0xd7, 0x99, 0xd9, 0x6d, 0x37, 0x64, + 0xab, 0x29, 0x15, 0xb1, 0x06, 0x30, 0x41, 0x30, 0x8d, 0x2d, 0x31, 0x8d, 0x8d, 0xf5, 0x11, 0x3f, + 0x98, 0x68, 0x34, 0xd1, 0xc4, 0x0f, 0x92, 0x34, 0x6d, 0x4c, 0xda, 0xd4, 0x6a, 0xa3, 0xfd, 0x64, + 0x48, 0x68, 0xa2, 0x51, 0xab, 0x1f, 0xac, 0x5f, 0x8c, 0x99, 0xf3, 0x98, 0xf7, 0x99, 0xc7, 0x65, + 0x21, 0x24, 0xe5, 0xdb, 0xee, 0x3d, 0xff, 0x73, 0xce, 0xef, 0xf7, 0xfb, 0x3f, 0xce, 0x39, 0x33, + 0x67, 0xf0, 0xf2, 0xa3, 0xcd, 0x29, 0x87, 0x18, 0x8d, 0x66, 0xd9, 0x26, 0xd6, 0xa4, 0x51, 0x27, + 0x65, 0xbd, 0xd1, 0x32, 0xda, 0x03, 0x1d, 0xcb, 0x74, 0x4c, 0xe8, 0x15, 0xad, 0x03, 0xbc, 0x55, + 0x5d, 0x3e, 0x6e, 0x9a, 0xe3, 0x4d, 0x52, 0xd6, 0x3b, 0x46, 0x59, 0x6f, 0xb7, 0x4d, 0x47, 0x77, + 0x0c, 0xb3, 0x6d, 0x33, 0x7b, 0xd5, 0x1f, 0x8d, 0x8e, 0x52, 0xee, 0x58, 0xe6, 0x71, 0x52, 0x77, + 0x78, 0xeb, 0x40, 0x72, 0x6b, 0xad, 0x61, 0xb6, 0x74, 0xa3, 0x5d, 0xd3, 0x1d, 0xc7, 0x32, 0xc6, + 0x26, 0x1c, 0x22, 0x46, 0xeb, 0x93, 0xd8, 0xc7, 0x0c, 0x97, 0x46, 0x0c, 0x1d, 0xdd, 0x3e, 0xc1, + 0x9b, 0x56, 0x44, 0x9a, 0x9e, 0x32, 0xad, 0x13, 0x47, 0x9b, 0xe6, 0x53, 0xbc, 0xb9, 0x5f, 0xd2, + 0x1c, 0x9f, 0xe3, 0xee, 0x88, 0x65, 0x53, 0x9f, 0x68, 0xd7, 0x8f, 0xd5, 0x3a, 0x4d, 0x9d, 0x8b, + 0xa5, 0xaa, 0x11, 0x0b, 0x32, 0x49, 0xda, 0x82, 0xfa, 0xca, 0x68, 0xdb, 0xd3, 0xa4, 0x3e, 0xe1, + 0x2a, 0x27, 0xa1, 0xda, 0xd2, 0x9d, 0xfa, 0x31, 0x7d, 0xac, 0x49, 0x6a, 0x16, 0xb1, 0xcd, 0x09, + 0xab, 0x4e, 0xb8, 0xe1, 0x9a, 0x88, 0x61, 0xdb, 0x6c, 0x90, 0x5a, 0x74, 0xb4, 0x35, 0x09, 0x7a, + 0xc4, 0x8c, 0xa2, 0xbe, 0x9a, 0x24, 0x96, 0xed, 0xb7, 0x2e, 0x8b, 0xb4, 0xd6, 0xcd, 0x56, 0x4b, + 0x8a, 0xb6, 0x41, 0xec, 0xba, 0x65, 0x74, 0xdc, 0xc1, 0x6b, 0xa4, 0xed, 0x18, 0xce, 0x14, 0x33, + 0xdc, 0xfc, 0x8b, 0x29, 0x3c, 0x67, 0xd8, 0x35, 0x39, 0xcc, 0xc2, 0x07, 0xce, 0x21, 0x8c, 0x77, + 0x5a, 0x44, 0x77, 0xc8, 0x11, 0xdd, 0x3e, 0x01, 0xab, 0xbd, 0x90, 0x18, 0x60, 0x61, 0xe7, 0xfe, + 0xca, 0xda, 0x47, 0xc9, 0x97, 0x26, 0x88, 0xed, 0xa8, 0x5a, 0x9a, 0x89, 0xdd, 0x31, 0xdb, 0x36, + 0xd1, 0xb6, 0x3f, 0xfb, 0xf6, 0xd5, 0x0b, 0xca, 0x16, 0x6d, 0x2e, 0x0d, 0xcb, 0xc9, 0x7b, 0x29, + 0x61, 0x7b, 0x08, 0x95, 0xaa, 0xab, 0x34, 0x35, 0xf4, 0x5b, 0xd9, 0xb4, 0xc6, 0xcb, 0x27, 0x8d, + 0xc6, 0x80, 0x69, 0x8d, 0x4f, 0x0f, 0xa1, 0x12, 0xbc, 0x8b, 0xf0, 0xec, 0x0a, 0x71, 0x28, 0x9a, + 0xbb, 0xa3, 0x53, 0x1d, 0x1c, 0x73, 0xe3, 0xad, 0x42, 0x1c, 0x01, 0x66, 0x61, 0x12, 0x18, 0xed, + 0x5b, 0x88, 0xce, 0x7f, 0x16, 0xc1, 0x03, 0xe1, 0xc9, 0xdc, 0x89, 0x78, 0xd0, 0x4e, 0xd3, 0x7f, + 0x58, 0xa4, 0xb3, 0xbf, 0xdb, 0x7a, 0x8b, 0xb0, 0xbf, 0xb8, 0xf2, 0xd3, 0xd5, 0x7d, 0xb0, 0x57, + 0x8e, 0xb6, 0xe8, 0x68, 0x70, 0x05, 0xe1, 0x3b, 0xf7, 0x19, 0x36, 0xe5, 0xb6, 0xa7, 0x61, 0xc3, + 0x60, 0x14, 0xfc, 0x01, 0xbd, 0x45, 0x1a, 0xbb, 0xa8, 0xbf, 0xf6, 0x34, 0x5c, 0xbf, 0x1d, 0x35, + 0x88, 0xe5, 0xf6, 0x10, 0x74, 0xd7, 0xe7, 0xee, 0xa1, 0x3d, 0x41, 0x25, 0x38, 0x0c, 0x6b, 0x82, + 0x04, 0x6a, 0x46, 0xc3, 0x2e, 0x9f, 0xf4, 0x31, 0x73, 0xc0, 0xd5, 0x8d, 0x50, 0x4a, 0xe0, 0xc9, + 0x48, 0xc6, 0xac, 0xe1, 0x55, 0x05, 0x7f, 0x42, 0xf0, 0xb0, 0x61, 0x4d, 0x14, 0xd3, 0x28, 0x4f, + 0x90, 0x20, 0xf0, 0x25, 0x49, 0x7e, 0xa2, 0x38, 0xff, 0xcc, 0x7c, 0xf5, 0x7b, 0x04, 0x83, 0x45, + 0x7d, 0x55, 0x7d, 0x28, 0xea, 0xdf, 0x82, 0xee, 0xa9, 0xf6, 0xc3, 0xba, 0x7c, 0x93, 0x56, 0xb7, + 0xc1, 0x96, 0x2e, 0xa6, 0x82, 0x97, 0x11, 0x9e, 0xc7, 0x72, 0xe3, 0x73, 0xbc, 0x90, 0xc1, 0xda, + 0xa8, 0x14, 0xa2, 0x25, 0x9c, 0x66, 0xeb, 0xb2, 0xcc, 0x78, 0xaa, 0x0d, 0x53, 0xf9, 0x3e, 0xad, + 0x2d, 0x10, 0xf0, 0x44, 0xc5, 0xa4, 0xe9, 0xb6, 0x46, 0x5b, 0x19, 0xfb, 0x3d, 0x96, 0x72, 0xef, + 0x23, 0x7c, 0x67, 0x85, 0x38, 0x1e, 0xc2, 0xec, 0xb4, 0x5b, 0x22, 0x03, 0xa7, 0xbd, 0xc8, 0xdc, + 0x79, 0x11, 0xc1, 0x70, 0x7c, 0xe2, 0xa2, 0xe9, 0x77, 0x08, 0x0e, 0xa4, 0xa3, 0x2f, 0x9c, 0x82, + 0x6f, 0x21, 0x3c, 0xdf, 0x0d, 0x3b, 0x81, 0xf7, 0x86, 0xa7, 0xa1, 0x4e, 0xe5, 0x78, 0x12, 0xfa, + 0xa2, 0x44, 0x64, 0xa9, 0x38, 0x08, 0x03, 0x12, 0xce, 0xb2, 0x74, 0xfc, 0x8d, 0x82, 0xe7, 0x06, + 0x39, 0xe5, 0x4c, 0xc9, 0xe5, 0x32, 0x1f, 0x52, 0xdc, 0x7f, 0x65, 0x7e, 0xbc, 0x8a, 0x60, 0x6b, + 0x37, 0x7e, 0xac, 0xee, 0x4c, 0xf2, 0x7f, 0xd1, 0xf4, 0x0c, 0x94, 0xa5, 0xec, 0xc9, 0xab, 0xdb, + 0xe1, 0xfe, 0x2e, 0xa7, 0x84, 0x9f, 0x20, 0xdc, 0xcb, 0xf2, 0x6a, 0x1f, 0xdd, 0x45, 0x1c, 0x6a, + 0xea, 0x6d, 0xe8, 0x8b, 0x0a, 0xe4, 0xb7, 0x85, 0x53, 0xb5, 0x3f, 0xdb, 0x90, 0x27, 0x6b, 0x85, + 0x8a, 0x3a, 0xac, 0x2d, 0x14, 0x40, 0x03, 0x9b, 0x16, 0x9a, 0xaf, 0xeb, 0xb4, 0xd5, 0x49, 0x4d, + 0xb1, 0x94, 0xfd, 0x17, 0xc2, 0x73, 0x2b, 0xc4, 0x09, 0xa0, 0xcd, 0x4e, 0x5a, 0x55, 0x0e, 0x53, + 0x7b, 0x99, 0xb9, 0xfb, 0x3b, 0x08, 0x46, 0x12, 0xe7, 0x2f, 0x9a, 0xb9, 0x87, 0xe1, 0xb3, 0x99, + 0x3c, 0x0a, 0x27, 0xef, 0xbf, 0x11, 0xbe, 0xab, 0x42, 0x9c, 0xe1, 0xba, 0x63, 0x4c, 0xa6, 0x3a, + 0x2a, 0x6a, 0x91, 0x47, 0x81, 0x4b, 0x4c, 0x81, 0xf3, 0x08, 0x1e, 0x14, 0xc8, 0x75, 0x3a, 0x4c, + 0xad, 0xa0, 0x10, 0xd5, 0xfd, 0xf0, 0x68, 0xda, 0x08, 0x05, 0x25, 0x70, 0x37, 0x44, 0x8b, 0xdc, + 0x94, 0x8c, 0x92, 0xb2, 0x61, 0x43, 0x16, 0xef, 0x60, 0xba, 0xaf, 0x94, 0x73, 0xa7, 0x09, 0x6f, + 0x51, 0xfa, 0x4d, 0xbf, 0xfa, 0x24, 0xb2, 0x8f, 0xd7, 0xab, 0x21, 0xf8, 0x54, 0x26, 0x5b, 0x59, + 0xe5, 0x7a, 0x0f, 0xe1, 0x05, 0xee, 0xe4, 0x3e, 0x94, 0x1b, 0x5e, 0x8f, 0x8f, 0x53, 0x9a, 0x0d, + 0xbf, 0xb0, 0x04, 0xd0, 0xca, 0x4a, 0xf2, 0x7d, 0x7e, 0x0d, 0x8c, 0x5a, 0xa7, 0xd2, 0x7b, 0x53, + 0x61, 0x8b, 0x4d, 0xd0, 0x67, 0xb9, 0x4a, 0x73, 0x96, 0xaf, 0xfe, 0xc3, 0x62, 0xf5, 0x9f, 0xc8, + 0xaf, 0x78, 0x45, 0x83, 0x74, 0xb7, 0x24, 0xd1, 0x8b, 0x96, 0xe8, 0x32, 0x6c, 0x2a, 0x04, 0xa1, + 0xfa, 0x19, 0x18, 0xea, 0x7e, 0x62, 0x38, 0xab, 0xe0, 0xde, 0xc7, 0x3a, 0x8d, 0xdc, 0x85, 0x9a, + 0xd9, 0xe6, 0x28, 0xd4, 0xc2, 0x90, 0x17, 0xea, 0x57, 0x98, 0xc2, 0x97, 0x91, 0x3a, 0x23, 0xf5, + 0xd0, 0x2d, 0xed, 0x87, 0xd5, 0x1b, 0x50, 0x12, 0xbf, 0x87, 0xf0, 0x7c, 0xb6, 0xc6, 0xec, 0x12, + 0xa7, 0x49, 0x88, 0xed, 0x1c, 0xbd, 0xa6, 0xf0, 0xb2, 0xd5, 0x97, 0x69, 0xc7, 0xc5, 0xd8, 0x41, + 0xb5, 0x18, 0xd2, 0x40, 0xe0, 0xf7, 0x4e, 0xae, 0x74, 0xcd, 0x5a, 0xad, 0x2e, 0x8f, 0x37, 0xf8, + 0x99, 0xe1, 0x2e, 0x57, 0xbf, 0x46, 0x78, 0xc1, 0x28, 0x61, 0xb4, 0x7d, 0xa0, 0xfd, 0x52, 0x00, + 0xc2, 0xb6, 0x30, 0xd4, 0xc7, 0x28, 0xd4, 0x83, 0xda, 0xb2, 0x04, 0x44, 0x16, 0x1f, 0xd4, 0xc5, + 0xbc, 0x49, 0xeb, 0x97, 0x61, 0x16, 0xae, 0x08, 0x98, 0xc3, 0x2f, 0x11, 0xee, 0x1d, 0x25, 0x75, + 0x73, 0x92, 0x58, 0x3e, 0xfc, 0xbe, 0x14, 0xf8, 0xd4, 0xb4, 0x30, 0xfa, 0xc3, 0x14, 0xfd, 0x7e, + 0xff, 0x88, 0x1c, 0x42, 0x4f, 0xc7, 0x74, 0xc1, 0x6f, 0xd4, 0xfa, 0xb2, 0xc1, 0x0b, 0x6b, 0xb7, + 0xc8, 0xce, 0xa9, 0x10, 0xc7, 0xc7, 0xbd, 0x41, 0xb6, 0xf1, 0xf3, 0x4c, 0x02, 0x9b, 0x86, 0xa5, + 0x52, 0xec, 0xda, 0x29, 0x96, 0x23, 0xd3, 0xb0, 0x2d, 0x01, 0x4d, 0x8e, 0xd2, 0x31, 0x02, 0x0f, + 0x67, 0xd1, 0xc8, 0xb1, 0x3c, 0xfe, 0x03, 0xe1, 0xf9, 0x2c, 0x79, 0xf3, 0x64, 0x40, 0xb8, 0x1e, + 0xf4, 0x65, 0xda, 0x71, 0xc7, 0x3c, 0xcf, 0xa8, 0x9e, 0x42, 0x6a, 0x77, 0x5c, 0x5d, 0xaf, 0x55, + 0xd4, 0x19, 0xa0, 0xeb, 0x3a, 0xf4, 0xb4, 0x82, 0x7b, 0x83, 0x0e, 0x1d, 0xd1, 0x1d, 0x1d, 0xca, + 0x79, 0x9c, 0xea, 0x5a, 0x0a, 0xee, 0x83, 0xf9, 0x3b, 0x70, 0x11, 0xce, 0x31, 0x11, 0xbe, 0x86, + 0xfc, 0x0a, 0xde, 0xd0, 0x1d, 0xbd, 0xa0, 0xd7, 0xf7, 0x40, 0x45, 0xd6, 0xbb, 0xa8, 0xeb, 0xaf, + 0x20, 0x3c, 0xcf, 0x5d, 0x0f, 0x3d, 0xc4, 0x39, 0x97, 0xd7, 0x15, 0x52, 0xb7, 0xd3, 0xd5, 0xd5, + 0xa4, 0x34, 0x0d, 0xd8, 0x50, 0xc0, 0xd5, 0xc1, 0x6d, 0x50, 0x51, 0x46, 0xf0, 0x21, 0xc2, 0x70, + 0x84, 0x58, 0x2d, 0xa3, 0x1d, 0x8a, 0xe2, 0xf5, 0x52, 0x98, 0x9e, 0xb1, 0x60, 0x54, 0xca, 0x63, + 0x1a, 0x8d, 0xe5, 0xd2, 0x75, 0xc4, 0x72, 0x69, 0x86, 0x62, 0xf9, 0x2f, 0x2c, 0x96, 0x0f, 0x98, + 0x0d, 0x92, 0x52, 0x58, 0x43, 0xcd, 0x81, 0xe2, 0xb4, 0x22, 0xd5, 0x50, 0xfb, 0xaa, 0x42, 0x99, + 0xfe, 0x0f, 0x41, 0x5b, 0xa0, 0x0d, 0x3f, 0x9d, 0x65, 0x74, 0xbd, 0x7f, 0x6b, 0x51, 0xc8, 0xa1, + 0x96, 0x20, 0xfe, 0x50, 0x83, 0xbf, 0x2c, 0xd3, 0xd1, 0x8d, 0xc6, 0x74, 0xf5, 0x1b, 0x08, 0xce, + 0x20, 0xd9, 0x9c, 0x42, 0xa6, 0xd0, 0x20, 0x9e, 0x66, 0x33, 0x8f, 0x06, 0x2e, 0xf7, 0xe0, 0x4f, + 0xba, 0xc9, 0x3c, 0xd5, 0xd6, 0x5b, 0x46, 0xdd, 0xd5, 0xc8, 0x7b, 0xd4, 0xb3, 0x29, 0xaa, 0x60, + 0xb2, 0x9d, 0x10, 0x3c, 0xb6, 0x74, 0x24, 0xda, 0xf2, 0x40, 0xfb, 0x3e, 0x93, 0xff, 0xbb, 0x0a, + 0x7c, 0xf9, 0xe6, 0xca, 0x5f, 0x6e, 0x30, 0x54, 0x35, 0xf1, 0x18, 0xa0, 0xfa, 0x22, 0x82, 0x17, + 0x6e, 0x19, 0x7f, 0xc4, 0xe0, 0xc1, 0x7f, 0x15, 0x0c, 0x6e, 0xe9, 0x09, 0x85, 0xaf, 0x1d, 0xdf, + 0x1f, 0x85, 0xda, 0x83, 0x15, 0x6d, 0x75, 0xa6, 0xa5, 0x76, 0x91, 0x79, 0xe3, 0x9c, 0x02, 0xb6, + 0xd4, 0x1b, 0xde, 0xb3, 0x29, 0x09, 0xe9, 0xe4, 0x76, 0x8f, 0x7a, 0x72, 0x33, 0x2b, 0xfb, 0x97, + 0x10, 0x5c, 0x48, 0xf7, 0x40, 0x72, 0x6f, 0xe6, 0x87, 0x1b, 0x07, 0x0c, 0xce, 0x7f, 0x1c, 0x2f, + 0x8d, 0x6b, 0xbf, 0xdb, 0xb4, 0xe8, 0x1b, 0x88, 0x72, 0xaa, 0xb0, 0xdc, 0xaa, 0xa0, 0x27, 0x5e, + 0x9a, 0x4d, 0x3d, 0x71, 0x69, 0x36, 0xfc, 0xa0, 0x47, 0x28, 0x52, 0x3f, 0x66, 0x34, 0x1b, 0x16, + 0x89, 0xbe, 0x19, 0xb2, 0xcb, 0x27, 0xc3, 0x3f, 0xd4, 0x44, 0x1c, 0x85, 0x7e, 0x91, 0xa8, 0x52, + 0xb8, 0xab, 0x27, 0x58, 0xe1, 0x9e, 0x3c, 0xca, 0xf3, 0xf4, 0xf3, 0xd2, 0x20, 0xc1, 0x9a, 0xbf, + 0xa3, 0x48, 0xe5, 0x20, 0x6c, 0x52, 0xc0, 0x0a, 0x13, 0x29, 0x2a, 0x61, 0x20, 0xce, 0x51, 0x49, + 0x36, 0x16, 0x71, 0xac, 0xa9, 0x9a, 0xee, 0x38, 0xa4, 0xd5, 0x71, 0xa6, 0xab, 0xd7, 0x7a, 0xe0, + 0x6a, 0xb6, 0xbb, 0x68, 0x24, 0x17, 0x56, 0x8f, 0x05, 0xf9, 0x6d, 0x4f, 0xdf, 0x12, 0x9e, 0x86, + 0x53, 0x3d, 0x78, 0x61, 0x74, 0x4f, 0x42, 0xf7, 0xd8, 0x1b, 0xb2, 0xf6, 0x25, 0xc1, 0xfd, 0xf5, + 0xc6, 0x7c, 0xc6, 0x7c, 0xad, 0x3c, 0xcf, 0xaa, 0xf3, 0x99, 0x40, 0x75, 0xa6, 0xbb, 0xe3, 0x9b, + 0xb4, 0x5f, 0xf9, 0x26, 0x82, 0xb3, 0x28, 0x75, 0xe2, 0x9b, 0xbe, 0x69, 0x79, 0x15, 0xe1, 0xf9, + 0xa3, 0x64, 0xdc, 0xb0, 0x1d, 0x62, 0x1d, 0x62, 0x23, 0xc6, 0xcf, 0x75, 0xbc, 0x41, 0xd8, 0x49, + 0xcf, 0x75, 0x31, 0x3b, 0x2e, 0xfb, 0x08, 0x55, 0xfd, 0x41, 0xad, 0x57, 0x70, 0xe7, 0xd8, 0xe9, + 0x73, 0x8d, 0x3e, 0x4d, 0x8b, 0xfe, 0xcc, 0x64, 0x10, 0x97, 0x28, 0xc4, 0xd3, 0x8d, 0x6f, 0x23, + 0x3c, 0x97, 0x1d, 0x18, 0x05, 0xd0, 0xc5, 0x12, 0x00, 0xea, 0x5a, 0x49, 0x43, 0xe4, 0xbc, 0xb9, + 0x9b, 0xe2, 0xda, 0xa1, 0x2e, 0x8a, 0x01, 0x38, 0x69, 0x34, 0xe8, 0x0e, 0xfc, 0x1e, 0x75, 0x55, + 0x32, 0x38, 0xe1, 0x13, 0x8a, 0xec, 0x39, 0x84, 0xe7, 0xb8, 0x6b, 0x0e, 0x9f, 0xc5, 0x06, 0x4d, + 0x32, 0x7f, 0xea, 0xbb, 0x5a, 0xd1, 0x5b, 0xbb, 0x8f, 0xc2, 0x1a, 0x84, 0x98, 0x5c, 0xd5, 0xe5, + 0xa0, 0xca, 0xe1, 0xc0, 0x3b, 0x08, 0xdf, 0x15, 0x7e, 0x0b, 0xba, 0x6b, 0x92, 0xb4, 0x9d, 0xf8, + 0xee, 0x33, 0x76, 0x06, 0xa5, 0x76, 0x02, 0xd8, 0x40, 0x5e, 0x73, 0xae, 0x62, 0x8d, 0xc2, 0x7d, + 0x42, 0x5b, 0xe2, 0x9d, 0x55, 0xdc, 0x66, 0x3b, 0xfc, 0x86, 0x74, 0x48, 0xdb, 0x16, 0x69, 0xa6, + 0xb8, 0xd9, 0x5d, 0x91, 0x78, 0xb4, 0x07, 0xfb, 0xc2, 0x6f, 0xbd, 0xa7, 0x6f, 0x34, 0xb9, 0x29, + 0xa7, 0xf5, 0xa9, 0x79, 0x1f, 0xe2, 0x53, 0xca, 0x63, 0xca, 0xb9, 0x54, 0x29, 0x97, 0x23, 0xfe, + 0x9b, 0x23, 0x0e, 0xd6, 0xcd, 0x17, 0xca, 0x63, 0xbb, 0xb6, 0x55, 0xce, 0x23, 0x31, 0x71, 0x45, + 0x57, 0xf8, 0x93, 0x47, 0xc3, 0xdd, 0xcd, 0x48, 0x68, 0xd0, 0xa6, 0x7c, 0x34, 0x92, 0x4c, 0x39, + 0x8d, 0x0e, 0xa5, 0x71, 0x3c, 0x46, 0xc3, 0xbb, 0x1f, 0x72, 0x50, 0xdb, 0x2b, 0xa7, 0xd1, 0xd1, + 0x2d, 0xd2, 0x76, 0x6a, 0x79, 0x56, 0x53, 0x31, 0x20, 0xbc, 0x71, 0x07, 0x3d, 0x61, 0x86, 0x30, + 0xc5, 0x4f, 0x98, 0xa1, 0xe6, 0xb4, 0x13, 0x66, 0xc8, 0x50, 0xfb, 0xfb, 0x2c, 0xca, 0xe7, 0xda, + 0x2c, 0x78, 0x41, 0x09, 0xdd, 0xb3, 0x88, 0x94, 0xec, 0xdc, 0x0b, 0x79, 0x81, 0x95, 0x3b, 0xf7, + 0x52, 0x9d, 0xb1, 0x36, 0x27, 0x2e, 0xc6, 0x49, 0xab, 0x6f, 0x7c, 0xb9, 0x4d, 0x5c, 0x5f, 0xe3, + 0x5b, 0xa7, 0xd7, 0x14, 0xb8, 0x22, 0x95, 0x47, 0x2c, 0x2c, 0xb9, 0xb6, 0x48, 0x1f, 0x75, 0x29, + 0xe1, 0x83, 0x1e, 0x76, 0x52, 0x0c, 0x85, 0x61, 0xc2, 0x49, 0x31, 0xd4, 0x9e, 0x7a, 0x3e, 0x89, + 0x59, 0x6a, 0x67, 0x7b, 0x68, 0x50, 0x9f, 0xee, 0x81, 0x1f, 0x22, 0x69, 0x50, 0xe7, 0x76, 0x43, + 0x5e, 0x1f, 0xe4, 0x73, 0x80, 0x5c, 0xfd, 0xea, 0x1f, 0x10, 0xbc, 0x8d, 0x52, 0xa3, 0x2c, 0x57, + 0x88, 0xdd, 0x72, 0xc4, 0xe0, 0x6f, 0x77, 0xd0, 0xfd, 0x68, 0xc8, 0x51, 0xc9, 0xfb, 0xd1, 0x68, + 0x15, 0x4b, 0xdd, 0x8f, 0x26, 0x1b, 0xf3, 0x3a, 0xfd, 0x21, 0x2b, 0x6c, 0x1f, 0xcc, 0x82, 0xcb, + 0x4a, 0x68, 0x5f, 0x78, 0xbb, 0xba, 0x45, 0xab, 0xdb, 0xef, 0x14, 0x78, 0x3d, 0x5d, 0xa3, 0xdb, + 0x25, 0xae, 0x48, 0x89, 0x7b, 0x43, 0xc1, 0x2b, 0x42, 0xbb, 0xe9, 0x11, 0x3a, 0xe4, 0xb0, 0x77, + 0xff, 0x17, 0xb6, 0x4a, 0x36, 0xa8, 0x51, 0xc3, 0xf0, 0xcb, 0x9e, 0x6d, 0x05, 0x7b, 0xf1, 0x4c, + 0xf8, 0x15, 0x7b, 0x5c, 0xfe, 0x33, 0xa4, 0xee, 0x8d, 0xec, 0x70, 0xe3, 0xd7, 0xa4, 0xcb, 0x27, + 0xc3, 0xb7, 0x94, 0xb9, 0x3c, 0x81, 0x1f, 0xb9, 0x3c, 0xee, 0x4e, 0xa7, 0xa1, 0xd6, 0xb2, 0x07, + 0xa4, 0xa1, 0x13, 0xe8, 0xcf, 0x62, 0x24, 0xff, 0x2c, 0x70, 0x4a, 0xc1, 0x6a, 0x85, 0x38, 0x32, + 0x29, 0xef, 0xcd, 0x29, 0x4a, 0x60, 0x4b, 0xb4, 0xb9, 0x48, 0x17, 0x2e, 0xe2, 0x33, 0x54, 0xc3, + 0xa7, 0xfd, 0x7b, 0x14, 0x29, 0x12, 0xc6, 0xef, 0x5f, 0xec, 0xf0, 0xaf, 0xe4, 0x64, 0x28, 0x25, + 0xbb, 0x89, 0x71, 0x41, 0xc1, 0x2b, 0x46, 0x48, 0x93, 0x5c, 0x7f, 0x4c, 0xb1, 0x51, 0x8a, 0xc6, + 0x94, 0xe8, 0xc5, 0xe5, 0x78, 0x8e, 0xc5, 0xd4, 0x33, 0xa5, 0xae, 0xf4, 0x70, 0x83, 0x67, 0x67, + 0xe9, 0x3a, 0x25, 0x71, 0x63, 0xe3, 0x79, 0x05, 0x2f, 0x0e, 0x65, 0x5a, 0x40, 0x8f, 0x01, 0x09, + 0x33, 0x59, 0x76, 0x95, 0x73, 0xdb, 0x73, 0x0d, 0xce, 0x32, 0x0d, 0x4e, 0x23, 0xb5, 0x1c, 0x65, + 0x92, 0x91, 0x50, 0x2e, 0xff, 0x43, 0xea, 0xa3, 0x33, 0x98, 0x3c, 0xfc, 0xde, 0xf9, 0x42, 0x3f, + 0x51, 0x02, 0x4a, 0x6c, 0xc8, 0x64, 0x16, 0x48, 0x8e, 0x8d, 0xf9, 0x8c, 0xb9, 0x06, 0x84, 0x4a, + 0x50, 0x83, 0x7b, 0xd2, 0x14, 0x10, 0x20, 0xab, 0xf7, 0xfb, 0x2f, 0xda, 0x0b, 0xf9, 0x1c, 0xae, + 0x21, 0xbc, 0x38, 0x14, 0xfd, 0x85, 0xfc, 0x1c, 0x8e, 0xf8, 0x72, 0x6e, 0x7b, 0xce, 0xf1, 0x04, + 0xe5, 0x48, 0x4a, 0xb9, 0x38, 0xd2, 0x03, 0x79, 0xa9, 0x3b, 0x9a, 0xf4, 0xae, 0x83, 0x82, 0x97, + 0xb0, 0x38, 0x13, 0x8f, 0x06, 0x02, 0x54, 0xa5, 0xaf, 0xc8, 0x65, 0x31, 0x3d, 0x98, 0xbf, 0x03, + 0x27, 0xfb, 0x1e, 0x0b, 0xea, 0x77, 0x90, 0x5a, 0x8d, 0xdd, 0xf7, 0xed, 0x62, 0x99, 0x08, 0xfd, + 0x26, 0x06, 0xa2, 0x22, 0x39, 0xaa, 0x99, 0x36, 0xc1, 0x75, 0x2d, 0x1b, 0xd2, 0x59, 0xe1, 0x2b, + 0x0a, 0x5e, 0x14, 0xb8, 0x28, 0x1e, 0xd0, 0x76, 0x63, 0xb6, 0x54, 0x81, 0x2c, 0xd9, 0x94, 0xd3, + 0x9a, 0xab, 0xfa, 0x75, 0xa6, 0xea, 0xb3, 0x08, 0xb6, 0xa7, 0xaa, 0x1a, 0x2b, 0x72, 0xfe, 0x9b, + 0x9f, 0xe9, 0xea, 0x2e, 0xd8, 0x99, 0xa9, 0x98, 0xa4, 0x56, 0x06, 0x86, 0x81, 0xf3, 0x0a, 0x5e, + 0xc2, 0x22, 0xbc, 0xbb, 0x08, 0x0b, 0x67, 0xd3, 0x60, 0xfe, 0x0e, 0xe2, 0x41, 0x31, 0xd3, 0xe2, + 0x0c, 0x2a, 0x75, 0xaf, 0x85, 0x1b, 0x40, 0x8f, 0x94, 0x66, 0x42, 0x0e, 0x37, 0x28, 0x5e, 0x47, + 0x78, 0xb1, 0x7b, 0x6e, 0xdc, 0x2f, 0xbe, 0x90, 0x4a, 0xab, 0x2e, 0x12, 0x43, 0x69, 0x75, 0x91, + 0xda, 0x73, 0x39, 0x0e, 0x50, 0x35, 0x1e, 0x01, 0xef, 0x06, 0x9a, 0xff, 0x9d, 0x96, 0xcf, 0x26, + 0xf8, 0x99, 0x47, 0x52, 0x7b, 0xe0, 0x49, 0xe5, 0xfb, 0xfc, 0x46, 0xaa, 0x7f, 0x81, 0xd4, 0x20, + 0x76, 0xfc, 0xb9, 0x73, 0xe0, 0x7e, 0x69, 0xf0, 0x70, 0xbd, 0x2a, 0xc3, 0x2e, 0x29, 0x92, 0xdd, + 0xcd, 0x71, 0x83, 0x7d, 0xa4, 0x65, 0xb8, 0x8e, 0x13, 0x9f, 0x97, 0xd5, 0x9c, 0xa9, 0x0e, 0x49, + 0x92, 0x3f, 0x18, 0xc9, 0x19, 0x9d, 0x53, 0xf7, 0x44, 0xa7, 0x14, 0x3c, 0xaf, 0x42, 0x02, 0x4c, + 0xa7, 0xe2, 0x9f, 0xa6, 0x04, 0x1a, 0x03, 0xe9, 0xbb, 0x2c, 0xc5, 0x4c, 0xfb, 0x31, 0xa3, 0xf8, + 0x0a, 0x82, 0x3d, 0x79, 0x29, 0x66, 0x5f, 0x1a, 0xfa, 0x3c, 0x3c, 0x5e, 0x84, 0x72, 0x81, 0x3b, + 0x44, 0x2f, 0x29, 0x78, 0x01, 0x2b, 0xe2, 0x41, 0x21, 0xfa, 0x53, 0x18, 0x86, 0xd7, 0x88, 0xf5, + 0x39, 0x2c, 0x79, 0xac, 0xfe, 0x94, 0x29, 0xf3, 0x23, 0xa4, 0xce, 0x9c, 0x32, 0x6e, 0x2a, 0x3f, + 0xa9, 0xde, 0x20, 0x71, 0xdc, 0xec, 0x6e, 0x62, 0x5c, 0x21, 0xce, 0xe3, 0xec, 0xbc, 0x16, 0xff, + 0x3c, 0xd0, 0x6f, 0x93, 0x7e, 0x1e, 0x18, 0x34, 0xe1, 0x4a, 0x2c, 0xa6, 0x42, 0x2c, 0x80, 0xf9, + 0x02, 0x37, 0x3f, 0x0f, 0xc2, 0xcf, 0x15, 0xba, 0x09, 0x1b, 0xf1, 0x3f, 0x5f, 0xe4, 0x0e, 0xc9, + 0xfe, 0xba, 0x21, 0x06, 0x2d, 0x36, 0x88, 0xf6, 0x2e, 0x73, 0xc0, 0x5b, 0x08, 0xbc, 0x93, 0x57, + 0xec, 0x43, 0x49, 0x2a, 0x1b, 0x3d, 0x94, 0x16, 0xf4, 0x44, 0xf8, 0xfb, 0x87, 0x36, 0x34, 0x53, + 0xa7, 0x88, 0x79, 0xe2, 0xfa, 0xe6, 0x83, 0xd7, 0x7a, 0x58, 0x11, 0x8e, 0x92, 0x36, 0x92, 0xd6, + 0xe6, 0x98, 0x32, 0xc1, 0x1a, 0xb6, 0x36, 0x97, 0xb5, 0xf6, 0x47, 0xf6, 0xc2, 0xf2, 0x4d, 0x05, + 0x0e, 0xa6, 0x6b, 0x59, 0x3c, 0xd9, 0xbf, 0x08, 0x5f, 0x28, 0x34, 0x64, 0xd1, 0x2b, 0xeb, 0x15, + 0xd8, 0x35, 0x23, 0x90, 0xab, 0x47, 0x60, 0x74, 0xe6, 0x81, 0xc2, 0x45, 0x85, 0x7e, 0xe5, 0xe2, + 0x3d, 0x9a, 0xdb, 0x4f, 0x1c, 0xcb, 0xa8, 0xdb, 0xb0, 0x39, 0xcf, 0x85, 0x4d, 0x6e, 0x2c, 0x9c, + 0xb9, 0xa5, 0x50, 0x1f, 0x9e, 0x9d, 0x09, 0x5f, 0xcf, 0xb6, 0x98, 0x49, 0xc1, 0xab, 0x9e, 0x81, + 0xaf, 0x67, 0x13, 0x06, 0x28, 0xe8, 0xb6, 0x87, 0x1f, 0xaa, 0x3e, 0x30, 0x6e, 0x38, 0xc7, 0x26, + 0xc6, 0x06, 0xea, 0x66, 0xab, 0x4c, 0x79, 0xb9, 0x43, 0xd0, 0x3f, 0xca, 0xde, 0xf7, 0xcf, 0xe3, + 0xa4, 0x5d, 0xee, 0x8c, 0x6d, 0x1a, 0x37, 0xcb, 0xd1, 0xef, 0xe8, 0xc7, 0x3e, 0x46, 0x3f, 0x81, + 0xde, 0xf2, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, 0x94, 0xbe, 0x0b, 0x03, 0x62, 0x3f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -270,6 +273,8 @@ type AdminServiceClient interface { TerminateExecution(ctx context.Context, in *admin.ExecutionTerminateRequest, opts ...grpc.CallOption) (*admin.ExecutionTerminateResponse, error) // Fetches a :ref:`ref_flyteidl.admin.NodeExecution`. GetNodeExecution(ctx context.Context, in *admin.NodeExecutionGetRequest, opts ...grpc.CallOption) (*admin.NodeExecution, error) + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + GetDynamicNodeWorkflow(ctx context.Context, in *admin.GetDynamicNodeWorkflowRequest, opts ...grpc.CallOption) (*admin.DynamicNodeWorkflowResponse, error) // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. ListNodeExecutions(ctx context.Context, in *admin.NodeExecutionListRequest, opts ...grpc.CallOption) (*admin.NodeExecutionList, error) // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution` launched by the reference :ref:`ref_flyteidl.admin.TaskExecution`. @@ -555,6 +560,15 @@ func (c *adminServiceClient) GetNodeExecution(ctx context.Context, in *admin.Nod return out, nil } +func (c *adminServiceClient) GetDynamicNodeWorkflow(ctx context.Context, in *admin.GetDynamicNodeWorkflowRequest, opts ...grpc.CallOption) (*admin.DynamicNodeWorkflowResponse, error) { + out := new(admin.DynamicNodeWorkflowResponse) + err := c.cc.Invoke(ctx, "/flyteidl.service.AdminService/GetDynamicNodeWorkflow", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *adminServiceClient) ListNodeExecutions(ctx context.Context, in *admin.NodeExecutionListRequest, opts ...grpc.CallOption) (*admin.NodeExecutionList, error) { out := new(admin.NodeExecutionList) err := c.cc.Invoke(ctx, "/flyteidl.service.AdminService/ListNodeExecutions", in, out, opts...) @@ -870,6 +884,8 @@ type AdminServiceServer interface { TerminateExecution(context.Context, *admin.ExecutionTerminateRequest) (*admin.ExecutionTerminateResponse, error) // Fetches a :ref:`ref_flyteidl.admin.NodeExecution`. GetNodeExecution(context.Context, *admin.NodeExecutionGetRequest) (*admin.NodeExecution, error) + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + GetDynamicNodeWorkflow(context.Context, *admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. ListNodeExecutions(context.Context, *admin.NodeExecutionListRequest) (*admin.NodeExecutionList, error) // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution` launched by the reference :ref:`ref_flyteidl.admin.TaskExecution`. @@ -1007,6 +1023,9 @@ func (*UnimplementedAdminServiceServer) TerminateExecution(ctx context.Context, func (*UnimplementedAdminServiceServer) GetNodeExecution(ctx context.Context, req *admin.NodeExecutionGetRequest) (*admin.NodeExecution, error) { return nil, status.Errorf(codes.Unimplemented, "method GetNodeExecution not implemented") } +func (*UnimplementedAdminServiceServer) GetDynamicNodeWorkflow(ctx context.Context, req *admin.GetDynamicNodeWorkflowRequest) (*admin.DynamicNodeWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetDynamicNodeWorkflow not implemented") +} func (*UnimplementedAdminServiceServer) ListNodeExecutions(ctx context.Context, req *admin.NodeExecutionListRequest) (*admin.NodeExecutionList, error) { return nil, status.Errorf(codes.Unimplemented, "method ListNodeExecutions not implemented") } @@ -1531,6 +1550,24 @@ func _AdminService_GetNodeExecution_Handler(srv interface{}, ctx context.Context return interceptor(ctx, in, info, handler) } +func _AdminService_GetDynamicNodeWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(admin.GetDynamicNodeWorkflowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(AdminServiceServer).GetDynamicNodeWorkflow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/flyteidl.service.AdminService/GetDynamicNodeWorkflow", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(AdminServiceServer).GetDynamicNodeWorkflow(ctx, req.(*admin.GetDynamicNodeWorkflowRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _AdminService_ListNodeExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(admin.NodeExecutionListRequest) if err := dec(in); err != nil { @@ -2153,6 +2190,10 @@ var _AdminService_serviceDesc = grpc.ServiceDesc{ MethodName: "GetNodeExecution", Handler: _AdminService_GetNodeExecution_Handler, }, + { + MethodName: "GetDynamicNodeWorkflow", + Handler: _AdminService_GetDynamicNodeWorkflow_Handler, + }, { MethodName: "ListNodeExecutions", Handler: _AdminService_ListNodeExecutions_Handler, diff --git a/flyteidl/gen/pb-go/flyteidl/service/admin.pb.gw.go b/flyteidl/gen/pb-go/flyteidl/service/admin.pb.gw.go index 3bffe4954b..e0d3d0c8c8 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/admin.pb.gw.go +++ b/flyteidl/gen/pb-go/flyteidl/service/admin.pb.gw.go @@ -3012,6 +3012,159 @@ func request_AdminService_GetNodeExecution_1(ctx context.Context, marshaler runt } +var ( + filter_AdminService_GetDynamicNodeWorkflow_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0, "execution_id": 1, "project": 2, "domain": 3, "name": 4, "node_id": 5}, Base: []int{1, 6, 1, 1, 2, 2, 5, 0, 0, 4, 0, 6, 0}, Check: []int{0, 1, 2, 3, 2, 5, 2, 4, 6, 7, 10, 2, 12}} +) + +func request_AdminService_GetDynamicNodeWorkflow_0(ctx context.Context, marshaler runtime.Marshaler, client AdminServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq admin.GetDynamicNodeWorkflowRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id.execution_id.project"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.project") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.project", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.project", err) + } + + val, ok = pathParams["id.execution_id.domain"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.domain") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.domain", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.domain", err) + } + + val, ok = pathParams["id.execution_id.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.name", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.name", err) + } + + val, ok = pathParams["id.node_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.node_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.node_id", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.node_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_AdminService_GetDynamicNodeWorkflow_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetDynamicNodeWorkflow(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +var ( + filter_AdminService_GetDynamicNodeWorkflow_1 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0, "execution_id": 1, "org": 2, "project": 3, "domain": 4, "name": 5, "node_id": 6}, Base: []int{1, 8, 1, 1, 2, 2, 3, 3, 0, 0, 0, 7, 6, 0, 8, 0}, Check: []int{0, 1, 2, 3, 2, 5, 2, 7, 4, 6, 8, 2, 12, 13, 2, 15}} +) + +func request_AdminService_GetDynamicNodeWorkflow_1(ctx context.Context, marshaler runtime.Marshaler, client AdminServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq admin.GetDynamicNodeWorkflowRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id.execution_id.org"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.org") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.org", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.org", err) + } + + val, ok = pathParams["id.execution_id.project"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.project") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.project", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.project", err) + } + + val, ok = pathParams["id.execution_id.domain"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.domain") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.domain", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.domain", err) + } + + val, ok = pathParams["id.execution_id.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.execution_id.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.execution_id.name", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.execution_id.name", err) + } + + val, ok = pathParams["id.node_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id.node_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "id.node_id", val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id.node_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_AdminService_GetDynamicNodeWorkflow_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetDynamicNodeWorkflow(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + var ( filter_AdminService_ListNodeExecutions_0 = &utilities.DoubleArray{Encoding: map[string]int{"workflow_execution_id": 0, "project": 1, "domain": 2, "name": 3}, Base: []int{1, 1, 1, 2, 3, 0, 0, 0}, Check: []int{0, 1, 2, 2, 2, 3, 4, 5}} ) @@ -7772,6 +7925,46 @@ func RegisterAdminServiceHandlerClient(ctx context.Context, mux *runtime.ServeMu }) + mux.Handle("GET", pattern_AdminService_GetDynamicNodeWorkflow_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_AdminService_GetDynamicNodeWorkflow_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_AdminService_GetDynamicNodeWorkflow_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_AdminService_GetDynamicNodeWorkflow_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_AdminService_GetDynamicNodeWorkflow_1(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_AdminService_GetDynamicNodeWorkflow_1(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + mux.Handle("GET", pattern_AdminService_ListNodeExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { ctx, cancel := context.WithCancel(req.Context()) defer cancel() @@ -9064,6 +9257,10 @@ var ( pattern_AdminService_GetNodeExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4, 1, 0, 4, 1, 5, 5, 1, 0, 4, 1, 5, 6, 1, 0, 4, 1, 5, 7, 1, 0, 4, 1, 5, 8}, []string{"api", "v1", "node_executions", "org", "id.execution_id.org", "id.execution_id.project", "id.execution_id.domain", "id.execution_id.name", "id.node_id"}, "")) + pattern_AdminService_GetDynamicNodeWorkflow_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 1, 0, 4, 1, 5, 4, 1, 0, 4, 1, 5, 5, 1, 0, 4, 1, 5, 6, 2, 7}, []string{"api", "v1", "node_executions", "id.execution_id.project", "id.execution_id.domain", "id.execution_id.name", "id.node_id", "dynamic_workflow"}, "")) + + pattern_AdminService_GetDynamicNodeWorkflow_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4, 1, 0, 4, 1, 5, 5, 1, 0, 4, 1, 5, 6, 1, 0, 4, 1, 5, 7, 1, 0, 4, 1, 5, 8, 2, 9}, []string{"api", "v1", "node_executions", "org", "id.execution_id.org", "id.execution_id.project", "id.execution_id.domain", "id.execution_id.name", "id.node_id", "dynamic_workflow"}, "")) + pattern_AdminService_ListNodeExecutions_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 1, 0, 4, 1, 5, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "node_executions", "workflow_execution_id.project", "workflow_execution_id.domain", "workflow_execution_id.name"}, "")) pattern_AdminService_ListNodeExecutions_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4, 1, 0, 4, 1, 5, 5, 1, 0, 4, 1, 5, 6, 1, 0, 4, 1, 5, 7}, []string{"api", "v1", "node_executions", "org", "workflow_execution_id.org", "workflow_execution_id.project", "workflow_execution_id.domain", "workflow_execution_id.name"}, "")) @@ -9292,6 +9489,10 @@ var ( forward_AdminService_GetNodeExecution_1 = runtime.ForwardResponseMessage + forward_AdminService_GetDynamicNodeWorkflow_0 = runtime.ForwardResponseMessage + + forward_AdminService_GetDynamicNodeWorkflow_1 = runtime.ForwardResponseMessage + forward_AdminService_ListNodeExecutions_0 = runtime.ForwardResponseMessage forward_AdminService_ListNodeExecutions_1 = runtime.ForwardResponseMessage diff --git a/flyteidl/gen/pb-go/flyteidl/service/admin.swagger.json b/flyteidl/gen/pb-go/flyteidl/service/admin.swagger.json index e3661ae4f5..c2ef978af0 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/admin.swagger.json +++ b/flyteidl/gen/pb-go/flyteidl/service/admin.swagger.json @@ -3865,6 +3865,59 @@ ] } }, + "/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow": { + "get": { + "summary": "Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`.", + "operationId": "GetDynamicNodeWorkflow2", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/adminDynamicNodeWorkflowResponse" + } + } + }, + "parameters": [ + { + "name": "id.execution_id.org", + "description": "Optional, org key applied to the resource.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.project", + "description": "Name of the project the resource belongs to.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.domain", + "description": "Name of the domain the resource belongs to.\nA domain can be considered as a subset within a specific project.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.name", + "description": "User or system provided value for the resource.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.node_id", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "AdminService" + ] + } + }, "/api/v1/node_executions/org/{workflow_execution_id.org}/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}": { "get": { "summary": "Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`.", @@ -4012,6 +4065,59 @@ ] } }, + "/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow": { + "get": { + "summary": "Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`.", + "operationId": "GetDynamicNodeWorkflow", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/adminDynamicNodeWorkflowResponse" + } + } + }, + "parameters": [ + { + "name": "id.execution_id.project", + "description": "Name of the project the resource belongs to.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.domain", + "description": "Name of the domain the resource belongs to.\nA domain can be considered as a subset within a specific project.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.name", + "description": "User or system provided value for the resource.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.node_id", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "id.execution_id.org", + "description": "Optional, org key applied to the resource.", + "in": "query", + "required": false, + "type": "string" + } + ], + "tags": [ + "AdminService" + ] + } + }, "/api/v1/node_executions/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}": { "get": { "summary": "Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`.", @@ -7564,6 +7670,14 @@ }, "description": "Namespace within a project commonly used to differentiate between different service instances.\ne.g. \"production\", \"development\", etc." }, + "adminDynamicNodeWorkflowResponse": { + "type": "object", + "properties": { + "compiled_workflow": { + "$ref": "#/definitions/coreCompiledWorkflowClosure" + } + } + }, "adminEmailNotification": { "type": "object", "properties": { diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/README.md b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/README.md index 4809e080ec..17325ae3e7 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/README.md +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/README.md @@ -45,6 +45,8 @@ Class | Method | HTTP request | Description *AdminServiceApi* | [**GetActiveLaunchPlan2**](docs/AdminServiceApi.md#getactivelaunchplan2) | **Get** /api/v1/active_launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name} | Fetch the active version of a :ref:`ref_flyteidl.admin.LaunchPlan`. *AdminServiceApi* | [**GetDescriptionEntity**](docs/AdminServiceApi.md#getdescriptionentity) | **Get** /api/v1/description_entities/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version} | Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. *AdminServiceApi* | [**GetDescriptionEntity2**](docs/AdminServiceApi.md#getdescriptionentity2) | **Get** /api/v1/description_entities/org/{id.org}/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version} | Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. +*AdminServiceApi* | [**GetDynamicNodeWorkflow**](docs/AdminServiceApi.md#getdynamicnodeworkflow) | **Get** /api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow | Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. +*AdminServiceApi* | [**GetDynamicNodeWorkflow2**](docs/AdminServiceApi.md#getdynamicnodeworkflow2) | **Get** /api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow | Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. *AdminServiceApi* | [**GetExecution**](docs/AdminServiceApi.md#getexecution) | **Get** /api/v1/executions/{id.project}/{id.domain}/{id.name} | Fetches a :ref:`ref_flyteidl.admin.Execution`. *AdminServiceApi* | [**GetExecution2**](docs/AdminServiceApi.md#getexecution2) | **Get** /api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name} | Fetches a :ref:`ref_flyteidl.admin.Execution`. *AdminServiceApi* | [**GetExecutionData**](docs/AdminServiceApi.md#getexecutiondata) | **Get** /api/v1/data/executions/{id.project}/{id.domain}/{id.name} | Fetches input and output data for a :ref:`ref_flyteidl.admin.Execution`. @@ -150,6 +152,7 @@ Class | Method | HTTP request | Description - [AdminDescriptionEntityList](docs/AdminDescriptionEntityList.md) - [AdminDescriptionFormat](docs/AdminDescriptionFormat.md) - [AdminDomain](docs/AdminDomain.md) + - [AdminDynamicNodeWorkflowResponse](docs/AdminDynamicNodeWorkflowResponse.md) - [AdminEmailNotification](docs/AdminEmailNotification.md) - [AdminEnvs](docs/AdminEnvs.md) - [AdminExecution](docs/AdminExecution.md) diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api/swagger.yaml b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api/swagger.yaml index 3e787f292d..cf9d7579c4 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api/swagger.yaml +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api/swagger.yaml @@ -3396,6 +3396,48 @@ paths: description: "A successful response." schema: $ref: "#/definitions/flyteidladminNodeExecution" + ? /api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow + : get: + tags: + - "AdminService" + summary: "Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`." + operationId: "GetDynamicNodeWorkflow2" + parameters: + - name: "id.execution_id.org" + in: "path" + description: "Optional, org key applied to the resource." + required: true + type: "string" + x-exportParamName: "IdExecutionIdOrg" + - name: "id.execution_id.project" + in: "path" + description: "Name of the project the resource belongs to." + required: true + type: "string" + x-exportParamName: "IdExecutionIdProject" + - name: "id.execution_id.domain" + in: "path" + description: "Name of the domain the resource belongs to.\nA domain can be\ + \ considered as a subset within a specific project." + required: true + type: "string" + x-exportParamName: "IdExecutionIdDomain" + - name: "id.execution_id.name" + in: "path" + description: "User or system provided value for the resource." + required: true + type: "string" + x-exportParamName: "IdExecutionIdName" + - name: "id.node_id" + in: "path" + required: true + type: "string" + x-exportParamName: "IdNodeId" + responses: + 200: + description: "A successful response." + schema: + $ref: "#/definitions/adminDynamicNodeWorkflowResponse" ? /api/v1/node_executions/org/{workflow_execution_id.org}/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name} : get: tags: @@ -3525,6 +3567,49 @@ paths: description: "A successful response." schema: $ref: "#/definitions/flyteidladminNodeExecution" + /api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow: + get: + tags: + - "AdminService" + summary: "Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`." + operationId: "GetDynamicNodeWorkflow" + parameters: + - name: "id.execution_id.project" + in: "path" + description: "Name of the project the resource belongs to." + required: true + type: "string" + x-exportParamName: "IdExecutionIdProject" + - name: "id.execution_id.domain" + in: "path" + description: "Name of the domain the resource belongs to.\nA domain can be\ + \ considered as a subset within a specific project." + required: true + type: "string" + x-exportParamName: "IdExecutionIdDomain" + - name: "id.execution_id.name" + in: "path" + description: "User or system provided value for the resource." + required: true + type: "string" + x-exportParamName: "IdExecutionIdName" + - name: "id.node_id" + in: "path" + required: true + type: "string" + x-exportParamName: "IdNodeId" + - name: "id.execution_id.org" + in: "query" + description: "Optional, org key applied to the resource." + required: false + type: "string" + x-exportParamName: "IdExecutionIdOrg" + x-optionalDataType: "String" + responses: + 200: + description: "A successful response." + schema: + $ref: "#/definitions/adminDynamicNodeWorkflowResponse" /api/v1/node_executions/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}: get: tags: @@ -6849,6 +6934,12743 @@ definitions: example: name: "name" id: "id" + adminDynamicNodeWorkflowResponse: + type: "object" + properties: + compiled_workflow: + $ref: "#/definitions/coreCompiledWorkflowClosure" + example: + compiled_workflow: + sub_workflows: + - template: + outputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + metadata: + on_failure: {} + quality_of_service: + tier: {} + spec: + queueing_budget: "queueing_budget" + tags: + key: "tags" + failure_node: + branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + nodes: + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + metadata_defaults: + interruptible: true + id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + interface: + outputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + inputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + connections: + upstream: + key: + ids: + - "ids" + - "ids" + downstream: + key: + ids: + - "ids" + - "ids" + - template: + outputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + metadata: + on_failure: {} + quality_of_service: + tier: {} + spec: + queueing_budget: "queueing_budget" + tags: + key: "tags" + failure_node: + branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + nodes: + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + metadata_defaults: + interruptible: true + id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + interface: + outputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + inputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + connections: + upstream: + key: + ids: + - "ids" + - "ids" + downstream: + key: + ids: + - "ids" + - "ids" + tasks: + - template: + container: + args: + - "args" + - "args" + image: "image" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + data_config: + io_strategy: + upload_mode: {} + download_mode: {} + format: {} + output_path: "output_path" + enabled: true + input_path: "input_path" + env: + - value: "value" + key: "key" + - value: "value" + key: "key" + ports: + - container_port: 2 + - container_port: 2 + config: + - value: "value" + key: "key" + - value: "value" + key: "key" + command: + - "command" + - "command" + architecture: {} + metadata: + retries: + retries: 0 + pod_template_name: "pod_template_name" + discoverable: true + runtime: + flavor: "flavor" + type: {} + version: "version" + cache_serializable: true + discovery_version: "discovery_version" + deprecated_error_message: "deprecated_error_message" + interruptible: true + timeout: "timeout" + generates_deck: true + cache_ignore_input_vars: + - "cache_ignore_input_vars" + - "cache_ignore_input_vars" + tags: + key: "tags" + task_type_version: 7 + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + custom: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + k8s_pod: + metadata: + annotations: + key: "annotations" + labels: + key: "labels" + pod_spec: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + data_config: + io_strategy: + upload_mode: {} + download_mode: {} + format: {} + output_path: "output_path" + enabled: true + input_path: "input_path" + id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + type: "type" + interface: + outputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + inputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + config: + key: "config" + security_context: + run_as: + execution_identity: "execution_identity" + iam_role: "iam_role" + oauth2_client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + k8s_service_account: "k8s_service_account" + tokens: + - idp_discovery_endpoint: "idp_discovery_endpoint" + name: "name" + client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + type: {} + token_endpoint: "token_endpoint" + - idp_discovery_endpoint: "idp_discovery_endpoint" + name: "name" + client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + type: {} + token_endpoint: "token_endpoint" + secrets: + - mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + - mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + sql: + dialect: {} + statement: "statement" + - template: + container: + args: + - "args" + - "args" + image: "image" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + data_config: + io_strategy: + upload_mode: {} + download_mode: {} + format: {} + output_path: "output_path" + enabled: true + input_path: "input_path" + env: + - value: "value" + key: "key" + - value: "value" + key: "key" + ports: + - container_port: 2 + - container_port: 2 + config: + - value: "value" + key: "key" + - value: "value" + key: "key" + command: + - "command" + - "command" + architecture: {} + metadata: + retries: + retries: 0 + pod_template_name: "pod_template_name" + discoverable: true + runtime: + flavor: "flavor" + type: {} + version: "version" + cache_serializable: true + discovery_version: "discovery_version" + deprecated_error_message: "deprecated_error_message" + interruptible: true + timeout: "timeout" + generates_deck: true + cache_ignore_input_vars: + - "cache_ignore_input_vars" + - "cache_ignore_input_vars" + tags: + key: "tags" + task_type_version: 7 + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + custom: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + k8s_pod: + metadata: + annotations: + key: "annotations" + labels: + key: "labels" + pod_spec: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + data_config: + io_strategy: + upload_mode: {} + download_mode: {} + format: {} + output_path: "output_path" + enabled: true + input_path: "input_path" + id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + type: "type" + interface: + outputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + inputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + config: + key: "config" + security_context: + run_as: + execution_identity: "execution_identity" + iam_role: "iam_role" + oauth2_client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + k8s_service_account: "k8s_service_account" + tokens: + - idp_discovery_endpoint: "idp_discovery_endpoint" + name: "name" + client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + type: {} + token_endpoint: "token_endpoint" + - idp_discovery_endpoint: "idp_discovery_endpoint" + name: "name" + client: + client_secret: + mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + client_id: "client_id" + type: {} + token_endpoint: "token_endpoint" + secrets: + - mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + - mount_requirement: {} + group_version: "group_version" + key: "key" + group: "group" + sql: + dialect: {} + statement: "statement" + primary: + template: + outputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + metadata: + on_failure: {} + quality_of_service: + tier: {} + spec: + queueing_budget: "queueing_budget" + tags: + key: "tags" + failure_node: + branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + nodes: + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + - branch_node: + if_else: + other: + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + - condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + error: + message: "message" + failed_node_id: "failed_node_id" + case: + condition: + conjunction: + operator: {} + comparison: + left_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + right_value: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + var: "var" + operator: {} + array_node: + min_successes: 5 + parallelism: 1 + min_success_ratio: 5.637377 + metadata: + retries: + retries: 0 + name: "name" + interruptible: true + timeout: "timeout" + gate_node: + sleep: + duration: "duration" + approve: + signal_id: "signal_id" + signal: + output_variable_name: "output_variable_name" + signal_id: "signal_id" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + upstream_node_ids: + - "upstream_node_ids" + - "upstream_node_ids" + inputs: + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + - var: "var" + binding: + scalar: + schema: + type: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + uri: "uri" + blob: + metadata: + type: + dimensionality: {} + format: "format" + uri: "uri" + none_type: {} + primitive: + duration: "duration" + datetime: "2000-01-23T04:56:07.000+00:00" + string_value: "string_value" + boolean: true + float_value: 5.962133916683182 + integer: "integer" + binary: + tag: "tag" + value: "value" + structured_dataset: + metadata: + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + uri: "uri" + union: + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + error: + message: "message" + failed_node_id: "failed_node_id" + generic: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + promise: + attr_path: + - string_value: "string_value" + int_value: 6 + - string_value: "string_value" + int_value: 6 + var: "var" + node_id: "node_id" + collection: + bindings: + - null + - null + union: + targetType: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + map: + bindings: {} + output_aliases: + - var: "var" + alias: "alias" + - var: "var" + alias: "alias" + task_node: + reference_id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + overrides: + extended_resources: + gpu_accelerator: + partition_size: "partition_size" + unpartitioned: true + device: "device" + resources: + requests: + - name: {} + value: "value" + - name: {} + value: "value" + limits: + - name: {} + value: "value" + - name: {} + value: "value" + id: "id" + workflow_node: + launchplan_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + sub_workflow_ref: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + metadata_defaults: + interruptible: true + id: + org: "org" + domain: "domain" + resource_type: {} + name: "name" + project: "project" + version: "version" + interface: + outputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + inputs: + variables: + key: + description: "description" + artifact_partial_id: + partitions: + value: + key: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + artifact_key: + domain: "domain" + name: "name" + project: "project" + version: "version" + type: + schema: + columns: + - name: "name" + type: {} + - name: "name" + type: {} + annotation: + annotations: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + structured_dataset_type: + external_schema_type: "external_schema_type" + columns: + - name: "name" + - name: "name" + format: "format" + external_schema_bytes: "external_schema_bytes" + metadata: + fields: + key: + list_value: + values: + - null + - null + number_value: 6.027456183070403 + string_value: "string_value" + null_value: {} + bool_value: true + blob: + dimensionality: {} + format: "format" + enum_type: + values: + - "values" + - "values" + union_type: + variants: + - null + - null + simple: {} + structure: + dataclass_type: {} + tag: "tag" + artifact_tag: + artifact_key: + domain: "domain" + name: "name" + project: "project" + value: + input_binding: + var: "var" + static_value: "static_value" + triggered_binding: + transform: "transform" + partition_key: "partition_key" + index: 1 + connections: + upstream: + key: + ids: + - "ids" + - "ids" + downstream: + key: + ids: + - "ids" + - "ids" adminEmailNotification: type: "object" properties: diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api_admin_service.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api_admin_service.go index 92678dc67b..fda419e76f 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api_admin_service.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/api_admin_service.go @@ -11,12 +11,12 @@ package flyteadmin import ( "context" + "fmt" + "github.com/antihax/optional" "io/ioutil" "net/http" "net/url" "strings" - "fmt" - "github.com/antihax/optional" ) // Linger please @@ -26,19 +26,19 @@ var ( type AdminServiceApiService service -/* +/* AdminServiceApiService Triggers the creation of a :ref:`ref_flyteidl.admin.Execution` - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) CreateExecution(ctx context.Context, body AdminExecutionCreateRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -86,49 +86,49 @@ func (a *AdminServiceApiService) CreateExecution(ctx context.Context, body Admin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Triggers the creation of a :ref:`ref_flyteidl.admin.Execution` - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param org Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param org Optional, org key applied to the resource. + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) CreateExecution2(ctx context.Context, org string, body AdminExecutionCreateRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -177,48 +177,48 @@ func (a *AdminServiceApiService) CreateExecution2(ctx context.Context, org strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.LaunchPlan` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminLaunchPlanCreateResponse */ func (a *AdminServiceApiService) CreateLaunchPlan(ctx context.Context, body AdminLaunchPlanCreateRequest) (AdminLaunchPlanCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanCreateResponse ) @@ -266,49 +266,49 @@ func (a *AdminServiceApiService) CreateLaunchPlan(ctx context.Context, body Admi if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.LaunchPlan` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param body @return AdminLaunchPlanCreateResponse */ func (a *AdminServiceApiService) CreateLaunchPlan2(ctx context.Context, idOrg string, body AdminLaunchPlanCreateRequest) (AdminLaunchPlanCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanCreateResponse ) @@ -357,48 +357,48 @@ func (a *AdminServiceApiService) CreateLaunchPlan2(ctx context.Context, idOrg st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.NodeExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminNodeExecutionEventResponse */ func (a *AdminServiceApiService) CreateNodeEvent(ctx context.Context, body AdminNodeExecutionEventRequest) (AdminNodeExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionEventResponse ) @@ -446,49 +446,49 @@ func (a *AdminServiceApiService) CreateNodeEvent(ctx context.Context, body Admin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.NodeExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param eventIdExecutionIdOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param eventIdExecutionIdOrg Optional, org key applied to the resource. + - @param body @return AdminNodeExecutionEventResponse */ func (a *AdminServiceApiService) CreateNodeEvent2(ctx context.Context, eventIdExecutionIdOrg string, body AdminNodeExecutionEventRequest) (AdminNodeExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionEventResponse ) @@ -537,48 +537,48 @@ func (a *AdminServiceApiService) CreateNodeEvent2(ctx context.Context, eventIdEx if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.Task` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return FlyteidladminTaskCreateResponse */ func (a *AdminServiceApiService) CreateTask(ctx context.Context, body FlyteidladminTaskCreateRequest) (FlyteidladminTaskCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminTaskCreateResponse ) @@ -626,49 +626,49 @@ func (a *AdminServiceApiService) CreateTask(ctx context.Context, body Flyteidlad if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminTaskCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.Task` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param body @return FlyteidladminTaskCreateResponse */ func (a *AdminServiceApiService) CreateTask2(ctx context.Context, idOrg string, body FlyteidladminTaskCreateRequest) (FlyteidladminTaskCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminTaskCreateResponse ) @@ -717,48 +717,48 @@ func (a *AdminServiceApiService) CreateTask2(ctx context.Context, idOrg string, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminTaskCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.TaskExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminTaskExecutionEventResponse */ func (a *AdminServiceApiService) CreateTaskEvent(ctx context.Context, body AdminTaskExecutionEventRequest) (AdminTaskExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionEventResponse ) @@ -806,49 +806,49 @@ func (a *AdminServiceApiService) CreateTaskEvent(ctx context.Context, body Admin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.TaskExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param eventParentNodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param eventParentNodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. + - @param body @return AdminTaskExecutionEventResponse */ func (a *AdminServiceApiService) CreateTaskEvent2(ctx context.Context, eventParentNodeExecutionIdExecutionIdOrg string, body AdminTaskExecutionEventRequest) (AdminTaskExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionEventResponse ) @@ -897,48 +897,48 @@ func (a *AdminServiceApiService) CreateTaskEvent2(ctx context.Context, eventPare if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.Workflow` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminWorkflowCreateResponse */ func (a *AdminServiceApiService) CreateWorkflow(ctx context.Context, body AdminWorkflowCreateRequest) (AdminWorkflowCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowCreateResponse ) @@ -986,49 +986,49 @@ func (a *AdminServiceApiService) CreateWorkflow(ctx context.Context, body AdminW if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Create and upload a :ref:`ref_flyteidl.admin.Workflow` definition - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param body @return AdminWorkflowCreateResponse */ func (a *AdminServiceApiService) CreateWorkflow2(ctx context.Context, idOrg string, body AdminWorkflowCreateRequest) (AdminWorkflowCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowCreateResponse ) @@ -1077,48 +1077,48 @@ func (a *AdminServiceApiService) CreateWorkflow2(ctx context.Context, idOrg stri if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.WorkflowExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminWorkflowExecutionEventResponse */ func (a *AdminServiceApiService) CreateWorkflowEvent(ctx context.Context, body AdminWorkflowExecutionEventRequest) (AdminWorkflowExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionEventResponse ) @@ -1166,49 +1166,49 @@ func (a *AdminServiceApiService) CreateWorkflowEvent(ctx context.Context, body A if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Indicates a :ref:`ref_flyteidl.event.WorkflowExecutionEvent` has occurred. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param eventExecutionIdOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param eventExecutionIdOrg Optional, org key applied to the resource. + - @param body @return AdminWorkflowExecutionEventResponse */ func (a *AdminServiceApiService) CreateWorkflowEvent2(ctx context.Context, eventExecutionIdOrg string, body AdminWorkflowExecutionEventRequest) (AdminWorkflowExecutionEventResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionEventResponse ) @@ -1257,49 +1257,49 @@ func (a *AdminServiceApiService) CreateWorkflowEvent2(ctx context.Context, event if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionEventResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param project Unique project id which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param project Unique project id which this set of attributes references. +required + - @param body @return AdminProjectAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteProjectAttributes(ctx context.Context, project string, body AdminProjectAttributesDeleteRequest) (AdminProjectAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesDeleteResponse ) @@ -1348,50 +1348,50 @@ func (a *AdminServiceApiService) DeleteProjectAttributes(ctx context.Context, pr if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param org Optional, org key applied to the project. - * @param project Unique project id which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param org Optional, org key applied to the project. + - @param project Unique project id which this set of attributes references. +required + - @param body @return AdminProjectAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteProjectAttributes2(ctx context.Context, org string, project string, body AdminProjectAttributesDeleteRequest) (AdminProjectAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesDeleteResponse ) @@ -1441,50 +1441,50 @@ func (a *AdminServiceApiService) DeleteProjectAttributes2(ctx context.Context, o if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param project Unique project id which this set of attributes references. +required - * @param domain Unique domain id which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param project Unique project id which this set of attributes references. +required + - @param domain Unique domain id which this set of attributes references. +required + - @param body @return AdminProjectDomainAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteProjectDomainAttributes(ctx context.Context, project string, domain string, body AdminProjectDomainAttributesDeleteRequest) (AdminProjectDomainAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesDeleteResponse ) @@ -1534,51 +1534,51 @@ func (a *AdminServiceApiService) DeleteProjectDomainAttributes(ctx context.Conte if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param org Optional, org key applied to the attributes. - * @param project Unique project id which this set of attributes references. +required - * @param domain Unique domain id which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param org Optional, org key applied to the attributes. + - @param project Unique project id which this set of attributes references. +required + - @param domain Unique domain id which this set of attributes references. +required + - @param body @return AdminProjectDomainAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteProjectDomainAttributes2(ctx context.Context, org string, project string, domain string, body AdminProjectDomainAttributesDeleteRequest) (AdminProjectDomainAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesDeleteResponse ) @@ -1629,51 +1629,51 @@ func (a *AdminServiceApiService) DeleteProjectDomainAttributes2(ctx context.Cont if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param project Unique project id which this set of attributes references. +required - * @param domain Unique domain id which this set of attributes references. +required - * @param workflow Workflow name which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param project Unique project id which this set of attributes references. +required + - @param domain Unique domain id which this set of attributes references. +required + - @param workflow Workflow name which this set of attributes references. +required + - @param body @return AdminWorkflowAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteWorkflowAttributes(ctx context.Context, project string, domain string, workflow string, body AdminWorkflowAttributesDeleteRequest) (AdminWorkflowAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesDeleteResponse ) @@ -1724,52 +1724,52 @@ func (a *AdminServiceApiService) DeleteWorkflowAttributes(ctx context.Context, p if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Deletes custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param org Optional, org key applied to the attributes. - * @param project Unique project id which this set of attributes references. +required - * @param domain Unique domain id which this set of attributes references. +required - * @param workflow Workflow name which this set of attributes references. +required - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param org Optional, org key applied to the attributes. + - @param project Unique project id which this set of attributes references. +required + - @param domain Unique domain id which this set of attributes references. +required + - @param workflow Workflow name which this set of attributes references. +required + - @param body @return AdminWorkflowAttributesDeleteResponse */ func (a *AdminServiceApiService) DeleteWorkflowAttributes2(ctx context.Context, org string, project string, domain string, workflow string, body AdminWorkflowAttributesDeleteRequest) (AdminWorkflowAttributesDeleteResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesDeleteResponse ) @@ -1821,36 +1821,36 @@ func (a *AdminServiceApiService) DeleteWorkflowAttributes2(ctx context.Context, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesDeleteResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch the active version of a :ref:`ref_flyteidl.admin.LaunchPlan`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -1862,16 +1862,16 @@ AdminServiceApiService Fetch the active version of a :ref:`ref_flyteidl.adm @return AdminLaunchPlan */ -type GetActiveLaunchPlanOpts struct { +type GetActiveLaunchPlanOpts struct { IdOrg optional.String } func (a *AdminServiceApiService) GetActiveLaunchPlan(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *GetActiveLaunchPlanOpts) (AdminLaunchPlan, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlan ) @@ -1923,51 +1923,51 @@ func (a *AdminServiceApiService) GetActiveLaunchPlan(ctx context.Context, idProj if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlan - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch the active version of a :ref:`ref_flyteidl.admin.LaunchPlan`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' @return AdminLaunchPlan */ func (a *AdminServiceApiService) GetActiveLaunchPlan2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string) (AdminLaunchPlan, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlan ) @@ -2017,36 +2017,36 @@ func (a *AdminServiceApiService) GetActiveLaunchPlan2(ctx context.Context, idOrg if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlan - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idResourceType Identifies the specific type of resource that this identifier corresponds to. @@ -2060,16 +2060,16 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity&# @return AdminDescriptionEntity */ -type GetDescriptionEntityOpts struct { +type GetDescriptionEntityOpts struct { IdOrg optional.String } func (a *AdminServiceApiService) GetDescriptionEntity(ctx context.Context, idResourceType string, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetDescriptionEntityOpts) (AdminDescriptionEntity, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntity ) @@ -2123,53 +2123,53 @@ func (a *AdminServiceApiService) GetDescriptionEntity(ctx context.Context, idRes if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntity - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idResourceType Identifies the specific type of resource that this identifier corresponds to. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. - * @param idVersion Specific version of the resource. + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idResourceType Identifies the specific type of resource that this identifier corresponds to. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. + - @param idVersion Specific version of the resource. @return AdminDescriptionEntity */ func (a *AdminServiceApiService) GetDescriptionEntity2(ctx context.Context, idOrg string, idResourceType string, idProject string, idDomain string, idName string, idVersion string) (AdminDescriptionEntity, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntity ) @@ -2221,36 +2221,236 @@ func (a *AdminServiceApiService) GetDescriptionEntity2(ctx context.Context, idOr if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntity - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() + return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr + } + + return localVarReturnValue, localVarHttpResponse, newErr + } + + return localVarReturnValue, localVarHttpResponse, nil +} + +/* +AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + * @param idExecutionIdProject Name of the project the resource belongs to. + * @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + * @param idExecutionIdName User or system provided value for the resource. + * @param idNodeId + * @param optional nil or *GetDynamicNodeWorkflowOpts - Optional Parameters: + * @param "IdExecutionIdOrg" (optional.String) - Optional, org key applied to the resource. + +@return AdminDynamicNodeWorkflowResponse +*/ + +type GetDynamicNodeWorkflowOpts struct { + IdExecutionIdOrg optional.String +} + +func (a *AdminServiceApiService) GetDynamicNodeWorkflow(ctx context.Context, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string, localVarOptionals *GetDynamicNodeWorkflowOpts) (AdminDynamicNodeWorkflowResponse, *http.Response, error) { + var ( + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte + localVarReturnValue AdminDynamicNodeWorkflowResponse + ) + + // create path and map variables + localVarPath := a.client.cfg.BasePath + "/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow" + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.project"+"}", fmt.Sprintf("%v", idExecutionIdProject), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.domain"+"}", fmt.Sprintf("%v", idExecutionIdDomain), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.name"+"}", fmt.Sprintf("%v", idExecutionIdName), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.node_id"+"}", fmt.Sprintf("%v", idNodeId), -1) + + localVarHeaderParams := make(map[string]string) + localVarQueryParams := url.Values{} + localVarFormParams := url.Values{} + + if localVarOptionals != nil && localVarOptionals.IdExecutionIdOrg.IsSet() { + localVarQueryParams.Add("id.execution_id.org", parameterToString(localVarOptionals.IdExecutionIdOrg.Value(), "")) + } + // to determine the Content-Type header + localVarHttpContentTypes := []string{"application/json"} + + // set Content-Type header + localVarHttpContentType := selectHeaderContentType(localVarHttpContentTypes) + if localVarHttpContentType != "" { + localVarHeaderParams["Content-Type"] = localVarHttpContentType + } + + // to determine the Accept header + localVarHttpHeaderAccepts := []string{"application/json"} + + // set Accept header + localVarHttpHeaderAccept := selectHeaderAccept(localVarHttpHeaderAccepts) + if localVarHttpHeaderAccept != "" { + localVarHeaderParams["Accept"] = localVarHttpHeaderAccept + } + r, err := a.client.prepareRequest(ctx, localVarPath, localVarHttpMethod, localVarPostBody, localVarHeaderParams, localVarQueryParams, localVarFormParams, localVarFileName, localVarFileBytes) + if err != nil { + return localVarReturnValue, nil, err + } + + localVarHttpResponse, err := a.client.callAPI(r) + if err != nil || localVarHttpResponse == nil { + return localVarReturnValue, localVarHttpResponse, err + } + + localVarBody, err := ioutil.ReadAll(localVarHttpResponse.Body) + localVarHttpResponse.Body.Close() + if err != nil { + return localVarReturnValue, localVarHttpResponse, err + } + + if localVarHttpResponse.StatusCode < 300 { + // If we succeed, return the data, otherwise pass on to decode error. + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { + return localVarReturnValue, localVarHttpResponse, err + } + } + + if localVarHttpResponse.StatusCode >= 300 { + newErr := GenericSwaggerError{ + body: localVarBody, + error: localVarHttpResponse.Status, + } + + if localVarHttpResponse.StatusCode == 200 { + var v AdminDynamicNodeWorkflowResponse + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() + return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr + } + + return localVarReturnValue, localVarHttpResponse, newErr + } + + return localVarReturnValue, localVarHttpResponse, nil +} + +/* +AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idExecutionIdOrg Optional, org key applied to the resource. + - @param idExecutionIdProject Name of the project the resource belongs to. + - @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idExecutionIdName User or system provided value for the resource. + - @param idNodeId + +@return AdminDynamicNodeWorkflowResponse +*/ +func (a *AdminServiceApiService) GetDynamicNodeWorkflow2(ctx context.Context, idExecutionIdOrg string, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string) (AdminDynamicNodeWorkflowResponse, *http.Response, error) { + var ( + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte + localVarReturnValue AdminDynamicNodeWorkflowResponse + ) + + // create path and map variables + localVarPath := a.client.cfg.BasePath + "/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow" + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.org"+"}", fmt.Sprintf("%v", idExecutionIdOrg), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.project"+"}", fmt.Sprintf("%v", idExecutionIdProject), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.domain"+"}", fmt.Sprintf("%v", idExecutionIdDomain), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.execution_id.name"+"}", fmt.Sprintf("%v", idExecutionIdName), -1) + localVarPath = strings.Replace(localVarPath, "{"+"id.node_id"+"}", fmt.Sprintf("%v", idNodeId), -1) + + localVarHeaderParams := make(map[string]string) + localVarQueryParams := url.Values{} + localVarFormParams := url.Values{} + + // to determine the Content-Type header + localVarHttpContentTypes := []string{"application/json"} + + // set Content-Type header + localVarHttpContentType := selectHeaderContentType(localVarHttpContentTypes) + if localVarHttpContentType != "" { + localVarHeaderParams["Content-Type"] = localVarHttpContentType + } + + // to determine the Accept header + localVarHttpHeaderAccepts := []string{"application/json"} + + // set Accept header + localVarHttpHeaderAccept := selectHeaderAccept(localVarHttpHeaderAccepts) + if localVarHttpHeaderAccept != "" { + localVarHeaderParams["Accept"] = localVarHttpHeaderAccept + } + r, err := a.client.prepareRequest(ctx, localVarPath, localVarHttpMethod, localVarPostBody, localVarHeaderParams, localVarQueryParams, localVarFormParams, localVarFileName, localVarFileBytes) + if err != nil { + return localVarReturnValue, nil, err + } + + localVarHttpResponse, err := a.client.callAPI(r) + if err != nil || localVarHttpResponse == nil { + return localVarReturnValue, localVarHttpResponse, err + } + + localVarBody, err := ioutil.ReadAll(localVarHttpResponse.Body) + localVarHttpResponse.Body.Close() + if err != nil { + return localVarReturnValue, localVarHttpResponse, err + } + + if localVarHttpResponse.StatusCode < 300 { + // If we succeed, return the data, otherwise pass on to decode error. + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { + return localVarReturnValue, localVarHttpResponse, err + } + } + + if localVarHttpResponse.StatusCode >= 300 { + newErr := GenericSwaggerError{ + body: localVarBody, + error: localVarHttpResponse.Status, + } + + if localVarHttpResponse.StatusCode == 200 { + var v AdminDynamicNodeWorkflowResponse + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -2262,16 +2462,16 @@ AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.Execution`. @return AdminExecution */ -type GetExecutionOpts struct { +type GetExecutionOpts struct { IdOrg optional.String } func (a *AdminServiceApiService) GetExecution(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *GetExecutionOpts) (AdminExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecution ) @@ -2323,51 +2523,51 @@ func (a *AdminServiceApiService) GetExecution(ctx context.Context, idProject str if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. @return AdminExecution */ func (a *AdminServiceApiService) GetExecution2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string) (AdminExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecution ) @@ -2417,36 +2617,36 @@ func (a *AdminServiceApiService) GetExecution2(ctx context.Context, idOrg string if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -2458,16 +2658,16 @@ AdminServiceApiService Fetches input and output data for a :ref:`ref_flytei @return AdminWorkflowExecutionGetDataResponse */ -type GetExecutionDataOpts struct { +type GetExecutionDataOpts struct { IdOrg optional.String } func (a *AdminServiceApiService) GetExecutionData(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *GetExecutionDataOpts) (AdminWorkflowExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionGetDataResponse ) @@ -2519,51 +2719,51 @@ func (a *AdminServiceApiService) GetExecutionData(ctx context.Context, idProject if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. @return AdminWorkflowExecutionGetDataResponse */ func (a *AdminServiceApiService) GetExecutionData2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string) (AdminWorkflowExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionGetDataResponse ) @@ -2613,36 +2813,36 @@ func (a *AdminServiceApiService) GetExecutionData2(ctx context.Context, idOrg st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches runtime metrics for a :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -2655,17 +2855,17 @@ AdminServiceApiService Fetches runtime metrics for a :ref:`ref_flyteidl.adm @return AdminWorkflowExecutionGetMetricsResponse */ -type GetExecutionMetricsOpts struct { +type GetExecutionMetricsOpts struct { IdOrg optional.String Depth optional.Int32 } func (a *AdminServiceApiService) GetExecutionMetrics(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *GetExecutionMetricsOpts) (AdminWorkflowExecutionGetMetricsResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionGetMetricsResponse ) @@ -2720,36 +2920,36 @@ func (a *AdminServiceApiService) GetExecutionMetrics(ctx context.Context, idProj if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionGetMetricsResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches runtime metrics for a :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -2762,16 +2962,16 @@ AdminServiceApiService Fetches runtime metrics for a :ref:`ref_flyteidl.adm @return AdminWorkflowExecutionGetMetricsResponse */ -type GetExecutionMetrics2Opts struct { +type GetExecutionMetrics2Opts struct { Depth optional.Int32 } func (a *AdminServiceApiService) GetExecutionMetrics2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, localVarOptionals *GetExecutionMetrics2Opts) (AdminWorkflowExecutionGetMetricsResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowExecutionGetMetricsResponse ) @@ -2824,36 +3024,36 @@ func (a *AdminServiceApiService) GetExecutionMetrics2(ctx context.Context, idOrg if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowExecutionGetMetricsResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.LaunchPlan` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -2867,17 +3067,17 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.LaunchPlan` de @return AdminLaunchPlan */ -type GetLaunchPlanOpts struct { +type GetLaunchPlanOpts struct { IdResourceType optional.String - IdOrg optional.String + IdOrg optional.String } func (a *AdminServiceApiService) GetLaunchPlan(ctx context.Context, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetLaunchPlanOpts) (AdminLaunchPlan, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlan ) @@ -2933,36 +3133,36 @@ func (a *AdminServiceApiService) GetLaunchPlan(ctx context.Context, idProject st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlan - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.LaunchPlan` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -2976,16 +3176,16 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.LaunchPlan` de @return AdminLaunchPlan */ -type GetLaunchPlan2Opts struct { +type GetLaunchPlan2Opts struct { IdResourceType optional.String } func (a *AdminServiceApiService) GetLaunchPlan2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetLaunchPlan2Opts) (AdminLaunchPlan, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlan ) @@ -3039,36 +3239,36 @@ func (a *AdminServiceApiService) GetLaunchPlan2(ctx context.Context, idOrg strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlan - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Returns a :ref:`ref_flyteidl.admin.NamedEntity` object. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Resource type of the metadata to get. One of Task, Workflow or LaunchPlan. +required @@ -3081,16 +3281,16 @@ AdminServiceApiService Returns a :ref:`ref_flyteidl.admin.NamedEntity` @return AdminNamedEntity */ -type GetNamedEntityOpts struct { +type GetNamedEntityOpts struct { IdOrg optional.String } func (a *AdminServiceApiService) GetNamedEntity(ctx context.Context, resourceType string, idProject string, idDomain string, idName string, localVarOptionals *GetNamedEntityOpts) (AdminNamedEntity, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntity ) @@ -3143,52 +3343,52 @@ func (a *AdminServiceApiService) GetNamedEntity(ctx context.Context, resourceTyp if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntity - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Returns a :ref:`ref_flyteidl.admin.NamedEntity` object. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param resourceType Resource type of the metadata to get. One of Task, Workflow or LaunchPlan. +required - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param resourceType Resource type of the metadata to get. One of Task, Workflow or LaunchPlan. +required + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' @return AdminNamedEntity */ func (a *AdminServiceApiService) GetNamedEntity2(ctx context.Context, resourceType string, idOrg string, idProject string, idDomain string, idName string) (AdminNamedEntity, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntity ) @@ -3239,36 +3439,36 @@ func (a *AdminServiceApiService) GetNamedEntity2(ctx context.Context, resourceTy if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntity - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.NodeExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idExecutionIdProject Name of the project the resource belongs to. @@ -3281,16 +3481,16 @@ AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.NodeExecution @return FlyteidladminNodeExecution */ -type GetNodeExecutionOpts struct { +type GetNodeExecutionOpts struct { IdExecutionIdOrg optional.String } func (a *AdminServiceApiService) GetNodeExecution(ctx context.Context, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string, localVarOptionals *GetNodeExecutionOpts) (FlyteidladminNodeExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminNodeExecution ) @@ -3343,52 +3543,52 @@ func (a *AdminServiceApiService) GetNodeExecution(ctx context.Context, idExecuti if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminNodeExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.NodeExecution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idExecutionIdOrg Optional, org key applied to the resource. - * @param idExecutionIdProject Name of the project the resource belongs to. - * @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idExecutionIdName User or system provided value for the resource. - * @param idNodeId + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idExecutionIdOrg Optional, org key applied to the resource. + - @param idExecutionIdProject Name of the project the resource belongs to. + - @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idExecutionIdName User or system provided value for the resource. + - @param idNodeId @return FlyteidladminNodeExecution */ func (a *AdminServiceApiService) GetNodeExecution2(ctx context.Context, idExecutionIdOrg string, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string) (FlyteidladminNodeExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminNodeExecution ) @@ -3439,36 +3639,36 @@ func (a *AdminServiceApiService) GetNodeExecution2(ctx context.Context, idExecut if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminNodeExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.NodeExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idExecutionIdProject Name of the project the resource belongs to. @@ -3481,16 +3681,16 @@ AdminServiceApiService Fetches input and output data for a :ref:`ref_flytei @return AdminNodeExecutionGetDataResponse */ -type GetNodeExecutionDataOpts struct { +type GetNodeExecutionDataOpts struct { IdExecutionIdOrg optional.String } func (a *AdminServiceApiService) GetNodeExecutionData(ctx context.Context, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string, localVarOptionals *GetNodeExecutionDataOpts) (AdminNodeExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionGetDataResponse ) @@ -3543,52 +3743,52 @@ func (a *AdminServiceApiService) GetNodeExecutionData(ctx context.Context, idExe if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.NodeExecution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idExecutionIdOrg Optional, org key applied to the resource. - * @param idExecutionIdProject Name of the project the resource belongs to. - * @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idExecutionIdName User or system provided value for the resource. - * @param idNodeId + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idExecutionIdOrg Optional, org key applied to the resource. + - @param idExecutionIdProject Name of the project the resource belongs to. + - @param idExecutionIdDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idExecutionIdName User or system provided value for the resource. + - @param idNodeId @return AdminNodeExecutionGetDataResponse */ func (a *AdminServiceApiService) GetNodeExecutionData2(ctx context.Context, idExecutionIdOrg string, idExecutionIdProject string, idExecutionIdDomain string, idExecutionIdName string, idNodeId string) (AdminNodeExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionGetDataResponse ) @@ -3639,36 +3839,36 @@ func (a *AdminServiceApiService) GetNodeExecutionData2(ctx context.Context, idEx if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Unique project id which this set of attributes references. +required @@ -3679,17 +3879,17 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminProjectAttributesGetResponse */ -type GetProjectAttributesOpts struct { +type GetProjectAttributesOpts struct { ResourceType optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) GetProjectAttributes(ctx context.Context, project string, localVarOptionals *GetProjectAttributesOpts) (AdminProjectAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesGetResponse ) @@ -3742,36 +3942,36 @@ func (a *AdminServiceApiService) GetProjectAttributes(ctx context.Context, proje if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the project. @@ -3782,16 +3982,16 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminProjectAttributesGetResponse */ -type GetProjectAttributes2Opts struct { +type GetProjectAttributes2Opts struct { ResourceType optional.String } func (a *AdminServiceApiService) GetProjectAttributes2(ctx context.Context, org string, project string, localVarOptionals *GetProjectAttributes2Opts) (AdminProjectAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesGetResponse ) @@ -3842,36 +4042,36 @@ func (a *AdminServiceApiService) GetProjectAttributes2(ctx context.Context, org if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Unique project id which this set of attributes references. +required @@ -3883,17 +4083,17 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminProjectDomainAttributesGetResponse */ -type GetProjectDomainAttributesOpts struct { +type GetProjectDomainAttributesOpts struct { ResourceType optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) GetProjectDomainAttributes(ctx context.Context, project string, domain string, localVarOptionals *GetProjectDomainAttributesOpts) (AdminProjectDomainAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesGetResponse ) @@ -3947,36 +4147,36 @@ func (a *AdminServiceApiService) GetProjectDomainAttributes(ctx context.Context, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the attributes. @@ -3988,16 +4188,16 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminProjectDomainAttributesGetResponse */ -type GetProjectDomainAttributes2Opts struct { +type GetProjectDomainAttributes2Opts struct { ResourceType optional.String } func (a *AdminServiceApiService) GetProjectDomainAttributes2(ctx context.Context, org string, project string, domain string, localVarOptionals *GetProjectDomainAttributes2Opts) (AdminProjectDomainAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesGetResponse ) @@ -4049,36 +4249,36 @@ func (a *AdminServiceApiService) GetProjectDomainAttributes2(ctx context.Context if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Task` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -4092,17 +4292,17 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Task` definiti @return AdminTask */ -type GetTaskOpts struct { +type GetTaskOpts struct { IdResourceType optional.String - IdOrg optional.String + IdOrg optional.String } func (a *AdminServiceApiService) GetTask(ctx context.Context, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetTaskOpts) (AdminTask, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTask ) @@ -4158,36 +4358,36 @@ func (a *AdminServiceApiService) GetTask(ctx context.Context, idProject string, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTask - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Task` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -4201,16 +4401,16 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Task` definiti @return AdminTask */ -type GetTask2Opts struct { +type GetTask2Opts struct { IdResourceType optional.String } func (a *AdminServiceApiService) GetTask2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetTask2Opts) (AdminTask, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTask ) @@ -4264,36 +4464,36 @@ func (a *AdminServiceApiService) GetTask2(ctx context.Context, idOrg string, idP if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTask - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idNodeExecutionIdExecutionIdProject Name of the project the resource belongs to. @@ -4313,18 +4513,18 @@ AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.TaskExecution @return FlyteidladminTaskExecution */ -type GetTaskExecutionOpts struct { - IdTaskIdResourceType optional.String - IdTaskIdOrg optional.String +type GetTaskExecutionOpts struct { + IdTaskIdResourceType optional.String + IdTaskIdOrg optional.String IdNodeExecutionIdExecutionIdOrg optional.String } func (a *AdminServiceApiService) GetTaskExecution(ctx context.Context, idNodeExecutionIdExecutionIdProject string, idNodeExecutionIdExecutionIdDomain string, idNodeExecutionIdExecutionIdName string, idNodeExecutionIdNodeId string, idTaskIdProject string, idTaskIdDomain string, idTaskIdName string, idTaskIdVersion string, idRetryAttempt int64, localVarOptionals *GetTaskExecutionOpts) (FlyteidladminTaskExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminTaskExecution ) @@ -4388,36 +4588,36 @@ func (a *AdminServiceApiService) GetTaskExecution(ctx context.Context, idNodeExe if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminTaskExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idNodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. @@ -4437,17 +4637,17 @@ AdminServiceApiService Fetches a :ref:`ref_flyteidl.admin.TaskExecution @return FlyteidladminTaskExecution */ -type GetTaskExecution2Opts struct { +type GetTaskExecution2Opts struct { IdTaskIdResourceType optional.String - IdTaskIdOrg optional.String + IdTaskIdOrg optional.String } func (a *AdminServiceApiService) GetTaskExecution2(ctx context.Context, idNodeExecutionIdExecutionIdOrg string, idNodeExecutionIdExecutionIdProject string, idNodeExecutionIdExecutionIdDomain string, idNodeExecutionIdExecutionIdName string, idNodeExecutionIdNodeId string, idTaskIdProject string, idTaskIdDomain string, idTaskIdName string, idTaskIdVersion string, idRetryAttempt int64, localVarOptionals *GetTaskExecution2Opts) (FlyteidladminTaskExecution, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue FlyteidladminTaskExecution ) @@ -4509,36 +4709,36 @@ func (a *AdminServiceApiService) GetTaskExecution2(ctx context.Context, idNodeEx if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v FlyteidladminTaskExecution - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idNodeExecutionIdExecutionIdProject Name of the project the resource belongs to. @@ -4558,18 +4758,18 @@ AdminServiceApiService Fetches input and output data for a :ref:`ref_flytei @return AdminTaskExecutionGetDataResponse */ -type GetTaskExecutionDataOpts struct { - IdTaskIdResourceType optional.String - IdTaskIdOrg optional.String +type GetTaskExecutionDataOpts struct { + IdTaskIdResourceType optional.String + IdTaskIdOrg optional.String IdNodeExecutionIdExecutionIdOrg optional.String } func (a *AdminServiceApiService) GetTaskExecutionData(ctx context.Context, idNodeExecutionIdExecutionIdProject string, idNodeExecutionIdExecutionIdDomain string, idNodeExecutionIdExecutionIdName string, idNodeExecutionIdNodeId string, idTaskIdProject string, idTaskIdDomain string, idTaskIdName string, idTaskIdVersion string, idRetryAttempt int64, localVarOptionals *GetTaskExecutionDataOpts) (AdminTaskExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionGetDataResponse ) @@ -4633,36 +4833,36 @@ func (a *AdminServiceApiService) GetTaskExecutionData(ctx context.Context, idNod if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches input and output data for a :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idNodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. @@ -4682,17 +4882,17 @@ AdminServiceApiService Fetches input and output data for a :ref:`ref_flytei @return AdminTaskExecutionGetDataResponse */ -type GetTaskExecutionData2Opts struct { +type GetTaskExecutionData2Opts struct { IdTaskIdResourceType optional.String - IdTaskIdOrg optional.String + IdTaskIdOrg optional.String } func (a *AdminServiceApiService) GetTaskExecutionData2(ctx context.Context, idNodeExecutionIdExecutionIdOrg string, idNodeExecutionIdExecutionIdProject string, idNodeExecutionIdExecutionIdDomain string, idNodeExecutionIdExecutionIdName string, idNodeExecutionIdNodeId string, idTaskIdProject string, idTaskIdDomain string, idTaskIdName string, idTaskIdVersion string, idRetryAttempt int64, localVarOptionals *GetTaskExecutionData2Opts) (AdminTaskExecutionGetDataResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionGetDataResponse ) @@ -4754,47 +4954,47 @@ func (a *AdminServiceApiService) GetTaskExecutionData2(ctx context.Context, idNo if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionGetDataResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). @return AdminGetVersionResponse */ func (a *AdminServiceApiService) GetVersion(ctx context.Context) (AdminGetVersionResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminGetVersionResponse ) @@ -4840,36 +5040,36 @@ func (a *AdminServiceApiService) GetVersion(ctx context.Context) (AdminGetVersio if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminGetVersionResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Workflow` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -4883,17 +5083,17 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Workflow` defi @return AdminWorkflow */ -type GetWorkflowOpts struct { +type GetWorkflowOpts struct { IdResourceType optional.String - IdOrg optional.String + IdOrg optional.String } func (a *AdminServiceApiService) GetWorkflow(ctx context.Context, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetWorkflowOpts) (AdminWorkflow, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflow ) @@ -4949,36 +5149,36 @@ func (a *AdminServiceApiService) GetWorkflow(ctx context.Context, idProject stri if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflow - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Workflow` definition. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -4992,16 +5192,16 @@ AdminServiceApiService Fetch a :ref:`ref_flyteidl.admin.Workflow` defi @return AdminWorkflow */ -type GetWorkflow2Opts struct { +type GetWorkflow2Opts struct { IdResourceType optional.String } func (a *AdminServiceApiService) GetWorkflow2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, idVersion string, localVarOptionals *GetWorkflow2Opts) (AdminWorkflow, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflow ) @@ -5055,36 +5255,36 @@ func (a *AdminServiceApiService) GetWorkflow2(ctx context.Context, idOrg string, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflow - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Unique project id which this set of attributes references. +required @@ -5097,17 +5297,17 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminWorkflowAttributesGetResponse */ -type GetWorkflowAttributesOpts struct { +type GetWorkflowAttributesOpts struct { ResourceType optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) GetWorkflowAttributes(ctx context.Context, project string, domain string, workflow string, localVarOptionals *GetWorkflowAttributesOpts) (AdminWorkflowAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesGetResponse ) @@ -5162,36 +5362,36 @@ func (a *AdminServiceApiService) GetWorkflowAttributes(ctx context.Context, proj if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the attributes. @@ -5204,16 +5404,16 @@ AdminServiceApiService Fetches custom :ref:`ref_flyteidl.admin.MatchableAtt @return AdminWorkflowAttributesGetResponse */ -type GetWorkflowAttributes2Opts struct { +type GetWorkflowAttributes2Opts struct { ResourceType optional.String } func (a *AdminServiceApiService) GetWorkflowAttributes2(ctx context.Context, org string, project string, domain string, workflow string, localVarOptionals *GetWorkflowAttributes2Opts) (AdminWorkflowAttributesGetResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesGetResponse ) @@ -5266,36 +5466,36 @@ func (a *AdminServiceApiService) GetWorkflowAttributes2(ctx context.Context, org if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesGetResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService List active versions of :ref:`ref_flyteidl.admin.LaunchPlan`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Name of the project that contains the identifiers. +required. @@ -5310,20 +5510,20 @@ AdminServiceApiService List active versions of :ref:`ref_flyteidl.admin.Lau @return AdminLaunchPlanList */ -type ListActiveLaunchPlansOpts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListActiveLaunchPlansOpts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) ListActiveLaunchPlans(ctx context.Context, project string, domain string, localVarOptionals *ListActiveLaunchPlansOpts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -5386,36 +5586,36 @@ func (a *AdminServiceApiService) ListActiveLaunchPlans(ctx context.Context, proj if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService List active versions of :ref:`ref_flyteidl.admin.LaunchPlan`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the resource. @@ -5430,19 +5630,19 @@ AdminServiceApiService List active versions of :ref:`ref_flyteidl.admin.Lau @return AdminLaunchPlanList */ -type ListActiveLaunchPlans2Opts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListActiveLaunchPlans2Opts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListActiveLaunchPlans2(ctx context.Context, org string, project string, domain string, localVarOptionals *ListActiveLaunchPlans2Opts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -5503,36 +5703,36 @@ func (a *AdminServiceApiService) ListActiveLaunchPlans2(ctx context.Context, org if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.DescriptionEntity` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Identifies the specific type of resource that this identifier corresponds to. @@ -5550,21 +5750,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Description @return AdminDescriptionEntityList */ -type ListDescriptionEntitiesOpts struct { - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListDescriptionEntitiesOpts struct { + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListDescriptionEntities(ctx context.Context, resourceType string, idProject string, idDomain string, idName string, localVarOptionals *ListDescriptionEntitiesOpts) (AdminDescriptionEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntityList ) @@ -5632,36 +5832,36 @@ func (a *AdminServiceApiService) ListDescriptionEntities(ctx context.Context, re if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.DescriptionEntity` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Identifies the specific type of resource that this identifier corresponds to. @@ -5679,20 +5879,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Description @return AdminDescriptionEntityList */ -type ListDescriptionEntities2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListDescriptionEntities2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListDescriptionEntities2(ctx context.Context, resourceType string, idOrg string, idProject string, idDomain string, idName string, localVarOptionals *ListDescriptionEntities2Opts) (AdminDescriptionEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntityList ) @@ -5758,36 +5958,36 @@ func (a *AdminServiceApiService) ListDescriptionEntities2(ctx context.Context, r if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.DescriptionEntity` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Identifies the specific type of resource that this identifier corresponds to. @@ -5805,22 +6005,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Description @return AdminDescriptionEntityList */ -type ListDescriptionEntities3Opts struct { - IdName optional.String - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListDescriptionEntities3Opts struct { + IdName optional.String + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListDescriptionEntities3(ctx context.Context, resourceType string, idProject string, idDomain string, localVarOptionals *ListDescriptionEntities3Opts) (AdminDescriptionEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntityList ) @@ -5890,36 +6090,36 @@ func (a *AdminServiceApiService) ListDescriptionEntities3(ctx context.Context, r if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.DescriptionEntity` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Identifies the specific type of resource that this identifier corresponds to. @@ -5937,21 +6137,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Description @return AdminDescriptionEntityList */ -type ListDescriptionEntities4Opts struct { - IdName optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListDescriptionEntities4Opts struct { + IdName optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListDescriptionEntities4(ctx context.Context, resourceType string, idOrg string, idProject string, idDomain string, localVarOptionals *ListDescriptionEntities4Opts) (AdminDescriptionEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminDescriptionEntityList ) @@ -6019,36 +6219,36 @@ func (a *AdminServiceApiService) ListDescriptionEntities4(ctx context.Context, r if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminDescriptionEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -6065,22 +6265,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Execution&# @return AdminExecutionList */ -type ListExecutionsOpts struct { - IdName optional.String - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListExecutionsOpts struct { + IdName optional.String + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListExecutions(ctx context.Context, idProject string, idDomain string, localVarOptionals *ListExecutionsOpts) (AdminExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionList ) @@ -6149,36 +6349,36 @@ func (a *AdminServiceApiService) ListExecutions(ctx context.Context, idProject s if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Execution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -6195,21 +6395,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Execution&# @return AdminExecutionList */ -type ListExecutions2Opts struct { - IdName optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListExecutions2Opts struct { + IdName optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListExecutions2(ctx context.Context, idOrg string, idProject string, idDomain string, localVarOptionals *ListExecutions2Opts) (AdminExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionList ) @@ -6276,36 +6476,36 @@ func (a *AdminServiceApiService) ListExecutions2(ctx context.Context, idOrg stri if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of launch plan objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Name of the project that contains the identifiers. +required @@ -6321,21 +6521,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListLaunchPlanIdsOpts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListLaunchPlanIdsOpts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String - Org optional.String + Filters optional.String + Org optional.String } func (a *AdminServiceApiService) ListLaunchPlanIds(ctx context.Context, project string, domain string, localVarOptionals *ListLaunchPlanIdsOpts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -6401,36 +6601,36 @@ func (a *AdminServiceApiService) ListLaunchPlanIds(ctx context.Context, project if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of launch plan objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the resource. @@ -6446,20 +6646,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListLaunchPlanIds2Opts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListLaunchPlanIds2Opts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String + Filters optional.String } func (a *AdminServiceApiService) ListLaunchPlanIds2(ctx context.Context, org string, project string, domain string, localVarOptionals *ListLaunchPlanIds2Opts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -6523,36 +6723,36 @@ func (a *AdminServiceApiService) ListLaunchPlanIds2(ctx context.Context, org str if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -6569,21 +6769,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan& @return AdminLaunchPlanList */ -type ListLaunchPlansOpts struct { - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListLaunchPlansOpts struct { + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListLaunchPlans(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *ListLaunchPlansOpts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -6650,36 +6850,36 @@ func (a *AdminServiceApiService) ListLaunchPlans(ctx context.Context, idProject if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -6696,20 +6896,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan& @return AdminLaunchPlanList */ -type ListLaunchPlans2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListLaunchPlans2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListLaunchPlans2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, localVarOptionals *ListLaunchPlans2Opts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -6774,36 +6974,36 @@ func (a *AdminServiceApiService) ListLaunchPlans2(ctx context.Context, idOrg str if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -6820,22 +7020,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan& @return AdminLaunchPlanList */ -type ListLaunchPlans3Opts struct { - IdName optional.String - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListLaunchPlans3Opts struct { + IdName optional.String + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListLaunchPlans3(ctx context.Context, idProject string, idDomain string, localVarOptionals *ListLaunchPlans3Opts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -6904,36 +7104,36 @@ func (a *AdminServiceApiService) ListLaunchPlans3(ctx context.Context, idProject if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -6950,21 +7150,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.LaunchPlan& @return AdminLaunchPlanList */ -type ListLaunchPlans4Opts struct { - IdName optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListLaunchPlans4Opts struct { + IdName optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListLaunchPlans4(ctx context.Context, idOrg string, idProject string, idDomain string, localVarOptionals *ListLaunchPlans4Opts) (AdminLaunchPlanList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanList ) @@ -7031,36 +7231,36 @@ func (a *AdminServiceApiService) ListLaunchPlans4(ctx context.Context, idOrg str if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Lists custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a specific resource type. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param optional nil or *ListMatchableAttributesOpts - Optional Parameters: @@ -7070,17 +7270,17 @@ AdminServiceApiService Lists custom :ref:`ref_flyteidl.admin.MatchableAttri @return AdminListMatchableAttributesResponse */ -type ListMatchableAttributesOpts struct { +type ListMatchableAttributesOpts struct { ResourceType optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) ListMatchableAttributes(ctx context.Context, localVarOptionals *ListMatchableAttributesOpts) (AdminListMatchableAttributesResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminListMatchableAttributesResponse ) @@ -7132,36 +7332,36 @@ func (a *AdminServiceApiService) ListMatchableAttributes(ctx context.Context, lo if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminListMatchableAttributesResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Lists custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a specific resource type. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org filter applied to list project requests. @@ -7171,16 +7371,16 @@ AdminServiceApiService Lists custom :ref:`ref_flyteidl.admin.MatchableAttri @return AdminListMatchableAttributesResponse */ -type ListMatchableAttributes2Opts struct { +type ListMatchableAttributes2Opts struct { ResourceType optional.String } func (a *AdminServiceApiService) ListMatchableAttributes2(ctx context.Context, org string, localVarOptionals *ListMatchableAttributes2Opts) (AdminListMatchableAttributesResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminListMatchableAttributesResponse ) @@ -7230,36 +7430,36 @@ func (a *AdminServiceApiService) ListMatchableAttributes2(ctx context.Context, o if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminListMatchableAttributesResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Returns a list of :ref:`ref_flyteidl.admin.NamedEntity` objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Resource type of the metadata to query. One of Task, Workflow or LaunchPlan. +required @@ -7276,21 +7476,21 @@ AdminServiceApiService Returns a list of :ref:`ref_flyteidl.admin.NamedEnti @return AdminNamedEntityList */ -type ListNamedEntitiesOpts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListNamedEntitiesOpts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String - Org optional.String + Filters optional.String + Org optional.String } func (a *AdminServiceApiService) ListNamedEntities(ctx context.Context, resourceType string, project string, domain string, localVarOptionals *ListNamedEntitiesOpts) (AdminNamedEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityList ) @@ -7357,36 +7557,36 @@ func (a *AdminServiceApiService) ListNamedEntities(ctx context.Context, resource if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Returns a list of :ref:`ref_flyteidl.admin.NamedEntity` objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param resourceType Resource type of the metadata to query. One of Task, Workflow or LaunchPlan. +required @@ -7403,20 +7603,20 @@ AdminServiceApiService Returns a list of :ref:`ref_flyteidl.admin.NamedEnti @return AdminNamedEntityList */ -type ListNamedEntities2Opts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListNamedEntities2Opts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String + Filters optional.String } func (a *AdminServiceApiService) ListNamedEntities2(ctx context.Context, resourceType string, org string, project string, domain string, localVarOptionals *ListNamedEntities2Opts) (AdminNamedEntityList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityList ) @@ -7481,36 +7681,36 @@ func (a *AdminServiceApiService) ListNamedEntities2(ctx context.Context, resourc if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param workflowExecutionIdProject Name of the project the resource belongs to. @@ -7519,7 +7719,7 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti * @param optional nil or *ListNodeExecutionsOpts - Optional Parameters: * @param "WorkflowExecutionIdOrg" (optional.String) - Optional, org key applied to the resource. * @param "Limit" (optional.Int64) - Indicates the number of resources to be returned. +required. - * @param "Token" (optional.String) - + * @param "Token" (optional.String) - * @param "Filters" (optional.String) - Indicates a list of filters passed as string. More info on constructing filters : <Link> +optional. * @param "SortByKey" (optional.String) - Indicates an attribute to sort the response values. +required. * @param "SortByDirection" (optional.String) - Indicates the direction to apply sort key for response values. +optional. - DESCENDING: By default, fields are sorted in descending order. @@ -7528,22 +7728,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti @return AdminNodeExecutionList */ -type ListNodeExecutionsOpts struct { +type ListNodeExecutionsOpts struct { WorkflowExecutionIdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String - SortByDirection optional.String - UniqueParentId optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String + SortByDirection optional.String + UniqueParentId optional.String } func (a *AdminServiceApiService) ListNodeExecutions(ctx context.Context, workflowExecutionIdProject string, workflowExecutionIdDomain string, workflowExecutionIdName string, localVarOptionals *ListNodeExecutionsOpts) (AdminNodeExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionList ) @@ -7613,36 +7813,36 @@ func (a *AdminServiceApiService) ListNodeExecutions(ctx context.Context, workflo if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param workflowExecutionIdOrg Optional, org key applied to the resource. @@ -7651,7 +7851,7 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti * @param workflowExecutionIdName User or system provided value for the resource. * @param optional nil or *ListNodeExecutions2Opts - Optional Parameters: * @param "Limit" (optional.Int64) - Indicates the number of resources to be returned. +required. - * @param "Token" (optional.String) - + * @param "Token" (optional.String) - * @param "Filters" (optional.String) - Indicates a list of filters passed as string. More info on constructing filters : <Link> +optional. * @param "SortByKey" (optional.String) - Indicates an attribute to sort the response values. +required. * @param "SortByDirection" (optional.String) - Indicates the direction to apply sort key for response values. +optional. - DESCENDING: By default, fields are sorted in descending order. @@ -7660,21 +7860,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti @return AdminNodeExecutionList */ -type ListNodeExecutions2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListNodeExecutions2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String - UniqueParentId optional.String + UniqueParentId optional.String } func (a *AdminServiceApiService) ListNodeExecutions2(ctx context.Context, workflowExecutionIdOrg string, workflowExecutionIdProject string, workflowExecutionIdDomain string, workflowExecutionIdName string, localVarOptionals *ListNodeExecutions2Opts) (AdminNodeExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionList ) @@ -7742,36 +7942,36 @@ func (a *AdminServiceApiService) ListNodeExecutions2(ctx context.Context, workfl if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution` launched by the reference :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param taskExecutionIdNodeExecutionIdExecutionIdProject Name of the project the resource belongs to. @@ -7796,23 +7996,23 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti @return AdminNodeExecutionList */ -type ListNodeExecutionsForTaskOpts struct { - TaskExecutionIdTaskIdResourceType optional.String - TaskExecutionIdTaskIdOrg optional.String +type ListNodeExecutionsForTaskOpts struct { + TaskExecutionIdTaskIdResourceType optional.String + TaskExecutionIdTaskIdOrg optional.String TaskExecutionIdNodeExecutionIdExecutionIdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String - SortByDirection optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String + SortByDirection optional.String } func (a *AdminServiceApiService) ListNodeExecutionsForTask(ctx context.Context, taskExecutionIdNodeExecutionIdExecutionIdProject string, taskExecutionIdNodeExecutionIdExecutionIdDomain string, taskExecutionIdNodeExecutionIdExecutionIdName string, taskExecutionIdNodeExecutionIdNodeId string, taskExecutionIdTaskIdProject string, taskExecutionIdTaskIdDomain string, taskExecutionIdTaskIdName string, taskExecutionIdTaskIdVersion string, taskExecutionIdRetryAttempt int64, localVarOptionals *ListNodeExecutionsForTaskOpts) (AdminNodeExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionList ) @@ -7891,36 +8091,36 @@ func (a *AdminServiceApiService) ListNodeExecutionsForTask(ctx context.Context, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution` launched by the reference :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param taskExecutionIdNodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. @@ -7945,22 +8145,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NodeExecuti @return AdminNodeExecutionList */ -type ListNodeExecutionsForTask2Opts struct { +type ListNodeExecutionsForTask2Opts struct { TaskExecutionIdTaskIdResourceType optional.String - TaskExecutionIdTaskIdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String - SortByDirection optional.String + TaskExecutionIdTaskIdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String + SortByDirection optional.String } func (a *AdminServiceApiService) ListNodeExecutionsForTask2(ctx context.Context, taskExecutionIdNodeExecutionIdExecutionIdOrg string, taskExecutionIdNodeExecutionIdExecutionIdProject string, taskExecutionIdNodeExecutionIdExecutionIdDomain string, taskExecutionIdNodeExecutionIdExecutionIdName string, taskExecutionIdNodeExecutionIdNodeId string, taskExecutionIdTaskIdProject string, taskExecutionIdTaskIdDomain string, taskExecutionIdTaskIdName string, taskExecutionIdTaskIdVersion string, taskExecutionIdRetryAttempt int64, localVarOptionals *ListNodeExecutionsForTask2Opts) (AdminNodeExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNodeExecutionList ) @@ -8037,36 +8237,36 @@ func (a *AdminServiceApiService) ListNodeExecutionsForTask2(ctx context.Context, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNodeExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.Project` * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param optional nil or *ListProjectsOpts - Optional Parameters: @@ -8080,21 +8280,21 @@ AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.Project&# @return AdminProjects */ -type ListProjectsOpts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListProjectsOpts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String - Org optional.String + Org optional.String } func (a *AdminServiceApiService) ListProjects(ctx context.Context, localVarOptionals *ListProjectsOpts) (AdminProjects, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjects ) @@ -8158,36 +8358,36 @@ func (a *AdminServiceApiService) ListProjects(ctx context.Context, localVarOptio if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjects - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.Project` * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org filter applied to list project requests. @@ -8201,20 +8401,20 @@ AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.Project&# @return AdminProjects */ -type ListProjects2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListProjects2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListProjects2(ctx context.Context, org string, localVarOptionals *ListProjects2Opts) (AdminProjects, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjects ) @@ -8276,36 +8476,36 @@ func (a *AdminServiceApiService) ListProjects2(ctx context.Context, org string, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjects - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param nodeExecutionIdExecutionIdProject Name of the project the resource belongs to. @@ -8323,21 +8523,21 @@ AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.TaskExecu @return AdminTaskExecutionList */ -type ListTaskExecutionsOpts struct { +type ListTaskExecutionsOpts struct { NodeExecutionIdExecutionIdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String - SortByDirection optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String + SortByDirection optional.String } func (a *AdminServiceApiService) ListTaskExecutions(ctx context.Context, nodeExecutionIdExecutionIdProject string, nodeExecutionIdExecutionIdDomain string, nodeExecutionIdExecutionIdName string, nodeExecutionIdNodeId string, localVarOptionals *ListTaskExecutionsOpts) (AdminTaskExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionList ) @@ -8405,36 +8605,36 @@ func (a *AdminServiceApiService) ListTaskExecutions(ctx context.Context, nodeExe if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.TaskExecution`. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param nodeExecutionIdExecutionIdOrg Optional, org key applied to the resource. @@ -8452,20 +8652,20 @@ AdminServiceApiService Fetches a list of :ref:`ref_flyteidl.admin.TaskExecu @return AdminTaskExecutionList */ -type ListTaskExecutions2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListTaskExecutions2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListTaskExecutions2(ctx context.Context, nodeExecutionIdExecutionIdOrg string, nodeExecutionIdExecutionIdProject string, nodeExecutionIdExecutionIdDomain string, nodeExecutionIdExecutionIdName string, nodeExecutionIdNodeId string, localVarOptionals *ListTaskExecutions2Opts) (AdminTaskExecutionList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskExecutionList ) @@ -8531,36 +8731,36 @@ func (a *AdminServiceApiService) ListTaskExecutions2(ctx context.Context, nodeEx if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskExecutionList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of task objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Name of the project that contains the identifiers. +required @@ -8576,21 +8776,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListTaskIdsOpts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListTaskIdsOpts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String - Org optional.String + Filters optional.String + Org optional.String } func (a *AdminServiceApiService) ListTaskIds(ctx context.Context, project string, domain string, localVarOptionals *ListTaskIdsOpts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -8656,36 +8856,36 @@ func (a *AdminServiceApiService) ListTaskIds(ctx context.Context, project string if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of task objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the resource. @@ -8701,20 +8901,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListTaskIds2Opts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListTaskIds2Opts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String + Filters optional.String } func (a *AdminServiceApiService) ListTaskIds2(ctx context.Context, org string, project string, domain string, localVarOptionals *ListTaskIds2Opts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -8778,36 +8978,36 @@ func (a *AdminServiceApiService) ListTaskIds2(ctx context.Context, org string, p if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -8824,21 +9024,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` @return AdminTaskList */ -type ListTasksOpts struct { - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListTasksOpts struct { + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListTasks(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *ListTasksOpts) (AdminTaskList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskList ) @@ -8905,36 +9105,36 @@ func (a *AdminServiceApiService) ListTasks(ctx context.Context, idProject string if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -8951,20 +9151,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` @return AdminTaskList */ -type ListTasks2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListTasks2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListTasks2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, localVarOptionals *ListTasks2Opts) (AdminTaskList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskList ) @@ -9029,36 +9229,36 @@ func (a *AdminServiceApiService) ListTasks2(ctx context.Context, idOrg string, i if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -9075,22 +9275,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` @return AdminTaskList */ -type ListTasks3Opts struct { - IdName optional.String - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListTasks3Opts struct { + IdName optional.String + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListTasks3(ctx context.Context, idProject string, idDomain string, localVarOptionals *ListTasks3Opts) (AdminTaskList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskList ) @@ -9159,36 +9359,36 @@ func (a *AdminServiceApiService) ListTasks3(ctx context.Context, idProject strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -9205,21 +9405,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Task` @return AdminTaskList */ -type ListTasks4Opts struct { - IdName optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListTasks4Opts struct { + IdName optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListTasks4(ctx context.Context, idOrg string, idProject string, idDomain string, localVarOptionals *ListTasks4Opts) (AdminTaskList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminTaskList ) @@ -9286,36 +9486,36 @@ func (a *AdminServiceApiService) ListTasks4(ctx context.Context, idOrg string, i if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminTaskList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of workflow objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param project Name of the project that contains the identifiers. +required @@ -9331,21 +9531,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListWorkflowIdsOpts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListWorkflowIdsOpts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String - Org optional.String + Filters optional.String + Org optional.String } func (a *AdminServiceApiService) ListWorkflowIds(ctx context.Context, project string, domain string, localVarOptionals *ListWorkflowIdsOpts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -9411,36 +9611,36 @@ func (a *AdminServiceApiService) ListWorkflowIds(ctx context.Context, project st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntityIdentifier` of workflow objects. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param org Optional, org key applied to the resource. @@ -9456,20 +9656,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.NamedEntity @return AdminNamedEntityIdentifierList */ -type ListWorkflowIds2Opts struct { - Limit optional.Int64 - Token optional.String - SortByKey optional.String +type ListWorkflowIds2Opts struct { + Limit optional.Int64 + Token optional.String + SortByKey optional.String SortByDirection optional.String - Filters optional.String + Filters optional.String } func (a *AdminServiceApiService) ListWorkflowIds2(ctx context.Context, org string, project string, domain string, localVarOptionals *ListWorkflowIds2Opts) (AdminNamedEntityIdentifierList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityIdentifierList ) @@ -9533,36 +9733,36 @@ func (a *AdminServiceApiService) ListWorkflowIds2(ctx context.Context, org strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityIdentifierList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -9579,21 +9779,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow&#x @return AdminWorkflowList */ -type ListWorkflowsOpts struct { - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListWorkflowsOpts struct { + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListWorkflows(ctx context.Context, idProject string, idDomain string, idName string, localVarOptionals *ListWorkflowsOpts) (AdminWorkflowList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowList ) @@ -9660,36 +9860,36 @@ func (a *AdminServiceApiService) ListWorkflows(ctx context.Context, idProject st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -9706,20 +9906,20 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow&#x @return AdminWorkflowList */ -type ListWorkflows2Opts struct { - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListWorkflows2Opts struct { + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListWorkflows2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, localVarOptionals *ListWorkflows2Opts) (AdminWorkflowList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowList ) @@ -9784,36 +9984,36 @@ func (a *AdminServiceApiService) ListWorkflows2(ctx context.Context, idOrg strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idProject Name of the project the resource belongs to. @@ -9830,22 +10030,22 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow&#x @return AdminWorkflowList */ -type ListWorkflows3Opts struct { - IdName optional.String - IdOrg optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListWorkflows3Opts struct { + IdName optional.String + IdOrg optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListWorkflows3(ctx context.Context, idProject string, idDomain string, localVarOptionals *ListWorkflows3Opts) (AdminWorkflowList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowList ) @@ -9914,36 +10114,36 @@ func (a *AdminServiceApiService) ListWorkflows3(ctx context.Context, idProject s if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow` definitions. * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). * @param idOrg Optional, org key applied to the resource. @@ -9960,21 +10160,21 @@ AdminServiceApiService Fetch a list of :ref:`ref_flyteidl.admin.Workflow&#x @return AdminWorkflowList */ -type ListWorkflows4Opts struct { - IdName optional.String - Limit optional.Int64 - Token optional.String - Filters optional.String - SortByKey optional.String +type ListWorkflows4Opts struct { + IdName optional.String + Limit optional.Int64 + Token optional.String + Filters optional.String + SortByKey optional.String SortByDirection optional.String } func (a *AdminServiceApiService) ListWorkflows4(ctx context.Context, idOrg string, idProject string, idDomain string, localVarOptionals *ListWorkflows4Opts) (AdminWorkflowList, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Get") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Get") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowList ) @@ -10041,48 +10241,48 @@ func (a *AdminServiceApiService) ListWorkflows4(ctx context.Context, idOrg strin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowList - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Recreates a previously-run workflow execution that will only start executing from the last known failure point. In Recover mode, users cannot change any input parameters or update the version of the execution. This is extremely useful to recover from system errors and byzantine faults like - Loss of K8s cluster, bugs in platform or instability, machine failures, downstream system failures (downstream services), or simply to recover executions that failed because of retry exhaustion and should complete if tried again. See :ref:`ref_flyteidl.admin.ExecutionRecoverRequest` for more details. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) RecoverExecution(ctx context.Context, body AdminExecutionRecoverRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -10130,49 +10330,49 @@ func (a *AdminServiceApiService) RecoverExecution(ctx context.Context, body Admi if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Recreates a previously-run workflow execution that will only start executing from the last known failure point. In Recover mode, users cannot change any input parameters or update the version of the execution. This is extremely useful to recover from system errors and byzantine faults like - Loss of K8s cluster, bugs in platform or instability, machine failures, downstream system failures (downstream services), or simply to recover executions that failed because of retry exhaustion and should complete if tried again. See :ref:`ref_flyteidl.admin.ExecutionRecoverRequest` for more details. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) RecoverExecution2(ctx context.Context, idOrg string, body AdminExecutionRecoverRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -10221,48 +10421,48 @@ func (a *AdminServiceApiService) RecoverExecution2(ctx context.Context, idOrg st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Registers a :ref:`ref_flyteidl.admin.Project` with the Flyte deployment. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminProjectRegisterResponse */ func (a *AdminServiceApiService) RegisterProject(ctx context.Context, body AdminProjectRegisterRequest) (AdminProjectRegisterResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectRegisterResponse ) @@ -10310,49 +10510,49 @@ func (a *AdminServiceApiService) RegisterProject(ctx context.Context, body Admin if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectRegisterResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Registers a :ref:`ref_flyteidl.admin.Project` with the Flyte deployment. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param projectOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param projectOrg Optional, org key applied to the resource. + - @param body @return AdminProjectRegisterResponse */ func (a *AdminServiceApiService) RegisterProject2(ctx context.Context, projectOrg string, body AdminProjectRegisterRequest) (AdminProjectRegisterResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectRegisterResponse ) @@ -10401,48 +10601,48 @@ func (a *AdminServiceApiService) RegisterProject2(ctx context.Context, projectOr if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectRegisterResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Triggers the creation of an identical :ref:`ref_flyteidl.admin.Execution` - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) RelaunchExecution(ctx context.Context, body AdminExecutionRelaunchRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -10490,49 +10690,49 @@ func (a *AdminServiceApiService) RelaunchExecution(ctx context.Context, body Adm if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Triggers the creation of an identical :ref:`ref_flyteidl.admin.Execution` - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param body @return AdminExecutionCreateResponse */ func (a *AdminServiceApiService) RelaunchExecution2(ctx context.Context, idOrg string, body AdminExecutionRelaunchRequest) (AdminExecutionCreateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Post") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Post") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionCreateResponse ) @@ -10581,51 +10781,51 @@ func (a *AdminServiceApiService) RelaunchExecution2(ctx context.Context, idOrg s if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionCreateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Terminates an in-progress :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. + - @param body @return AdminExecutionTerminateResponse */ func (a *AdminServiceApiService) TerminateExecution(ctx context.Context, idProject string, idDomain string, idName string, body AdminExecutionTerminateRequest) (AdminExecutionTerminateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionTerminateResponse ) @@ -10676,52 +10876,52 @@ func (a *AdminServiceApiService) TerminateExecution(ctx context.Context, idProje if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionTerminateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Terminates an in-progress :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. + - @param body @return AdminExecutionTerminateResponse */ func (a *AdminServiceApiService) TerminateExecution2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, body AdminExecutionTerminateRequest) (AdminExecutionTerminateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Delete") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Delete") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionTerminateResponse ) @@ -10773,51 +10973,51 @@ func (a *AdminServiceApiService) TerminateExecution2(ctx context.Context, idOrg if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionTerminateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Update execution belonging to project domain :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. + - @param body @return AdminExecutionUpdateResponse */ func (a *AdminServiceApiService) UpdateExecution(ctx context.Context, idProject string, idDomain string, idName string, body AdminExecutionUpdateRequest) (AdminExecutionUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionUpdateResponse ) @@ -10868,52 +11068,52 @@ func (a *AdminServiceApiService) UpdateExecution(ctx context.Context, idProject if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Update execution belonging to project domain :ref:`ref_flyteidl.admin.Execution`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User or system provided value for the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User or system provided value for the resource. + - @param body @return AdminExecutionUpdateResponse */ func (a *AdminServiceApiService) UpdateExecution2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, body AdminExecutionUpdateRequest) (AdminExecutionUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminExecutionUpdateResponse ) @@ -10965,52 +11165,52 @@ func (a *AdminServiceApiService) UpdateExecution2(ctx context.Context, idOrg str if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminExecutionUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates the status of a registered :ref:`ref_flyteidl.admin.LaunchPlan`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. - * @param idVersion Specific version of the resource. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. + - @param idVersion Specific version of the resource. + - @param body @return AdminLaunchPlanUpdateResponse */ func (a *AdminServiceApiService) UpdateLaunchPlan(ctx context.Context, idProject string, idDomain string, idName string, idVersion string, body AdminLaunchPlanUpdateRequest) (AdminLaunchPlanUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanUpdateResponse ) @@ -11062,52 +11262,52 @@ func (a *AdminServiceApiService) UpdateLaunchPlan(ctx context.Context, idProject if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates the status of a registered :ref:`ref_flyteidl.admin.LaunchPlan`. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. - * @param idVersion Specific version of the resource. + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. + - @param idVersion Specific version of the resource. @return AdminLaunchPlanUpdateResponse */ func (a *AdminServiceApiService) UpdateLaunchPlan2(ctx context.Context, idOrg string, idProject string, idDomain string, idName string, idVersion string) (AdminLaunchPlanUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminLaunchPlanUpdateResponse ) @@ -11158,52 +11358,52 @@ func (a *AdminServiceApiService) UpdateLaunchPlan2(ctx context.Context, idOrg st if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminLaunchPlanUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates a :ref:`ref_flyteidl.admin.NamedEntity` object. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param resourceType Resource type of the metadata to update +required - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param resourceType Resource type of the metadata to update +required + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' + - @param body @return AdminNamedEntityUpdateResponse */ func (a *AdminServiceApiService) UpdateNamedEntity(ctx context.Context, resourceType string, idProject string, idDomain string, idName string, body AdminNamedEntityUpdateRequest) (AdminNamedEntityUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityUpdateResponse ) @@ -11255,53 +11455,53 @@ func (a *AdminServiceApiService) UpdateNamedEntity(ctx context.Context, resource if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates a :ref:`ref_flyteidl.admin.NamedEntity` object. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param resourceType Resource type of the metadata to update +required - * @param idOrg Optional, org key applied to the resource. - * @param idProject Name of the project the resource belongs to. - * @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. - * @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param resourceType Resource type of the metadata to update +required + - @param idOrg Optional, org key applied to the resource. + - @param idProject Name of the project the resource belongs to. + - @param idDomain Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. + - @param idName User provided value for the resource. The combination of project + domain + name uniquely identifies the resource. +optional - in certain contexts - like 'List API', 'Launch plans' + - @param body @return AdminNamedEntityUpdateResponse */ func (a *AdminServiceApiService) UpdateNamedEntity2(ctx context.Context, resourceType string, idOrg string, idProject string, idDomain string, idName string, body AdminNamedEntityUpdateRequest) (AdminNamedEntityUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminNamedEntityUpdateResponse ) @@ -11354,49 +11554,49 @@ func (a *AdminServiceApiService) UpdateNamedEntity2(ctx context.Context, resourc if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminNamedEntityUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates an existing :ref:`ref_flyteidl.admin.Project` flyteidl.admin.Project should be passed but the domains property should be empty; it will be ignored in the handler as domains cannot be updated via this API. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param id Globally unique project name. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param id Globally unique project name. + - @param body @return AdminProjectUpdateResponse */ func (a *AdminServiceApiService) UpdateProject(ctx context.Context, id string, body AdminProject) (AdminProjectUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectUpdateResponse ) @@ -11445,50 +11645,50 @@ func (a *AdminServiceApiService) UpdateProject(ctx context.Context, id string, b if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Updates an existing :ref:`ref_flyteidl.admin.Project` flyteidl.admin.Project should be passed but the domains property should be empty; it will be ignored in the handler as domains cannot be updated via this API. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param org Optional, org key applied to the resource. - * @param id Globally unique project name. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param org Optional, org key applied to the resource. + - @param id Globally unique project name. + - @param body @return AdminProjectUpdateResponse */ func (a *AdminServiceApiService) UpdateProject2(ctx context.Context, org string, id string, body AdminProject) (AdminProjectUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectUpdateResponse ) @@ -11538,49 +11738,49 @@ func (a *AdminServiceApiService) UpdateProject2(ctx context.Context, org string, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` at the project level - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param body @return AdminProjectAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateProjectAttributes(ctx context.Context, attributesProject string, body AdminProjectAttributesUpdateRequest) (AdminProjectAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesUpdateResponse ) @@ -11629,50 +11829,50 @@ func (a *AdminServiceApiService) UpdateProjectAttributes(ctx context.Context, at if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` at the project level - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesOrg Optional, org key applied to the project. - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesOrg Optional, org key applied to the project. + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param body @return AdminProjectAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateProjectAttributes2(ctx context.Context, attributesOrg string, attributesProject string, body AdminProjectAttributesUpdateRequest) (AdminProjectAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectAttributesUpdateResponse ) @@ -11722,50 +11922,50 @@ func (a *AdminServiceApiService) UpdateProjectAttributes2(ctx context.Context, a if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param attributesDomain Unique domain id for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param attributesDomain Unique domain id for which this set of attributes will be applied. + - @param body @return AdminProjectDomainAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateProjectDomainAttributes(ctx context.Context, attributesProject string, attributesDomain string, body AdminProjectDomainAttributesUpdateRequest) (AdminProjectDomainAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesUpdateResponse ) @@ -11815,51 +12015,51 @@ func (a *AdminServiceApiService) UpdateProjectDomainAttributes(ctx context.Conte if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project and domain. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesOrg Optional, org key applied to the attributes. - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param attributesDomain Unique domain id for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesOrg Optional, org key applied to the attributes. + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param attributesDomain Unique domain id for which this set of attributes will be applied. + - @param body @return AdminProjectDomainAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateProjectDomainAttributes2(ctx context.Context, attributesOrg string, attributesProject string, attributesDomain string, body AdminProjectDomainAttributesUpdateRequest) (AdminProjectDomainAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminProjectDomainAttributesUpdateResponse ) @@ -11910,51 +12110,51 @@ func (a *AdminServiceApiService) UpdateProjectDomainAttributes2(ctx context.Cont if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminProjectDomainAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param attributesDomain Unique domain id for which this set of attributes will be applied. - * @param attributesWorkflow Workflow name for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param attributesDomain Unique domain id for which this set of attributes will be applied. + - @param attributesWorkflow Workflow name for which this set of attributes will be applied. + - @param body @return AdminWorkflowAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateWorkflowAttributes(ctx context.Context, attributesProject string, attributesDomain string, attributesWorkflow string, body AdminWorkflowAttributesUpdateRequest) (AdminWorkflowAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesUpdateResponse ) @@ -12005,52 +12205,52 @@ func (a *AdminServiceApiService) UpdateWorkflowAttributes(ctx context.Context, a if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } return localVarReturnValue, localVarHttpResponse, nil } -/* +/* AdminServiceApiService Creates or updates custom :ref:`ref_flyteidl.admin.MatchableAttributesConfiguration` for a project, domain and workflow. - * @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). - * @param attributesOrg Optional, org key applied to the attributes. - * @param attributesProject Unique project id for which this set of attributes will be applied. - * @param attributesDomain Unique domain id for which this set of attributes will be applied. - * @param attributesWorkflow Workflow name for which this set of attributes will be applied. - * @param body + - @param ctx context.Context - for authentication, logging, cancellation, deadlines, tracing, etc. Passed from http.Request or context.Background(). + - @param attributesOrg Optional, org key applied to the attributes. + - @param attributesProject Unique project id for which this set of attributes will be applied. + - @param attributesDomain Unique domain id for which this set of attributes will be applied. + - @param attributesWorkflow Workflow name for which this set of attributes will be applied. + - @param body @return AdminWorkflowAttributesUpdateResponse */ func (a *AdminServiceApiService) UpdateWorkflowAttributes2(ctx context.Context, attributesOrg string, attributesProject string, attributesDomain string, attributesWorkflow string, body AdminWorkflowAttributesUpdateRequest) (AdminWorkflowAttributesUpdateResponse, *http.Response, error) { var ( - localVarHttpMethod = strings.ToUpper("Put") - localVarPostBody interface{} - localVarFileName string - localVarFileBytes []byte + localVarHttpMethod = strings.ToUpper("Put") + localVarPostBody interface{} + localVarFileName string + localVarFileBytes []byte localVarReturnValue AdminWorkflowAttributesUpdateResponse ) @@ -12102,29 +12302,29 @@ func (a *AdminServiceApiService) UpdateWorkflowAttributes2(ctx context.Context, if localVarHttpResponse.StatusCode < 300 { // If we succeed, return the data, otherwise pass on to decode error. - err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err == nil { + err = a.client.decode(&localVarReturnValue, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err == nil { return localVarReturnValue, localVarHttpResponse, err } } if localVarHttpResponse.StatusCode >= 300 { newErr := GenericSwaggerError{ - body: localVarBody, + body: localVarBody, error: localVarHttpResponse.Status, } - + if localVarHttpResponse.StatusCode == 200 { var v AdminWorkflowAttributesUpdateResponse - err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")); - if err != nil { - newErr.error = err.Error() - return localVarReturnValue, localVarHttpResponse, newErr - } - newErr.model = v + err = a.client.decode(&v, localVarBody, localVarHttpResponse.Header.Get("Content-Type")) + if err != nil { + newErr.error = err.Error() return localVarReturnValue, localVarHttpResponse, newErr + } + newErr.model = v + return localVarReturnValue, localVarHttpResponse, newErr } - + return localVarReturnValue, localVarHttpResponse, newErr } diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_dynamic_node_workflow_response.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_dynamic_node_workflow_response.go new file mode 100644 index 0000000000..a431f23d17 --- /dev/null +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_dynamic_node_workflow_response.go @@ -0,0 +1,14 @@ +/* + * flyteidl/service/admin.proto + * + * No description provided (generated by Swagger Codegen https://github.com/swagger-api/swagger-codegen) + * + * API version: version not set + * Generated by: Swagger Codegen (https://github.com/swagger-api/swagger-codegen.git) + */ + +package flyteadmin + +type AdminDynamicNodeWorkflowResponse struct { + CompiledWorkflow *CoreCompiledWorkflowClosure `json:"compiled_workflow,omitempty"` +} diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_execution_create_request.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_execution_create_request.go index c7eff0aded..bbcb4f1c9a 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_execution_create_request.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_execution_create_request.go @@ -11,11 +11,11 @@ package flyteadmin // Request to launch an execution with the given project, domain and optionally-assigned name. type AdminExecutionCreateRequest struct { - Project string `json:"project,omitempty"` - Domain string `json:"domain,omitempty"` - Name string `json:"name,omitempty"` - Spec *AdminExecutionSpec `json:"spec,omitempty"` - Inputs *CoreLiteralMap `json:"inputs,omitempty"` + Project string `json:"project,omitempty"` + Domain string `json:"domain,omitempty"` + Name string `json:"name,omitempty"` + Spec *AdminExecutionSpec `json:"spec,omitempty"` + Inputs *CoreLiteralMap `json:"inputs,omitempty"` // Optional, org key applied to the resource. Org string `json:"org,omitempty"` } diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_matchable_attributes_configuration.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_matchable_attributes_configuration.go index a9381abc9b..59dde4b013 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_matchable_attributes_configuration.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_matchable_attributes_configuration.go @@ -12,10 +12,10 @@ package flyteadmin // Represents a custom set of attributes applied for either a domain (and optional org); a domain and project (and optional org); or domain, project and workflow name (and optional org). These are used to override system level defaults for kubernetes cluster resource management, default execution values, and more all across different levels of specificity. type AdminMatchableAttributesConfiguration struct { Attributes *AdminMatchingAttributes `json:"attributes,omitempty"` - Domain string `json:"domain,omitempty"` - Project string `json:"project,omitempty"` - Workflow string `json:"workflow,omitempty"` - LaunchPlan string `json:"launch_plan,omitempty"` + Domain string `json:"domain,omitempty"` + Project string `json:"project,omitempty"` + Workflow string `json:"workflow,omitempty"` + LaunchPlan string `json:"launch_plan,omitempty"` // Optional, org key applied to the resource. Org string `json:"org,omitempty"` } diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_named_entity_identifier.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_named_entity_identifier.go index ac55ee330b..6cf6c21ef4 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_named_entity_identifier.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_named_entity_identifier.go @@ -15,7 +15,7 @@ type AdminNamedEntityIdentifier struct { Project string `json:"project,omitempty"` // Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. Domain string `json:"domain,omitempty"` - Name string `json:"name,omitempty"` + Name string `json:"name,omitempty"` // Optional, org key applied to the resource. Org string `json:"org,omitempty"` } diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project.go index fafa42e708..9e8bfb1d68 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project.go @@ -14,12 +14,12 @@ type AdminProject struct { // Globally unique project name. Id string `json:"id,omitempty"` // Display name. - Name string `json:"name,omitempty"` - Domains []AdminDomain `json:"domains,omitempty"` - Description string `json:"description,omitempty"` + Name string `json:"name,omitempty"` + Domains []AdminDomain `json:"domains,omitempty"` + Description string `json:"description,omitempty"` // Leverage Labels from flyteidl.admin.common.proto to tag projects with ownership information. - Labels *AdminLabels `json:"labels,omitempty"` - State *ProjectProjectState `json:"state,omitempty"` + Labels *AdminLabels `json:"labels,omitempty"` + State *ProjectProjectState `json:"state,omitempty"` // Optional, org key applied to the resource. Org string `json:"org,omitempty"` } diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes.go index 7772a1b781..d12a58bb99 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes.go @@ -11,7 +11,7 @@ package flyteadmin type AdminProjectAttributes struct { // Unique project id for which this set of attributes will be applied. - Project string `json:"project,omitempty"` + Project string `json:"project,omitempty"` MatchingAttributes *AdminMatchingAttributes `json:"matching_attributes,omitempty"` // Optional, org key applied to the project. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes_delete_request.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes_delete_request.go index c94bdf8d78..6d444fb2c2 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes_delete_request.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_attributes_delete_request.go @@ -10,7 +10,7 @@ package flyteadmin type AdminProjectAttributesDeleteRequest struct { - Project string `json:"project,omitempty"` + Project string `json:"project,omitempty"` ResourceType *AdminMatchableResource `json:"resource_type,omitempty"` // Optional, org key applied to the project. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes.go index 7f25ab536c..66b4f292b5 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes.go @@ -13,7 +13,7 @@ type AdminProjectDomainAttributes struct { // Unique project id for which this set of attributes will be applied. Project string `json:"project,omitempty"` // Unique domain id for which this set of attributes will be applied. - Domain string `json:"domain,omitempty"` + Domain string `json:"domain,omitempty"` MatchingAttributes *AdminMatchingAttributes `json:"matching_attributes,omitempty"` // Optional, org key applied to the attributes. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes_delete_request.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes_delete_request.go index 6431b0ffc9..1fe2b26dd3 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes_delete_request.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_project_domain_attributes_delete_request.go @@ -10,8 +10,8 @@ package flyteadmin type AdminProjectDomainAttributesDeleteRequest struct { - Project string `json:"project,omitempty"` - Domain string `json:"domain,omitempty"` + Project string `json:"project,omitempty"` + Domain string `json:"domain,omitempty"` ResourceType *AdminMatchableResource `json:"resource_type,omitempty"` // Optional, org key applied to the attributes. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes.go index f8fdf30748..ac58d13889 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes.go @@ -15,7 +15,7 @@ type AdminWorkflowAttributes struct { // Unique domain id for which this set of attributes will be applied. Domain string `json:"domain,omitempty"` // Workflow name for which this set of attributes will be applied. - Workflow string `json:"workflow,omitempty"` + Workflow string `json:"workflow,omitempty"` MatchingAttributes *AdminMatchingAttributes `json:"matching_attributes,omitempty"` // Optional, org key applied to the attributes. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes_delete_request.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes_delete_request.go index 26df12fc94..0a25a0afd5 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes_delete_request.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_admin_workflow_attributes_delete_request.go @@ -10,9 +10,9 @@ package flyteadmin type AdminWorkflowAttributesDeleteRequest struct { - Project string `json:"project,omitempty"` - Domain string `json:"domain,omitempty"` - Workflow string `json:"workflow,omitempty"` + Project string `json:"project,omitempty"` + Domain string `json:"domain,omitempty"` + Workflow string `json:"workflow,omitempty"` ResourceType *AdminMatchableResource `json:"resource_type,omitempty"` // Optional, org key applied to the attributes. Org string `json:"org,omitempty"` diff --git a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_core_catalog_cache_status.go b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_core_catalog_cache_status.go index 855a733fbc..62eb9ba85a 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_core_catalog_cache_status.go +++ b/flyteidl/gen/pb-go/flyteidl/service/flyteadmin/model_core_catalog_cache_status.go @@ -8,17 +8,18 @@ */ package flyteadmin + // CoreCatalogCacheStatus : - CACHE_DISABLED: Used to indicate that caching was disabled - CACHE_MISS: Used to indicate that the cache lookup resulted in no matches - CACHE_HIT: used to indicate that the associated artifact was a result of a previous execution - CACHE_POPULATED: used to indicate that the resultant artifact was added to the cache - CACHE_LOOKUP_FAILURE: Used to indicate that cache lookup failed because of an error - CACHE_PUT_FAILURE: Used to indicate that cache lookup failed because of an error - CACHE_SKIPPED: Used to indicate the cache lookup was skipped - CACHE_EVICTED: Used to indicate that the cache was evicted type CoreCatalogCacheStatus string // List of coreCatalogCacheStatus const ( - CoreCatalogCacheStatusDISABLED CoreCatalogCacheStatus = "CACHE_DISABLED" - CoreCatalogCacheStatusMISS CoreCatalogCacheStatus = "CACHE_MISS" - CoreCatalogCacheStatusHIT CoreCatalogCacheStatus = "CACHE_HIT" - CoreCatalogCacheStatusPOPULATED CoreCatalogCacheStatus = "CACHE_POPULATED" + CoreCatalogCacheStatusDISABLED CoreCatalogCacheStatus = "CACHE_DISABLED" + CoreCatalogCacheStatusMISS CoreCatalogCacheStatus = "CACHE_MISS" + CoreCatalogCacheStatusHIT CoreCatalogCacheStatus = "CACHE_HIT" + CoreCatalogCacheStatusPOPULATED CoreCatalogCacheStatus = "CACHE_POPULATED" CoreCatalogCacheStatusLOOKUP_FAILURE CoreCatalogCacheStatus = "CACHE_LOOKUP_FAILURE" - CoreCatalogCacheStatusPUT_FAILURE CoreCatalogCacheStatus = "CACHE_PUT_FAILURE" - CoreCatalogCacheStatusSKIPPED CoreCatalogCacheStatus = "CACHE_SKIPPED" - CoreCatalogCacheStatusEVICTED CoreCatalogCacheStatus = "CACHE_EVICTED" + CoreCatalogCacheStatusPUT_FAILURE CoreCatalogCacheStatus = "CACHE_PUT_FAILURE" + CoreCatalogCacheStatusSKIPPED CoreCatalogCacheStatus = "CACHE_SKIPPED" + CoreCatalogCacheStatusEVICTED CoreCatalogCacheStatus = "CACHE_EVICTED" ) diff --git a/flyteidl/gen/pb-go/flyteidl/service/openapi.go b/flyteidl/gen/pb-go/flyteidl/service/openapi.go index d8912c8295..22404a1679 100644 --- a/flyteidl/gen/pb-go/flyteidl/service/openapi.go +++ b/flyteidl/gen/pb-go/flyteidl/service/openapi.go @@ -1,6 +1,6 @@ // Code generated by go-bindata. (@generated) DO NOT EDIT. - //Package service generated by go-bindata.// sources: +//Package service generated by go-bindata.// sources: // ../service/admin.swagger.json package service @@ -78,7 +78,7 @@ func (fi bindataFileInfo) Sys() interface{} { return nil } -var _adminSwaggerJson = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xec\xfd\x7b\x73\xe3\x36\xb6\x2f\x0c\xff\x3f\x9f\x02\xd5\xfb\xad\xea\x64\xc6\x76\x67\x2e\x67\xbf\x53\xde\x75\xea\x79\x14\x5b\xdd\xd1\x89\xdb\x76\x7c\x49\x76\xea\x68\x97\x02\x91\x90\x84\x31\x05\x28\x00\x68\xb7\x66\x6a\xbe\xfb\x53\x58\x00\x48\x90\x22\x25\xea\x6a\xca\x8d\x9e\xaa\x89\x2c\x91\xb8\x2c\x00\x0b\xeb\xfa\x5b\xff\xfa\x03\x42\xef\xe4\x0b\x1e\x8f\x89\x78\x77\x8e\xde\xfd\xe5\xec\xbb\x77\x27\xfa\x3b\xca\x46\xfc\xdd\x39\xd2\xbf\x23\xf4\x4e\x51\x95\x10\xfd\xfb\x28\x99\x2b\x42\xe3\xe4\x83\x24\xe2\x99\x46\xe4\x03\x8e\xa7\x94\x9d\xcd\x04\x57\x1c\x5e\x44\xe8\xdd\x33\x11\x92\x72\xa6\x1f\xb7\x1f\x11\xe3\x0a\x49\xa2\xde\xfd\x01\xa1\x7f\x43\xf3\x32\x9a\x90\x29\x91\xef\xce\xd1\xff\x35\x2f\x4d\x94\x9a\xb9\x06\xf4\x67\xa9\x9f\xfd\x1f\x78\x36\xe2\x4c\xa6\x85\x87\xf1\x6c\x96\xd0\x08\x2b\xca\xd9\x87\x7f\x48\xce\xf2\x67\x67\x82\xc7\x69\xd4\xf0\x59\xac\x26\x32\x9f\xe3\x07\x3c\xa3\x1f\x9e\xff\xfc\x01\x47\x8a\x3e\x93\x41\x82\x53\x16\x4d\x06\xb3\x04\x33\xf9\x81\x8b\xf1\x87\x7f\xd1\xf8\x8c\x8b\xf1\xbf\xe1\xc3\x4c\xf0\x7f\x90\x48\x99\x3f\x62\x3e\xc5\x94\x99\xcf\x0c\x4f\xc9\xbf\xb3\x46\x11\x7a\x37\x26\xca\xfb\x53\x4f\x3d\x9d\x4e\xb1\x98\x6b\xf2\x7c\x24\x2a\x9a\x20\x35\x21\xc8\x74\x8a\x1c\xbd\xf8\x08\x61\x74\x2e\xc8\xe8\xfc\x37\x41\x46\x03\x47\xf5\x33\x43\xed\x2b\x18\xda\x6d\x82\xd9\x6f\x67\x96\x66\xd0\x32\x9f\x11\x01\x13\xed\xc5\xba\xf5\x4f\x44\x75\xa0\xd9\xfc\xf9\xbf\xf8\x8f\x0b\x22\x67\x9c\x49\x22\x0b\xe3\x43\xe8\xdd\x5f\xbe\xfb\xae\xf4\x15\x42\xef\x62\x22\x23\x41\x67\xca\xae\x6c\x07\xc9\x34\x8a\x88\x94\xa3\x34\x41\xae\x25\x7f\x34\x66\xae\x7a\x99\xf1\x42\x63\x08\xbd\xfb\xff\x09\x32\xd2\xed\xfc\xc7\x87\x98\x8c\x28\xa3\xba\x5d\x69\x76\x53\x3e\xdc\x77\x85\xb7\xfe\xfd\x87\xaa\xcf\xff\xf6\x66\x34\xc3\x02\x4f\x89\x22\x22\x5f\x7f\xf3\xaf\x34\x17\xbd\x48\xba\x73\xb3\xa2\xe5\x41\x97\x66\x7a\x03\x9f\x70\x72\x82\xb8\x18\xa3\x27\x32\x47\xb0\xa5\x48\x8c\x14\x87\xb5\x13\x44\xf2\x54\x44\x8b\xb3\xa7\xf0\xbe\xde\x66\xe5\x5f\x04\xf9\x3d\xa5\x82\xe8\x65\x52\x22\x25\xa5\x5f\xd5\x7c\x06\xc3\x93\x4a\x50\x36\xf6\x89\xf0\xef\x93\x46\x93\xb2\xbb\x73\xc5\xc4\xae\xf1\x94\xe8\x9d\xa6\xe7\x60\xdf\x28\xcc\x07\x0d\x49\xc2\xd9\x58\x22\xc5\xdb\x33\x35\x73\xd6\xd6\x98\x99\x79\xa1\x76\x62\x7d\xd6\x71\x8f\x44\x98\xa1\x21\x41\x9a\xdd\xd0\x98\x08\x12\x23\x2c\x11\x46\x32\x1d\x4a\xa2\xd0\x0b\x55\x13\xca\xf4\xdf\x33\x12\xd1\x11\x8d\x1c\xcd\xda\x43\x1b\xf8\xb8\x9c\x32\x8f\x92\x08\x3d\xf0\x67\x1a\x93\x18\x3d\xe3\x24\x25\x68\xc4\x45\x71\x1f\xf7\xd9\xc3\x44\xd3\x61\x3a\xa4\x0c\xf8\x89\xa6\xa5\xdb\x21\x7f\x72\xe4\xfa\x13\xd2\xfd\xa1\x94\xd1\xdf\x53\x92\xcc\x11\x8d\x09\x53\x74\x44\x89\x2c\xb7\xf6\x27\x6e\x8f\x10\x3a\x45\x9a\xce\x44\x28\xa0\x37\x67\x8a\x7c\x51\x12\x9d\xa2\x84\x3e\x11\xf4\xfe\x8a\x4a\x85\x3a\xb7\xbd\xf7\x27\xe8\xbd\x61\x02\x08\xd8\xef\xfb\x03\x50\x38\xfb\xfc\x3f\x1e\x3f\x51\x78\x5c\xe6\x24\xef\x3a\x9a\x45\xdd\x9b\xdb\x2f\x6f\xe1\x7f\xfe\xe0\xb7\x63\xd7\x6b\xf5\x95\x62\xee\x93\xfc\x32\xb1\x37\x49\xd3\xfb\x03\x08\x56\xbc\x3a\xa4\x5e\xab\x6d\x6f\x0e\xdd\x6e\xf9\xea\x90\x47\x76\x77\xe8\x39\xec\xfb\xfe\x78\x7b\x97\xc7\x36\x37\x07\x56\x70\xa4\x31\x65\x86\x03\x64\x0c\x41\x48\xcd\x04\xdc\xb0\x5b\x32\xd3\x6d\x2e\x12\x6f\x66\xde\x5d\xe2\xae\x08\x8f\x2a\x2d\x9c\x77\x42\xa7\x74\xd5\xfa\xf6\x58\xac\x45\x66\xcb\xc9\x59\x3a\x1d\x12\xa1\xc9\xe0\x36\x2b\xcc\x76\xa8\x37\xaf\x4a\x05\x23\x71\x83\x69\xfe\x9e\x12\x31\x5f\x32\xcf\x11\x4e\x64\xdd\x44\x29\x53\x44\xeb\x27\xa5\x9f\x47\x5c\x4c\xb1\xb2\x0f\xfc\xe7\xdf\xd6\x25\x84\xe2\x4f\x64\xd5\xfa\xf7\xcc\x6a\x46\x58\xc2\x36\x98\xa6\x89\xa2\xb3\x84\xa0\x19\x1e\x13\x69\x29\x92\x26\x4a\x9e\xc0\x63\x5a\x27\x22\xe2\x34\xbb\x5e\xa1\x07\x27\x56\xa4\xd2\x1c\xfa\x51\x26\xf3\x33\xf2\x45\x41\x4b\x7d\x06\x82\x05\x90\xc8\xbf\x2e\xf7\x40\xca\xcd\xf6\x8c\xe4\x42\x0d\x86\xf3\xb3\x27\xb2\xd0\x6f\xed\xce\xc1\x0c\x61\xa5\x04\x1d\xa6\x8a\xe8\x79\xeb\x36\x1c\xc7\x03\x86\x6f\xa4\x8f\x26\xac\xe1\xf5\x26\x1c\x53\x41\x22\x98\xdb\x3a\x07\x26\x7b\x4b\xcf\x5b\xf3\xfb\xb9\x99\xbd\x66\xff\x5a\xd8\xaa\xa0\x40\xb6\xe4\x7d\xd6\x67\xe8\x14\x5d\x76\xef\x2f\xba\xd7\x97\xbd\xeb\x4f\xe7\xe8\xfb\x39\x8a\xc9\x08\xa7\x89\x3a\x41\x23\x4a\x92\x58\x22\x2c\x08\x34\x49\x62\x2d\x50\xe9\xc1\x10\x16\x53\x36\x46\x5c\xc4\x44\xec\x8f\x8c\xa5\x5f\x09\x4b\xa7\xa5\x9b\x12\xbe\xcf\x47\x5f\x7a\x43\xcb\x4f\xd9\x4f\x85\x5f\xfe\x67\x81\xc0\x30\x63\xdd\xb7\xd7\xda\xab\x4a\x6c\x47\xac\xf7\x1f\x97\xe8\x76\x00\xb5\x3f\x68\xc8\x41\x43\xae\xa6\x4c\xd0\x90\xb7\xa2\xf0\xfe\x55\xa2\x1d\x4b\x03\x87\xbf\x46\x8e\x43\xdd\x3f\xae\x2b\xe3\x10\xda\x7e\xd0\x8d\x83\x6e\x1c\x74\xe3\xa0\x1b\x17\x49\x15\x74\xe3\xa0\x1b\xb7\x4e\x37\x6e\xb0\x8c\x41\x50\xf3\x05\xb5\x68\x42\x93\x58\x10\xf6\x41\x61\xf9\x34\x20\x5f\x48\x94\x1a\x39\x03\xdc\x34\xc5\x2f\x07\x5a\x91\xe0\x31\x29\x7e\x53\xf8\xc3\xf8\x75\xd6\x7e\x2d\x97\x0c\xd7\x7e\x35\xb3\x45\xac\xfd\x26\x58\x2e\x9a\xbd\x07\xdf\xd0\xb8\xf2\x69\xf8\x66\xc5\x1c\xdc\x33\x4b\x06\xeb\x1e\xa9\x1d\x95\x7b\xc0\x0a\xc0\x95\xcf\x08\xa2\xc4\x7c\x80\x95\x22\xd3\x99\x5a\xd3\x2a\x83\x51\xa2\xc5\xec\x65\x62\xf5\x35\x8f\x49\xd7\xf5\xf7\x1b\x32\xd2\x3d\x89\xd1\x70\x6e\x8f\xc5\x88\x08\xc2\x22\x52\xdf\xc2\x03\x96\x4f\x79\x0b\xab\x64\xf3\x42\x7f\xf2\x23\x17\xfa\xf5\xa3\x70\xc7\x15\x46\x7e\x08\x19\x7d\x93\x93\xfa\xc6\x5c\x78\x9b\x72\x9d\x37\x67\x0b\xdb\x90\x87\x06\xcb\xd9\xf6\x94\x6c\x6a\x67\xe3\x02\xc9\xb9\x54\x64\xba\xd2\xe2\x76\x3c\x84\xb0\x97\x64\x5b\x07\x5c\xba\xa7\xbf\x82\x53\x5f\x94\x3a\xc2\xf1\x5e\x83\x64\xbb\xb2\x97\xb7\x7d\x9e\x2e\x64\x79\xf9\x54\xef\xdd\xf2\x79\xee\xba\xa3\x98\x66\x41\x1e\xde\xf5\x20\xf7\x64\x81\xaa\x5d\x2b\x47\xed\x01\x0c\x60\x85\xed\xa1\xe8\x71\xc9\xce\x9f\x7e\xd5\x37\xda\x19\x0b\xad\x9a\x50\xe9\xd9\x2f\x51\xc4\x85\x11\x87\x63\x7b\xde\x8d\xf9\xa1\xf3\xd0\xb9\xef\x3e\x9c\xa3\x0e\x8a\xb1\xc2\xfa\x80\x0b\x32\x13\x44\x12\xa6\xc0\xb4\xa3\xdf\x57\x73\x34\xe5\x31\x49\x8c\x11\xe2\xa3\x96\xfe\xd1\x25\x56\xf8\x02\x2b\x9c\xf0\xf1\x19\xea\xc0\x9f\xfa\x65\x2a\x11\x4e\x24\x47\xd8\x6d\x2b\x12\xbb\x26\x30\x8b\x1d\x6b\xc1\x28\xe2\xd3\x19\x4d\x32\x6f\x53\x66\x6f\xa3\x2c\xa6\xcf\x34\x4e\x71\x82\xf8\x50\x73\x15\x79\xd6\x67\xdd\x67\xc2\x54\x8a\x93\x64\x8e\x70\x92\x20\xdb\xad\x7b\x00\xc9\x09\x4f\x93\x58\xb7\xeb\x46\x29\xe9\x94\x26\x58\x68\x99\xd6\x8c\xf6\xc6\xb6\x85\x1e\x26\x24\x1b\x2b\x8c\x4b\x53\x73\x8a\x9f\x88\x44\x54\xa1\x19\x97\x92\x0e\x93\xfc\xcc\x3f\xf6\x10\x8c\xfb\xe2\xaa\x07\x26\x9e\x48\x21\x6e\x78\xa8\xeb\xdc\x9a\xf4\x5c\x8f\x53\xcc\x18\x81\x8e\xb9\x9a\x10\x61\xbb\xb7\x0f\xbf\xb6\xb5\xe6\xf1\xfa\xfe\xb6\x7b\xd1\xfb\xd8\xeb\x5e\x2e\x9a\x6b\x1e\x3a\xf7\x3f\x2e\x7e\xfb\xcb\xcd\xdd\x8f\x1f\xaf\x6e\x7e\x59\xfc\xe5\xaa\xf3\x78\x7d\xf1\xc3\xe0\xf6\xaa\x73\xbd\xf8\xa3\xdd\x56\x8d\x2d\x3f\xfe\xc8\x76\x76\xb6\x8e\xce\x28\x14\x8c\xfa\x6b\x2c\xfb\xae\x8d\xfa\x27\x6f\xd7\xaa\x3f\xa2\x09\x18\x1d\x1a\x5b\xf4\x33\xab\x91\x7d\x13\xcd\xb0\x94\x46\x0e\x34\x23\x38\xeb\xb3\xcf\x5c\x68\x76\x3d\xe2\x9a\x23\x6a\x59\x51\x89\x34\x52\x94\x8d\xb3\x97\xce\x51\x3f\xfd\xee\xbb\xbf\x46\x57\x94\x3d\xc1\x27\xd2\x46\xe2\x04\x97\x47\x70\x79\xb4\xce\xe5\xf1\x87\x8a\x57\xf7\xef\x1e\x08\x36\xfe\x60\xe3\xdf\x9f\x8d\x3f\x98\xf8\xbd\x31\x04\xfb\xf6\xb6\x84\x08\x06\xb0\x60\xdf\xde\x9e\x10\xc1\xbe\xdd\xd2\x19\x87\xe3\x1d\xec\xdb\xc1\xbe\x1d\xec\xdb\xc1\xbe\x1d\xec\xdb\xc1\xbe\xfd\xd5\xd8\xb7\x5b\x18\xf2\x14\x8c\xfc\xc1\xc8\x1f\x8c\xfc\xc1\xc8\x1f\x8c\xfc\xc1\xc8\x7f\x3c\x46\x7e\x2d\xed\x7e\x28\x87\xfe\xef\x09\xf4\x4f\x0b\x97\x6c\x96\x2a\x10\x25\x79\xaa\xf4\x47\xdd\x3f\xec\x95\x25\x10\x00\xcd\x0c\xca\x9f\x88\xca\x1e\xd4\xa2\xed\x51\xc4\x8a\xff\xc2\xc5\xd3\x28\xe1\x2f\xd9\xc8\x3f\x11\xa5\x07\x7f\x67\x7b\x09\x60\x80\x01\x0c\x10\x05\xa8\x83\x5d\x43\x1d\xb4\xca\x44\x7d\x50\xfe\x7e\xd4\x2c\x3d\x70\xf4\xc0\xfc\x02\xf3\xab\xa3\xcd\x51\x32\xbf\x66\x53\x3b\x3a\xeb\xcd\xfe\x79\x7a\xd1\xde\x95\x0b\xee\x15\x29\xb9\xf5\xd1\x38\xb5\xc1\x36\x35\xb1\x34\x5e\xa8\xcc\x41\xae\x89\x62\x40\xca\x8a\xab\xa2\xf0\xf0\xd1\x68\x00\x85\x51\x1f\xfe\xae\x78\xe3\x79\xa1\x5f\x4b\x58\x4c\x88\x7a\xd9\x90\x50\x6f\xf8\xd2\x3c\x5c\xcc\xca\xe1\x6f\xbb\xaf\xf6\x4e\x0b\x57\x9a\xfd\x17\x18\x7e\x60\xf8\x81\xe1\xbf\x12\xc3\xdf\x80\xee\x41\x85\x5b\xb8\xd4\x2a\x61\x97\x9a\xe2\x2c\xad\x93\x74\xb1\x46\x96\x45\xe3\xb4\x8a\x15\x79\x14\x95\x89\x13\x55\x99\x12\x8b\xa9\x11\x95\xb9\x10\xdb\x25\x3f\x6c\x7a\x57\x37\x4f\x67\xf8\x44\x54\xe1\xe1\xa3\xd1\x3f\x0b\xa3\x3e\xfc\x65\xfd\xea\x61\x3b\xaf\xc5\xa7\xbf\xbe\xd4\x8d\x90\xab\xb1\x47\xd2\xbd\x75\xb1\xa6\xbd\xd9\x18\x5f\x41\xfa\x45\xc8\xb7\x58\x8b\x46\x6f\x2b\xc1\xe2\xad\x66\x54\x1c\x67\x0a\x45\xc8\x99\x08\x39\x13\x3b\x55\x79\x4b\xbf\x7e\x55\x39\x13\xc7\x9c\x24\x71\x78\xf3\x44\x30\x39\xb4\xdf\xe4\x10\x2c\x0e\xf6\x5f\xd0\xbe\xd7\x9e\x79\x10\xed\x83\xf6\xdd\x64\xe6\x41\xfb\x0e\xda\x77\x0b\x8f\x68\xd0\xbe\x83\xf6\x1d\xb4\xef\xa0\x7d\x07\xed\x1b\x05\xed\xdb\x6b\xe8\xb5\x52\x6b\xdb\xe0\xdc\x3c\x2a\x9b\x43\x3e\xf5\x81\x3b\xe0\x8b\xe9\xa8\x05\xb6\xdb\x24\x43\x15\x3e\x39\xa5\x7e\x5d\xc4\xc4\x5a\x2d\xfd\x32\x1f\x6c\x17\x98\xe4\x6f\x96\x55\xac\x50\xd8\x17\xde\x3b\x8a\x18\x81\x85\x51\x87\xb4\xd4\x4d\x45\x9e\x57\x12\x1a\xf6\x44\x81\x23\xb9\xbe\xd6\x5f\xa8\x37\xac\x4b\x06\x1d\x72\xfb\x14\xba\xa3\xd1\x1d\x8f\x47\x67\x3c\xbc\x6c\xf1\x16\xc5\x89\x20\x4d\x78\x63\x08\x17\x6f\xb8\x78\xc3\xc5\x1b\x2e\xde\x70\xf1\x86\x0c\x7c\xfb\xfc\x5e\xe5\x89\x92\x30\xd1\x08\x4a\x6b\xe7\x35\x1c\x2a\x44\x09\xef\x06\x5e\x55\x91\xa1\xfc\x36\x25\xf2\x6f\x47\x29\x53\x1c\xa2\x28\x43\x10\x2a\x8e\x44\xa8\x78\x93\xb6\xa4\x20\x29\x05\x49\xa9\x9a\x32\x8d\x24\xa5\x3e\x7b\x98\x68\x3a\x4c\x87\x94\x65\xde\x3c\xb7\x43\xfe\xe4\xc8\xf5\x27\xa4\xfb\x43\x29\xa3\xbf\xa7\x24\x99\xe7\x3c\x49\x96\x5b\xcb\x90\x3d\xd1\x29\xd2\x74\x26\x42\x01\xbd\x39\x53\xe4\x8b\x92\xe8\x14\x25\xf4\x89\xa0\xf7\x9a\x31\xa3\xce\x6d\xef\xfd\x09\x7a\x7f\x05\x05\x86\xd0\x2c\xc1\x4c\xbe\x6f\x8d\xe3\x26\xc0\x2a\xef\x0b\x56\x39\xa0\x2a\x07\x54\xe5\xa6\x04\x0a\xa8\xca\x01\x55\xf9\x78\x51\x95\x77\xa6\x1f\x6e\x88\xcb\xf9\x2a\x9a\xe2\x71\xfa\xb2\x83\xa6\x88\x82\xa6\x18\x34\xc5\xa0\x29\x06\x4d\xf1\x48\x34\xc5\x76\x50\x38\xa8\x89\x41\x4d\x0c\x6a\xe2\x0e\x89\x13\xd4\xc4\xa0\x26\x06\x35\x71\x41\x4d\x3c\x5e\xcf\xe1\x5f\x83\x3e\x18\xf4\x41\xff\xfb\xe3\xd3\x07\x83\xea\x14\x54\xa7\x6a\xca\x1c\xa7\xea\xd4\x1a\xd9\xe7\x18\x43\x8a\x82\x52\xd8\x9c\x10\x41\x29\x6c\x4c\xaa\xa0\x14\x2e\x21\x4e\x50\x0a\x83\x52\x18\x94\xc2\xc6\x4a\xe1\x31\xb9\x0b\x83\x76\x18\xb4\x43\xff\xfb\xa0\x1d\x06\xed\x30\x68\x87\xc1\xb1\x16\x54\xc3\xa0\x1a\x06\xd5\x30\xa8\x86\xab\x88\x13\x54\xc3\xa0\x1a\x7e\x5d\xaa\x21\x79\x26\x4c\x49\x28\x86\xe8\x2b\x4a\xef\x66\x5c\xd6\x2b\x78\x3e\x77\xa8\x50\xee\xa0\xcd\x62\x51\x42\x40\x6d\xfb\x0d\x4d\xb0\x44\x3c\x8a\x52\x51\x3a\x03\x65\xf5\xee\x42\x10\xac\x08\xb4\xa0\x5f\x3c\x06\xb5\x6e\x71\xba\x87\x02\x20\x1e\xf2\x78\x61\xb7\x9b\x83\x50\xf5\xcb\x72\x31\x6b\x67\x53\xff\x3d\x25\xcd\xb4\xda\x3d\x6e\x6a\x88\x86\x36\x9b\x71\xb1\xda\xd9\x8b\x2d\xaa\xbf\xe3\x5d\xbf\x50\xab\x7f\xa3\x9d\x9f\xb5\xa2\x5f\x3e\x8a\x18\xe8\xea\x79\x1f\xea\x08\x54\x2f\xf2\x1b\x0b\xbb\x7d\xf5\x73\x5e\xb7\xc6\x2d\x3b\xeb\x95\xc5\x0d\xdb\x7d\xc1\x1d\xc5\x11\x7f\xbd\x1b\xae\x76\x5d\xc3\x09\xff\xea\x6e\xf2\x19\x16\x84\xa9\x41\x93\x82\xa6\x0a\xcb\xa7\x1d\x9f\xf9\x42\xa1\x89\x8d\xce\x3c\xb4\x70\x34\x67\x7e\x71\xbe\x87\x3d\xf3\x8d\x57\x3b\x70\x82\xdd\x72\x82\xaa\x85\x6f\x03\x27\x68\xf7\x99\x0e\x47\x1a\xfe\x85\x4d\xbd\xde\xa6\x3e\x1e\x5d\xf4\x18\x36\xf8\xeb\xaa\xa2\xaf\xbe\xc9\xdb\xa9\xa5\x65\x35\xdf\x1a\x6f\xf1\x07\x41\xc7\x63\x22\x8c\xa5\x39\xd2\x5b\xd1\xba\x33\x97\x80\x9e\xe6\x55\xce\x56\x6e\xeb\xec\xd1\x63\xd8\xd2\xd9\x60\xcd\xd8\xbf\x9a\xbd\xbc\x30\xef\x96\x6c\xe2\x22\xd0\x82\x20\x11\x7f\x26\xa2\xf1\xce\xbe\x23\xb0\x9d\x81\x79\xcf\x04\x79\xa6\x3c\x95\xc9\xfc\x54\xa4\x0c\xb9\x9b\x00\x65\x7d\x99\x28\x9b\x17\x9a\x24\x88\xb3\x64\x8e\xa4\xc2\x42\xb9\x9f\xd9\x18\x8d\x04\x9f\xc2\x11\x49\xb0\x54\xe8\x89\xf1\x17\x86\x46\x98\x26\xa9\x20\x68\xc6\x29\x53\x67\x7d\xd6\x63\xe8\xce\x8c\x11\xaa\xa2\x9c\xa0\x54\xea\x83\x15\x61\xc6\xb8\x42\xd1\x04\xb3\x31\x41\x98\xcd\x6d\x79\xc1\x7c\x9b\x20\x2e\x50\x3a\x8b\xb1\x22\xd0\x45\x09\x52\x32\x1b\x23\x84\x1d\x50\x89\xa8\x44\xe4\x8b\x12\x64\x4a\x92\xb9\xee\x43\x1f\x04\xc5\x91\xa5\x8f\x19\xaa\x2d\x56\x46\x84\xe0\x42\x42\x3d\x95\xe1\xfc\x9f\x98\x29\xca\x08\x02\x4f\x88\x34\x21\x05\xa7\xe8\x8a\x4b\xf0\xcb\xfe\xf8\x77\x89\xa2\x24\x95\x8a\x88\x13\x34\x4c\xc7\x12\x51\x86\x66\x09\x56\x23\x2e\xa6\x7a\x84\x94\x49\x85\x87\x34\xa1\x6a\x7e\x82\xa6\x38\x9a\x98\xb6\x80\x06\xf2\xa4\xcf\x62\xfe\xc2\xa4\x12\x04\x67\xbd\xbb\x1f\xd1\x37\xfe\x6f\x66\x37\xc8\x6f\x4f\xa0\xa8\x1a\x9d\xce\x92\xb9\x3f\xfc\x7c\xf9\xcd\x9a\xe8\x46\x48\x8c\x86\x24\xc2\xa9\xb4\x91\x51\x4a\xcc\x11\xf9\x32\xc1\xa9\x84\xb5\xd3\xd3\xb3\x15\x69\x22\x3e\x9d\x25\x44\x11\x44\x47\x48\x09\xad\x79\xe0\x31\xa6\x9a\x74\xf7\x84\x34\xe0\x68\x76\x01\xed\x11\xf8\x0d\xfc\x6b\x53\x2e\x08\x8a\x89\xc2\x34\x59\x1a\x2d\x67\xdf\xcd\xda\x3a\x0a\xd5\xf3\x95\x78\x5e\x50\x27\xf7\xca\xc8\x8b\xdb\xb8\x7d\x9c\x3c\x81\xe0\xa5\x1d\x08\x29\xcc\x46\x55\x45\x38\xd9\x52\x5e\xb9\xb3\x83\x0a\xc7\x37\x1c\xdf\xf2\x48\x0e\x7f\x7c\xcd\x5e\x6c\xe9\xf9\x3d\x58\x62\x73\xb3\x72\xca\x57\x54\xaa\xec\xc9\xe3\xc0\xb2\xca\x86\x7b\x88\xa8\xf4\x37\x79\x58\x43\x10\x77\x08\xe2\xae\xa5\xcc\x71\x06\x71\xb7\x26\x5c\x31\x04\x3c\xef\x29\xe0\x99\xca\x10\xf1\x1c\x22\x9e\x9b\x12\x28\x44\x3c\x87\x88\xe7\xe3\x8d\x78\x5e\x53\x77\xd8\x30\xff\xb5\xce\x35\xb7\x8e\xfe\xf0\x89\xa8\x23\x55\xfa\x83\xe6\x10\x34\x87\xa0\x39\xec\x5c\x73\xe0\xc2\x79\x30\x5a\x50\xb5\x6c\x57\x5c\xda\xbd\xfd\x2e\x26\x09\x51\xa4\xde\xd6\x4a\xc4\x54\x2b\x44\x46\x02\xa1\x4c\x8b\xaa\x63\x41\xa4\xdc\x96\xcd\x66\x0d\x1f\x29\xb3\xcd\xc6\x1f\x8c\xac\x81\xfb\xd6\x4c\x2d\x70\xdf\x37\xc6\x7d\x8f\xca\x6d\xe0\x71\xa8\x43\xf9\x0d\xb2\x5b\x65\x96\xd6\x4b\xea\x8f\x26\xb6\x21\x0f\xb6\x30\x3b\x5c\xab\x5b\x8a\x67\x87\xdb\xee\xf3\x2d\x6f\x19\xd3\xd7\x91\x5e\x31\x66\xf0\xe1\x7e\x09\xf7\x4b\xcd\xd4\xc2\xfd\x12\xee\x97\xd7\xbb\x5f\x1c\x7b\x6a\x95\x53\x9a\x8b\x71\xc1\x64\xb4\xec\x22\x3a\x54\xb0\xeb\x71\xdd\x3a\x87\x0d\x1d\x79\x7b\x57\x4e\x7b\x0e\x68\xdb\xe2\x77\x43\xc8\x6e\x08\xd9\x3d\xaa\x90\xdd\xc0\xb7\x8f\x80\xcb\xb5\x2e\xb8\xf5\x38\xe2\x59\xc3\xde\x3e\x8a\xbd\xdd\xb6\xc8\xcf\x56\x07\x7b\x1e\xd5\x9e\x3e\x50\xac\x67\x30\x7f\x04\xf3\x47\x35\x65\x42\x58\x64\x80\xb7\x5d\x9c\x56\x88\xf6\x0c\xd1\x9e\x21\xda\x73\x97\xc4\x09\xd1\x9e\x21\xda\xf3\xab\x8d\xf6\x6c\x79\x80\xe7\x51\x69\x0c\x41\x5b\x08\xda\x42\x70\x96\xae\x39\xb5\xa3\x93\xd1\x77\xc5\x99\xdd\xdb\x2d\x8a\xf0\x3c\x2a\x6e\xfb\x1a\x01\x9e\x81\xfd\x06\xf6\x5b\x4d\x99\xa3\x64\xbf\xed\x31\xa4\x87\x58\xc8\x85\x58\xc8\xa3\x62\xc6\x07\x0f\x85\x0c\x9c\x38\x70\xe2\x6a\xca\x04\x4e\x7c\xfc\x51\x83\xc6\xab\x3a\x98\x25\x98\x0d\x68\xec\x85\x0e\x7e\xf8\x57\x6e\xac\xd8\x97\x67\x53\x1f\xad\xd8\x54\x21\xcd\xaa\x7e\x8a\xdf\xf4\x2b\x49\xee\xe8\x40\x7c\xa8\x87\xb1\xb2\x1e\xab\xf1\x8d\xdc\x26\x98\xf5\xe2\xe3\x00\xbb\xa9\x9c\xfe\x21\x9c\xa1\x6f\x2f\xd4\x70\x9b\x4b\x0a\x2b\x70\xba\x61\xca\x8c\xd9\x35\xaf\x26\x5b\x30\x2a\xb7\x63\xa2\xdb\x5c\x59\xde\xc4\xbc\x5b\xcb\x5d\x46\x1e\x51\xda\x37\xed\xe0\x8b\x0b\xa5\x26\x83\xb7\xa9\xe1\x84\x83\xb7\xa9\xbd\xde\xa6\x06\xcb\xb8\x17\x17\xf2\x81\x8f\xe7\x41\x65\xd6\xa3\x96\x54\x83\xa0\x8a\x82\x58\x17\xc4\xba\xfa\x59\x07\xb1\x2e\x88\x75\x41\xac\x0b\x62\x5d\x10\xeb\x5e\x5f\xac\x6b\x30\xcd\xaf\x36\xca\x60\x95\xa8\xda\xbc\xf4\x90\xc9\xf1\x81\x54\xc0\x74\x96\x70\x1c\x2f\x8b\xf4\xca\x85\xc9\xdf\x50\x2e\xb8\x2d\x91\x40\x4d\xeb\xf9\x6b\xc7\x20\x80\xe6\xa3\xfd\xca\xf2\x9f\x16\x27\xde\x16\x6f\x41\x11\xbe\xb2\xa5\x7b\xfb\x28\xdc\x00\xaf\xb5\xb9\xdf\x24\xcc\x4d\x38\xb1\x0d\x4f\xec\xe1\xf2\x17\xab\x4f\xf1\x1a\x56\x12\xf9\xb7\xe3\x3a\xc6\xa1\x76\x45\xc0\xa8\xaa\x98\x5a\x88\x36\x09\x49\x9a\x21\x9b\xf1\xcd\x99\xda\x42\x36\x63\xc8\x66\x0c\x86\xc8\xe5\xd3\x0e\x86\xc8\x37\x91\xcd\xb8\xbe\x32\xb1\x61\x72\xe3\x61\xd4\x8a\x23\xb3\x0e\x04\xb5\x22\xa8\x15\x15\x53\x0b\x6a\xc5\x57\xa8\x56\xb4\x83\xc2\x41\xa7\x08\x3a\x45\xd0\x29\x82\x4e\x11\x74\x8a\x9d\x93\x31\xe8\x14\x0d\x74\x0a\xf8\x64\x21\x86\xd7\x56\x30\xd6\x54\x2c\x56\xe0\xa8\x1c\xad\xcf\x31\x68\x14\x41\xa3\x08\x1a\xc5\xc1\x35\x8a\xd6\x4c\xc8\xb2\xcf\x15\x73\xba\x77\x0b\x52\x02\x74\x6f\xdf\x7c\xdc\x88\x06\xd0\xd2\x0a\x61\xa2\xa8\xa0\x65\xbb\x4e\xbf\xea\xeb\x28\x26\x9a\x1c\xc4\xf2\x3c\xd8\x1a\x45\x5c\x18\xa6\x1c\xdb\x5d\x6e\xe4\x89\xce\x43\xe7\xbe\xfb\x70\x8e\x3a\x28\xc6\x0a\xeb\x6d\x2d\xc8\x4c\x10\x49\x98\x02\x59\x8d\x40\x1c\x3d\xc0\xea\x27\x46\xaa\xf8\xa8\xef\x1f\x74\x89\x15\xbe\xc0\x0a\x27\x7c\x7c\x86\x3a\xf0\xa7\x7e\x99\x4a\x84\x13\xc9\x11\x76\xa4\x27\xb1\x6b\x02\xb3\xd8\x1d\x28\x0c\x68\xf1\x34\xc9\x94\xd3\x4c\xbd\xa0\x2c\xa6\xcf\x34\x4e\x71\x92\xa5\x27\xf4\x59\xf7\x99\x30\x95\xe2\x24\x99\x23\x9c\x24\xc8\x76\xeb\x1e\x70\x00\xf4\x43\x92\x8d\x52\xd2\x29\x4d\xb0\xd0\xec\xd8\x8c\xf6\xc6\xb6\x85\xb4\x62\xec\xc6\x0a\xe3\xd2\xd4\x9c\xe2\x27\x22\x11\x55\x68\xc6\xa5\xa4\xc3\x24\x3f\x00\x8f\x3d\x04\xe3\xbe\xb8\xea\x81\xcc\x16\x29\xc4\x0d\xe7\x70\x9d\x5b\x05\xc6\xf5\x38\xc5\x8c\x11\xe8\x98\xab\x09\x11\xb6\x7b\xfb\xf0\x6b\x8b\x5f\x8f\xd7\xf7\xb7\xdd\x8b\xde\xc7\x5e\xf7\x72\x51\xfe\x7a\xe8\xdc\xff\xb8\xf8\xed\x2f\x37\x77\x3f\x7e\xbc\xba\xf9\x65\xf1\x97\xab\xce\xe3\xf5\xc5\x0f\x83\xdb\xab\xce\xf5\xe2\x8f\x76\x5b\x35\x16\xe5\xfc\x91\xed\x43\x96\x73\x6f\x37\xc0\xf4\xb0\x87\x4b\x61\x95\x4a\x53\x53\x46\x90\x31\x95\x0a\xd8\x7f\x13\x29\x6c\x35\x94\xc7\xd1\x4a\x5f\xa1\xb0\x59\x90\xc5\x82\x2c\x16\x64\xb1\x63\x93\xc5\x0e\x67\x12\x38\xa2\x30\xc5\xbf\x1e\xd7\xdd\x13\xca\x2e\x04\xe6\xdc\x7e\xe6\xdc\x3a\xd7\x5b\x6b\x4c\xe7\xc7\x08\xe9\x1a\x9c\x8a\xcd\x09\x11\x9c\x8a\xcd\x69\x15\x9c\x8a\x4b\x88\x13\x9c\x8a\xc1\xa9\xf8\x15\x3b\x15\x8f\x32\x36\x31\xa8\x12\xee\xb9\xa0\x4a\x04\x55\xe2\x8d\xaa\x12\xad\xa1\x70\xd0\x23\x82\x1e\x11\xf4\x88\xa0\x47\x2c\x27\x4e\xd0\x23\x82\x1e\x11\xf4\x88\x63\x8b\x47\x3c\x2e\x4d\x22\x68\x11\x41\x8b\x68\xb7\x16\xd1\x9a\x09\x1d\x8f\xb7\xb8\xd9\x7c\x42\xe4\x5e\x88\xdc\x0b\x91\x7b\xb5\x91\x7b\x6f\x54\x93\xdf\x95\xfc\xe6\xde\x6e\x5b\x40\xe2\x71\x89\x5f\xa1\xba\x58\xf6\x6b\x10\xc6\x82\x30\xf6\x95\x0a\x63\x2d\x02\x51\x6c\x45\x91\xb4\x29\x56\xd1\x04\x0f\x13\x32\xc8\x6c\x59\xb2\xa9\x7a\x7f\x45\xa5\x92\x28\x4a\xa5\xe2\xd3\xfa\xcb\xe5\xb3\xeb\xa1\x93\x75\x70\xc1\xd9\x88\x8e\x53\x73\xb7\xfc\x06\x5b\xdf\x3b\xd1\xb9\x80\x3b\x9f\x91\x55\x7e\xc5\x8a\xd6\x8f\xe2\x5a\xaa\x1e\xfa\xa1\x6e\xa7\x75\xf4\x91\xdc\x76\x69\x95\x09\x2d\x42\x0e\xee\xba\xf7\x37\x8f\x77\x17\xdd\x73\xd4\x01\x11\x0b\xdc\x09\x66\x2b\xd0\x7f\xea\x49\x21\x85\xe5\x53\xbe\x96\xc2\x6c\x73\x09\x72\x36\xf8\x2f\xb4\xc8\x8f\x4e\xd1\xc5\xd5\xe3\xfd\x43\xf7\xae\xa6\x41\xbb\x51\xa0\x54\x2a\x99\xce\x12\xac\x48\x8c\x9e\xd2\x21\x11\x8c\x68\x69\x27\x4a\x52\x2d\xdc\xe4\x5e\x0d\xd3\x68\xf7\xbf\xbb\x17\x8f\x0f\xbd\x9b\xeb\xc1\x4f\x8f\xdd\xc7\xee\x39\x72\x3b\x4e\x37\xab\xc7\xa5\x47\x11\xcf\x19\x9e\x6a\xc5\x4a\x7f\x91\x17\x67\xfd\x3d\x25\x29\x41\x58\x4a\x3a\x66\x53\xc2\x54\xb9\x45\x37\xe0\xab\xce\xf7\xdd\xab\x62\xcb\x13\x82\x7e\xfc\x7b\x3e\xa8\x04\x0f\x49\x62\xdd\x2c\xe0\x39\xd0\x1b\x3d\xef\xc8\xfa\x5f\x52\x43\xd5\x9f\x1e\x3b\x57\xbd\x87\x5f\x07\x37\x1f\x07\xf7\xdd\xbb\x9f\x7b\x17\xdd\x81\x15\x96\x2f\x3a\xba\xdf\x42\x4f\x56\xa6\x46\xbf\xa7\x38\xd1\x4a\x17\x1f\x81\xdf\x82\x46\x04\xbd\x4c\x08\x43\x29\x83\x1d\x67\x34\x39\xad\xde\x65\x9d\xea\x53\x66\x66\x74\x7b\xf5\xf8\xa9\x77\x3d\xb8\xf9\xb9\x7b\x77\xd7\xbb\xec\x9e\xa3\x7b\x92\x80\xae\xe3\x88\x0e\xab\x38\x4b\xd2\x31\x65\x88\x4e\x67\x09\xd1\xd4\x30\xba\xdc\x90\x4c\xf0\x33\xe5\xc2\x1e\xdd\x31\x7d\x26\xcc\xd0\x11\xce\x2c\xb4\xef\x74\x8a\x81\x47\xba\x9b\xeb\x8f\xbd\x4f\xe7\xa8\x13\xc7\xd9\x1c\x24\xb4\x51\xd8\x39\x2f\x5c\x3c\x8d\x12\xfe\x72\x5a\x1c\xb6\x66\x0e\xd0\xbd\xd9\x44\xfc\x99\x08\x41\x63\x52\xda\x47\x9d\xfb\xfb\xde\xa7\xeb\xcf\xdd\xeb\x07\xa0\x98\x12\x3c\x91\x68\xc2\x5f\xc0\x42\x0f\x33\x04\xc3\xfd\x33\xa6\x09\x74\xe6\x16\x8b\x33\xf4\x32\xa1\xe0\xd5\xa1\xd2\x27\x98\x51\x3b\x45\xca\x5e\xdd\xe8\x5c\x38\x78\x8b\xda\x58\xf9\x24\x2d\x3e\x51\x3a\x16\xcb\x1e\x28\xec\xf2\xc5\x07\x57\xed\xd6\xc5\x37\x4a\xdb\xad\x5e\x07\x5d\xd8\x2f\xf5\x33\xcd\xd7\xba\xb1\x0a\x5a\xa4\xe1\x9a\xc2\xc3\xba\x1a\xa8\xf1\x81\xf9\x4a\x28\xb8\xd4\x9c\x98\xef\x78\xe2\x71\x69\xa3\x8d\xc5\x88\xbc\xe0\xea\x91\x0b\x14\xc7\x91\x78\xf7\xba\x12\xc5\x61\x8f\xc6\xa1\xb5\x86\x20\x2f\x05\x79\x29\xc8\x4b\x41\x5e\x0a\xf2\x52\xf6\x6f\xcf\xf2\x04\x51\x82\x46\xf2\x43\xb6\xaf\xf6\x0b\xca\x4a\xa4\xde\xb0\x8a\x4e\x09\xb2\x3d\xdb\x93\x5a\x2b\x84\x64\xa5\xee\x97\x5a\xcc\x3f\x11\x95\x3d\xf8\xd9\x34\x7c\x14\xc2\xc4\x2f\x96\xa3\x64\x83\xff\x44\x94\x1d\x7f\x48\xe8\x0f\x09\xfd\x35\x53\x0b\x5e\x81\xed\xbd\x02\x5c\x20\x39\x97\x8a\x4c\x8f\xc4\x3f\x10\x93\xd9\x62\x87\xa5\x89\xc1\x33\x26\xbe\x6b\x21\x1c\xd9\x78\xce\x6d\xec\x40\x42\x9e\x49\x02\x82\xac\x12\xf8\x99\x08\x69\xc5\xb3\xa1\x20\xf8\x49\xcb\xb4\x31\x7f\xf1\x85\xb3\x98\x28\x4c\x93\x7d\xe8\xcf\x4d\xc2\x95\xff\xfa\x97\x57\xbd\x0f\x8f\xf7\x0a\x0c\x37\x60\x70\x21\x87\xcb\xe2\x2b\xbc\x2c\x8e\x31\x88\x27\xdc\x81\x6d\xb9\x03\x35\xb9\xe3\x81\x0b\xe7\xfb\xf0\xaf\x82\x51\xee\xdf\xfb\xd2\x0f\xef\x20\x3f\x4a\x2e\xbb\x08\x35\x97\x8a\xbb\xb0\x7c\xbf\xd9\x20\xc0\x15\x37\xa2\xf7\xc6\x51\xe8\x83\xde\x78\xdb\xe4\x9a\xbe\xf3\x0d\xfb\xee\x9e\x98\x12\x85\x63\xac\xb0\x3e\x42\x63\xa2\xce\xd0\x0d\x83\xdf\x1e\xb0\x7c\x3a\x41\xee\x5e\xd7\xbc\x33\x0f\xc5\xf0\x53\x74\xf6\xc4\x2a\x1b\xda\xac\x5e\x3b\x78\xb3\x8d\x1c\x3d\x68\xe9\xbb\x9c\x5a\x10\xbc\x42\x3a\x76\x9b\x71\xf3\xdc\xdb\x8d\xa2\x9d\x77\x77\x2b\x9b\x16\x8f\xf8\x62\x3e\x6c\x7c\xf3\x4e\xaf\xe9\x14\x86\x1e\xee\x60\xf3\x2f\xdc\xc1\xe1\x0e\x0e\x77\xf0\x32\xca\x84\x3b\xf8\x88\x03\xe0\x2b\xae\xac\x57\x8d\x80\x6f\x62\x56\x30\x36\x85\xdc\xa0\xb0\x26\x50\x6f\x6e\x43\x58\x09\xaf\x55\x21\xb5\xac\xc4\xd6\xca\xdf\xa1\x47\x12\xac\xe6\xcd\xf2\x10\xf8\x5a\x3b\x15\x56\x0c\xc2\x4b\xb0\x2a\xec\x51\xa2\x79\x7b\xe2\xcc\x36\xb2\x0c\x56\x70\xc9\x60\xca\xcc\x9d\x94\x67\x7b\xcb\x03\xec\xb0\x35\x27\xba\x8d\x64\xe3\x4d\xcc\x13\x6e\x9c\xcc\xe2\x11\xa5\x25\x8b\xba\x17\x18\xae\xd7\xb1\xe3\xbf\x3e\xf4\x16\x79\xbb\xc8\x5b\x01\x5c\x6a\xad\x43\x12\xc0\xa5\x0e\x01\x2e\xd5\x60\x19\xf7\x82\x18\x77\xe0\xe3\xf9\x9a\xaa\xc3\xf1\x38\x21\x8f\x4c\x67\x38\x2a\x7d\x21\xf8\x20\xf7\x6f\xff\x0c\xa6\xc2\x60\x2a\xac\xa6\x4c\x30\x15\x7e\x5d\xe1\x5a\xbb\xba\xef\xdd\xdb\xaf\xed\x85\x3c\xb2\x9b\x39\x38\x21\xc3\x25\x1c\x2e\xe1\x70\x09\x87\x4b\x78\x57\x14\x0e\xfe\xba\x35\x95\xee\xa3\xf0\xd2\x1d\xd9\xb5\x1e\x9c\x74\xe1\xc6\x77\x2f\x07\x97\xd6\xaa\x79\x06\x97\x56\x70\x69\x05\x97\x56\x70\x69\x05\x97\x56\xf6\x7d\x70\x69\x1d\x74\xb7\x7e\xb5\x66\xbb\x6a\x8d\x81\xc7\x64\x50\x81\x24\x93\x7d\x35\xf0\xd3\x06\x0b\xdf\x16\xdc\x77\x85\x5f\x7c\x5f\x5e\xe1\x87\xbc\x5c\x0e\xf4\x4b\xe3\xb5\xf3\xf0\x97\x99\x13\x79\x4c\x1a\xe7\xdd\x17\x1e\x6e\x7b\x5c\xa0\x9b\xa8\x51\x3d\xfc\x91\x1f\x20\xcf\xbe\xbc\x13\xde\x58\xf4\x59\xcd\xae\x7e\x8b\x96\xba\x8a\x33\x1a\xcc\x76\x2b\x09\xf5\x86\x01\x07\x2c\x13\x3e\xc0\x78\x0e\x7f\x87\x39\xe0\xbf\x41\xc5\x4d\x56\xfd\x5b\x7e\x9f\x55\xff\x9e\xdd\x6a\xd5\x3f\xef\xa7\xa4\x74\xf3\x4b\x0d\xec\x68\xfe\xd3\x47\x12\xee\xee\x0f\xf9\x10\xb6\xb4\xda\x8d\xf1\xc6\x2e\xb6\xa5\x9b\xfc\xcd\x5d\x6f\xcb\x8e\x6c\xb8\xe4\x1a\x92\xeb\xad\x5e\x75\xa1\x26\xf4\x32\x2b\x5e\x28\xba\xdc\x42\x2b\x45\x30\x22\x06\x23\xe2\x31\x1a\x11\x4d\xf0\xc1\x60\x86\x05\x61\xaa\x42\xb7\x28\x5f\x27\xf0\xb8\x5f\xa4\xd3\x49\x1d\xd0\x00\xd2\xa2\x3d\xb2\x17\x72\x76\x55\xbd\xb1\x40\xf9\x92\xf6\xf2\x86\xed\x6c\x2d\xd7\x47\xda\x63\x66\x7b\xab\x52\x7a\x30\x42\x05\x23\x54\x79\x9e\x87\x33\x42\x6d\x40\xf7\xe0\x2c\x5a\x7a\x55\x7d\x25\x86\xb4\x96\xdf\x5b\x6d\xb2\xa3\xbd\xd5\x9b\x2b\xd8\x97\x76\x40\xae\xb7\x7a\x8b\xbd\x96\x49\xf9\xf0\x5a\x7a\xb0\xa4\x05\x4b\x5a\xb0\xa4\x05\x4b\x5a\xb0\xa4\xf9\xdf\x07\x4b\xda\x32\xba\x1f\x4c\x3d\xb1\x22\x90\x5f\x66\xf5\x5f\xf9\xe7\x4c\x2d\xf1\x55\x8b\x65\x19\xac\x17\x82\xc0\xa9\xe0\xc2\xe6\x3f\xee\xa2\xfc\x2a\x56\x05\xa1\x17\x2a\x1a\x2c\xd1\x45\x4c\x56\xd1\xad\x79\xfa\xb8\xca\xb8\x2f\x0c\xfa\xb0\xf9\xaf\x8b\x0b\xdf\xec\x00\xb9\x95\xa1\xa6\xc4\xa6\x57\xc7\x51\xcb\xdd\x7c\x94\x5f\x0c\x12\xbd\xd0\x24\xd1\x92\x8c\x95\xda\x5a\x22\x8d\xbe\x7a\x56\x5c\xed\xca\xbf\x6a\x6e\x5c\x15\x77\xa8\x62\x09\x4d\xac\xe3\xbb\x2a\xc3\xec\x36\x1b\x94\x90\x05\xbd\x6f\x85\x39\xfd\x6d\x70\x82\x4f\x44\x1d\x8a\x0d\x6c\x7a\xf6\x97\x9e\x7b\x41\x46\x44\x10\x16\x91\x16\x66\x64\xad\x93\x2a\xf8\x8b\x99\xa4\xcd\x13\xcc\xea\x94\xfb\x53\x55\xdc\xea\x69\x05\x51\x37\x54\x6f\x0e\xd5\x9b\x43\xf5\xe6\xf2\x51\x0f\xd5\x9b\xdf\x46\xf5\xe6\x06\x5c\x76\x07\xb6\xbd\xe5\x06\xdb\x96\xea\x5d\xee\xed\x77\x31\x49\x88\x22\xb5\x82\xd2\x25\xfc\xfc\x5a\x82\x92\xe9\xfd\x6d\xc8\x4a\x66\x2e\x41\x5c\xfa\x6a\x94\x25\xb7\xe0\xad\x50\x96\xcc\x59\xf3\x75\x26\x48\xae\xf0\x94\x6b\x93\x51\xf1\x36\xcd\x2c\x47\x91\x40\xd1\x1e\x3b\xcb\xde\x2f\xc5\x43\x33\x87\x60\x43\x5a\x1c\x49\xb0\x21\x6d\xc7\x16\x0b\x5f\x56\xa0\x30\x1d\x9c\x55\x6e\x22\x60\x15\xf8\xe5\x25\x3c\x7e\x9c\x5c\xb3\x3c\xf6\x63\xe6\x9d\x5e\x6b\xed\x60\x22\x5f\x2f\xfb\x5c\x38\xe2\xcd\x26\x6e\x43\x7f\x8e\x77\xde\x6d\xb9\x36\xea\x8e\x75\x9b\x2f\x8f\x62\x1d\xad\xe3\xf5\x48\x1c\x13\xfb\x7f\x15\x97\xc4\x9b\x13\x93\xbf\x3a\xa3\x41\xf0\xb1\x04\x1f\x4b\xf0\xb1\x04\x1f\x4b\xf0\xb1\xa0\x4d\x7d\x2c\xbb\x92\xb4\x8e\xda\x21\x71\x9c\xa2\xd2\x61\x3d\x12\x41\x5a\x3a\x76\x69\xa9\x2d\x4a\xe1\x71\xb9\x58\xb6\x2c\xab\xdc\x16\xbd\xf0\x2d\x19\x07\x8f\x4b\x47\x6c\x9d\x39\xf0\xab\x63\x7c\x9b\x99\xfe\x8e\x76\xba\x41\x2b\x0e\x5a\x71\xd0\x8a\x83\x56\x1c\xb4\x62\x14\xb4\xe2\xb5\xb5\xe2\xb7\x24\x28\x1e\x9d\x86\x1c\x64\xc5\xd7\x9e\xf0\x57\x26\x2b\xb6\xc5\x26\x50\x77\x72\x5b\x6a\x19\xf8\x3a\x03\x8a\x8e\xf8\x26\x08\x39\xaf\xc8\x5b\xba\x10\x70\xf3\xb5\xf0\xd1\x96\x07\xdc\xbc\x3d\xbb\xea\x11\xf3\xc8\x90\x0d\x1c\xc4\xca\x1d\x4d\x37\x98\x20\x83\x09\x32\x98\x20\x83\x09\x32\x98\x20\x51\xbb\x93\x9f\x57\x1a\x9c\x42\xfe\xf3\xbe\x0c\xab\x47\x2c\x29\x86\x5c\xe8\x20\x2c\xee\x6e\xba\x6d\xd5\x9d\xdb\x64\x83\x94\xeb\xd7\x8d\x58\x89\xc4\x6d\xa7\xfd\xdb\x12\x06\x76\x45\xa5\x53\x74\x8f\x89\x5f\xc9\x7d\xb3\xa4\x6d\xe0\x77\xdd\x8a\xbe\x59\xf4\xdd\x2d\xaa\xd1\x03\xef\x7a\xa3\xe5\xe8\x03\x38\xf0\x12\xe2\x04\x70\xe0\xb5\xb8\x49\x00\x07\x6e\x09\x38\xf0\xba\x2a\x98\x39\x95\xbe\x16\x06\x87\xdc\x49\xad\xce\x2c\x75\xa4\xca\xd8\x8c\xcb\x7a\xc9\xe4\x8e\x8c\xa9\x04\x96\xb4\xa4\xa6\x95\x93\x49\xa0\xb0\x02\x6c\xf5\x8f\xfa\x01\x14\x93\x59\xc2\xe7\x60\xff\x5a\x22\xae\xb8\x2e\x6e\x17\x34\x86\xb6\x4b\x2c\x6e\xe4\x87\xd2\xa9\xda\x22\x73\xe7\xf3\x6e\x85\x94\x9d\x87\xfc\xbf\xbe\xbc\x7d\x4c\x81\x57\x7b\x17\xb8\x0f\xcb\x67\x8f\xa9\x2e\x6e\x50\x27\x82\x3a\xd1\x64\xd7\x04\x75\x62\x15\x81\x82\x3a\x11\xd4\x89\x7d\xaa\x13\x07\x96\x60\x3e\xfc\xab\x54\x6e\x76\x59\x00\xe2\xa3\x8d\x3a\x04\xe7\x2c\x95\x70\xe4\x57\x0a\x32\x7d\x56\xfd\x83\x73\x52\x0e\x89\xe3\x31\xc3\x54\x79\x65\xd7\xa4\xbe\xb4\x66\x44\xa8\xb9\xf7\x24\x99\xce\xd4\xfc\xbf\xfa\x8c\xaa\x2c\xc4\x8b\x8e\x19\x17\x66\xc7\xe8\x97\x27\x98\xc5\x89\xbe\xd4\x65\xd6\x4e\x84\x19\xe3\x0a\x58\x39\x4c\x20\x46\xcf\x14\x1b\xc6\xdf\xb9\xed\x35\x0e\x74\x3c\x26\x51\xeb\xb0\x91\x8c\xfb\xaf\x9d\x76\xf8\xca\xa2\x2b\x26\xf4\x29\xe1\x43\x9c\x24\x73\x94\x16\x3d\x4a\xba\x81\x96\xcc\xa1\x2d\xda\x5b\x3b\xd4\x35\x07\x2b\x50\x56\xdb\x5a\x66\x8d\x38\x26\x26\x73\x68\x73\x84\xb7\x84\x6f\x8c\xdd\xb4\xe5\xa8\xb6\xcd\xd0\x12\x84\x93\xa6\xc2\xc9\x11\xb1\x8d\xc3\xca\x26\xe1\x26\x3f\xfe\x9b\x5c\x61\xf9\xe4\x57\x32\x87\x0b\xdd\x15\xa3\x2f\x54\xf1\x2d\x97\xf4\xfd\x77\x93\xe7\xf2\xf4\x82\xd5\xcf\x66\xd5\xd0\x57\x3f\x0a\x95\xd1\x6b\x1e\xb4\x65\xf4\xcd\xcf\x30\xbd\xf2\x38\xdc\x97\x7e\x87\xee\xbb\xbc\x65\xf7\xcd\x33\x11\x92\x72\xfb\x98\x20\x4a\xcc\x07\x58\x29\xcd\x90\x36\xb0\x51\xd7\x72\xcd\x07\x2c\x9f\x9a\x55\x65\xff\x44\x54\xe1\xe1\xb6\x8b\x35\x6e\xa2\x30\xcf\xc2\xc8\xf7\xcf\x9f\x1a\x6c\xe3\x37\x26\xee\x34\x3e\x92\x2b\xe6\x7d\x7c\x85\xe9\x9b\x32\x98\x35\x26\xfe\xb5\x14\xa9\x6f\xc6\x70\x57\xc5\x43\x1e\x63\xc1\xfa\x65\x37\x48\x6b\x46\x58\xba\xc4\xde\xe2\xc9\x2d\x5e\xc9\xe1\x88\x2e\xa3\x51\xd3\xb3\x78\x34\x27\xb0\x24\x69\xad\x98\xdb\xbd\x5b\x20\xfb\xb8\xdb\x09\xed\x9b\x57\x41\x58\xdc\xf5\xa8\xf6\xe3\x00\xf6\x56\x63\x9d\x14\xc2\x9e\x2b\xb6\x6e\xdc\x59\xd9\x19\x72\x49\x85\xd9\xd1\x54\x13\x28\x48\x45\xa5\x5f\x9f\x3d\xe2\xc2\x48\x9b\xb1\x3d\xb3\xc6\xa1\xd5\x79\xe8\xdc\x77\x1f\xce\x51\x07\xc5\x58\x61\x7d\x48\x05\x99\x09\x22\x09\x53\xc6\x14\xc1\x14\x55\x73\x34\xe5\x31\x49\x8c\x1d\xc0\x18\x07\x2f\xb1\xc2\x17\x58\xe1\x84\x8f\xcf\x50\x07\xfe\xd4\x2f\x53\x89\x70\x22\x39\xc2\x6e\xe3\x90\xd8\x35\x81\x59\xec\xd8\x03\x46\x11\x9f\xce\x68\x62\xf2\xda\x7c\xff\x36\x65\x31\x7d\xa6\x71\x8a\x13\xc4\x87\x60\x43\x39\xeb\xb3\xee\x33\x61\x2a\x05\x1d\x17\x27\x09\xb2\xdd\xba\x07\x3c\x03\x86\x1b\xa5\xa4\x53\x9a\x60\xa1\xa5\x47\x33\xda\x1b\xdb\x16\x7a\x98\x90\x6c\xac\x30\x2e\x4d\xcd\x29\x7e\x22\x12\x51\x85\x66\x5c\x4a\x3a\x4c\xf2\x63\xfc\xd8\x43\x30\xee\x8b\xab\x1e\x38\x0d\x23\x85\xb8\xe1\x83\xae\x73\xeb\x41\x77\x3d\x4e\x31\x63\x04\x3a\xe6\x6a\x42\x84\xed\xde\x3e\xfc\xda\xfe\xbf\xc7\x6b\x9b\x2e\xd6\xbd\x5c\x74\x00\x3e\x74\xee\x7f\x5c\xfc\xd6\xe5\x87\x2d\xfe\x72\xd5\x79\xbc\xbe\xf8\x61\x70\x7b\xd5\xa9\xc8\x3b\xb3\xdb\xaa\xb1\x2f\xd1\x1f\xd9\xe6\x87\x69\xff\x8a\x46\x4b\x43\x13\x9b\x1b\x1d\x1a\x59\x1c\x1a\x9b\x1b\x9a\xda\x1a\x9a\x19\x1a\xea\xad\x0c\x7b\x08\x53\x6b\x6e\x0a\xb8\xa2\xb2\x68\x0b\x38\x8e\x98\xb5\xc2\x90\xf5\x1c\xf6\x6d\x08\xf8\xea\xac\x00\x5f\xa9\x09\x20\xe8\xff\x7b\xa1\xdb\x5b\x55\xfe\x5b\xae\xf9\x6f\x13\x94\x9a\xe1\x5f\x84\xa8\xd4\xc5\xa8\x54\x12\x82\x52\x43\x50\x6a\x53\x02\x85\xa0\xd4\x10\x94\x7a\xb4\x41\xa9\x65\x45\x2b\x78\x6c\xdb\xe0\xb1\x6d\xb9\x8e\xd6\x66\x87\xed\x5b\xd5\x5c\x82\xf3\x32\x38\x2f\x83\xf3\xf2\x48\x4f\x6e\x70\x5e\x36\xa7\x51\x70\x5e\x06\xe7\x65\x70\x5e\x06\xe7\x65\x70\x5e\x06\xe7\xe5\x6b\x9a\x46\xda\x10\x1b\x7a\xcc\x2e\xdb\xe0\x89\x5d\xe1\x89\x6d\xb9\x92\xdf\x4a\x47\xec\x5b\xd5\x11\x82\x6a\x1f\xfc\x92\x5b\x4d\xbb\x55\x4a\xfd\x5b\xbb\x37\x83\x2b\xb6\x39\x21\x82\x2b\xb6\x31\xa9\x82\x2b\x76\x09\x71\x82\x2b\x36\xb8\x62\xbf\x42\x57\x2c\x8d\xb7\x2e\xb9\xd5\x44\x6f\xd1\xb2\x62\xdc\x05\xf3\x50\x66\xdc\x12\xbf\x81\xf4\x88\xe5\x53\x66\x01\x6a\xa0\xcf\xf4\xe2\xa3\x50\x64\x2a\x27\x7c\x08\x85\x66\x1b\x8d\x05\x2b\xcd\xc1\x15\x40\x15\xe8\x5f\x72\xa3\x62\x0b\x6b\x04\x6c\xa3\xa3\x78\x13\xf3\xd4\x14\xa7\x7d\x78\x44\x69\xdf\xb4\x83\xe0\x17\x04\xbf\x20\xdb\x34\x9c\x70\x90\x6d\xda\x2b\xdb\xbc\x96\xc2\xd2\xbe\xe3\x79\x74\xf6\x89\xbd\x8b\xa5\xb2\x31\x68\x9b\x29\x93\x0d\xae\xbb\x74\x96\x70\x1c\xaf\x0a\x90\xfb\x0d\xe5\xb2\xda\x12\x71\xd3\xb4\xab\x5f\x68\xb9\xb4\xb9\x10\x1b\x67\x46\xfe\x35\xa0\xc6\xd7\x4e\xfd\x55\xf1\xcc\x60\xff\x66\xa8\x45\x6b\x21\x10\xee\x7f\x33\xb7\x3d\x1b\xef\x95\x77\xf3\x9b\x4c\xbd\x0b\x47\x74\xf5\x11\x85\x0f\x85\x00\xef\x7d\x59\x42\xca\xc7\xb6\x91\xd1\x43\xfe\xad\xe5\xe7\x36\x5b\xdf\x43\x98\x38\xde\xe4\x29\x7d\xc3\xce\xe6\xe0\x50\x5e\x1e\xf5\xb3\xa3\x00\xd4\x3e\x7b\x98\x68\x3a\x4c\x87\x94\x65\xf1\x76\x6e\x87\xfc\xc9\x91\xeb\x4f\x80\x75\x69\xf1\x2f\x93\x79\x6e\x0a\x93\xe5\xd6\x32\x45\x09\x9d\x6a\x2d\x35\x22\x42\x01\xbd\x39\x53\xe4\x8b\x92\xe8\x14\x25\xf4\x89\xa0\xf7\xfa\xc8\xa3\xce\x6d\xef\xfd\x09\x7a\x7f\x85\x53\x16\x4d\xd0\x2c\xc1\x4c\xbe\x6f\x8d\x82\x15\x6c\x66\xa1\x9a\x4a\xf0\x96\xee\x92\x38\xc1\xa2\x18\x2c\x8a\xad\xb3\x28\xb6\x45\x67\x30\x49\xa5\x78\x4a\xda\xa2\x3d\xb4\x5d\xeb\x0f\xda\x43\xd0\x1e\x82\xf6\x10\xb4\x87\x82\xf6\xd0\x0e\x0a\x07\xd5\x21\xa8\x0e\x41\x75\x08\xaa\x43\x50\x1d\x76\x4e\xc6\xa0\x3a\x2c\x53\x1d\xe0\x93\xc3\x8d\x59\x57\x8f\x68\xac\x3f\x34\x00\x89\x39\x1a\xe5\x21\x28\x0e\x41\x71\x08\x8a\xc3\xc1\x15\x87\xd6\x4c\xe8\xed\xe1\x5d\x04\xc4\x88\x80\x18\x11\x10\x23\x6a\x10\x23\x0e\x25\xb2\x19\x79\xed\xc8\x52\x64\x8e\x42\x68\x7b\xb5\x1c\x99\xb7\x27\xc6\x85\xac\x9f\x90\xf5\x13\xcc\x90\x21\xeb\x27\x18\xda\x82\xa1\xad\xd5\x86\xb6\xd7\xb2\x9e\x1f\xf8\x78\x1e\x40\x38\x6d\x79\xc4\xf2\x5f\x8f\x41\x02\x3d\x60\xcc\x41\xb0\xb2\x05\x2b\x5b\x35\x65\x8e\xd3\x3d\xdf\x9a\x5b\x3f\x00\x3c\x05\x89\x3f\x04\x1e\x84\xc0\x83\x95\xc4\x09\xfa\x50\xd0\x87\x5a\xa7\x0f\xbd\xa2\xa2\xd0\xba\x30\xe5\xa0\x31\x04\x8d\x21\x68\x0c\x6f\x56\x63\x68\x0d\x85\x83\xba\x10\xd4\x85\xa0\x2e\x04\x75\x61\x39\x71\x82\xba\x10\xd4\x85\xa0\x2e\xb4\x3a\x34\xf9\x58\x14\x86\xa0\x2c\x04\x65\xa1\xdd\xca\x42\x6b\x26\x14\x82\x78\x43\x10\x6f\x08\xe2\xfd\x6a\x82\x78\xdf\xa8\xc2\xbe\x57\x31\xcd\xb1\xc8\x65\x82\xd7\xa2\xbc\xf4\xf3\x02\x63\x6d\xad\xc8\x94\x8f\x76\x53\xdc\xc7\x5d\x91\xfa\x85\x8b\xa7\x51\xc2\x5f\x06\x99\x56\x67\x83\xc2\xf3\xbf\x6d\x3e\x9f\xf7\x45\x2e\x3c\x7b\x5f\x66\x42\xb4\xf7\x9d\x6b\xbd\x08\x10\x9a\xae\xc2\x07\x95\x88\x0b\x94\xce\x62\xf8\x18\xa5\x52\xf1\x69\xbd\x54\xfd\x19\xab\x68\x82\x87\x09\xe9\x64\xfd\x5e\x70\x36\xa2\xe3\xd4\xec\x8f\xdf\x80\x15\x62\x27\xd9\x9c\x38\xc9\x48\x33\x45\x37\xbe\x65\x92\xf8\x23\x8c\xe3\x17\xfb\x64\xde\xc9\x51\x04\xa0\x2f\x0e\xdb\x4c\xe7\x50\x70\xa3\xc5\x5d\xb4\x2d\x8b\xf3\x5a\x6b\x87\xf8\xb3\x78\x26\x56\x89\xaa\x60\x85\xce\x34\x13\x1a\xc3\xe6\x7c\x99\x50\xb0\xac\x81\x25\x0e\xac\x4f\x79\xc3\xe8\x85\x26\x09\x48\x1c\x86\x16\xed\x9b\x79\x23\xed\xc5\x4e\xdc\x9e\xbd\x37\x31\x6f\xc7\x3c\x56\xcc\xdc\x1d\x41\xe3\x86\x38\xd2\x69\xbf\x26\xc2\xee\x0a\x46\xf6\xaa\x38\xbb\xb5\xd7\x67\x4d\x4e\xd5\x87\x7f\x55\x5e\x89\x4d\x6a\xa7\xbe\xf6\x3d\xf8\x89\xa8\x37\x73\x09\x7e\x22\xea\x50\x37\xe0\x5b\xbc\xf6\x36\xbd\xeb\x96\x32\x3e\x41\x46\x44\x10\x16\x91\x63\xcd\xc9\x5a\xb8\xe2\x8e\x76\xba\x1b\xdd\x6c\x47\x3b\xdb\x75\x0c\x58\xbf\x98\x49\x5a\x73\xd5\xd4\xb1\x5c\x7f\xaa\x8a\x5b\xf7\x72\xc1\x05\x66\x8d\x55\x0f\x9d\xfb\x1f\x07\x77\xdd\xfb\x9b\xc7\xbb\x8b\xee\x39\xea\xc0\x41\x87\x77\x0c\x7b\xa7\xff\x84\xe6\x20\x1f\x36\x33\x86\x09\x73\xc7\x49\x60\xd5\xe0\x06\xd7\x54\x44\xa7\xe8\xe2\xea\xf1\xfe\xa1\x7b\x57\xd3\xa0\x65\xfe\x94\x8d\x91\x22\xd3\x59\x82\x15\x89\xd1\x53\x3a\x24\x82\x11\x50\xac\x92\x54\x2a\x22\x72\xe7\xb8\x69\xb4\xfb\xdf\xdd\x8b\xc7\x87\xde\xcd\xf5\xe0\xa7\xc7\xee\x63\xf7\x1c\xb9\x5b\x44\x37\xab\xc7\xa5\x47\x11\xcf\x19\x9e\xd2\xc8\x7c\x91\x95\xa2\x45\xbf\xa7\x24\x25\x08\x4b\x49\xc7\x6c\x4a\x98\x2a\xb7\xe8\x06\x7c\xd5\xf9\xbe\x7b\x55\x6c\x79\x42\xd0\x8f\x7f\xcf\x07\x95\xe0\x21\x49\xac\xb7\x1e\x1c\xd0\xfa\xf2\xca\x3b\xb2\x6e\xfc\xd4\x50\xf5\xa7\xc7\xce\x55\xef\xe1\xd7\xc1\xcd\xc7\xc1\x7d\xf7\xee\xe7\xde\x45\x77\x60\x8d\x31\x17\x1d\xdd\x6f\xa1\x27\x6b\xb3\x41\xbf\xa7\x38\xa1\x6a\xae\xd7\x51\x9a\x4b\x1f\xbd\x4c\x08\x43\x29\x83\x0b\xc4\x58\x0a\x31\xf3\x3a\x95\x33\x12\x99\x19\xdd\x5e\x3d\x7e\xea\x5d\x0f\x6e\x7e\xee\xde\xdd\xf5\x2e\xbb\xe7\xe8\x9e\x24\x60\x4b\x73\x44\x87\x55\x9c\x25\xe9\x58\x73\x82\xe9\x2c\x21\x9a\x1a\xc6\x56\x38\x24\x13\xfc\x4c\xb9\xb0\xd7\xf1\x98\x3e\x13\x66\xe8\xa8\xb7\x95\x69\xdf\xd9\xac\x06\x1e\xe9\x6e\xae\x3f\xf6\x3e\x9d\xa3\x4e\x1c\x67\x73\x90\xd0\x46\x61\xe7\xb8\xa3\x7b\x5a\x1c\x36\x1d\xd1\x08\xba\x37\x9b\x88\x3f\x13\x21\x68\x4c\x4a\xfb\xa8\x73\x7f\xdf\xfb\x74\xfd\xb9\x7b\xfd\x00\x14\x53\x82\x27\x12\x4d\xf8\x0b\x38\x7a\x61\x86\xe0\xff\x7d\xc6\x34\x81\xce\xdc\x62\x71\xe6\x9f\x7e\xaf\x67\x63\xd6\x14\x29\x7b\x75\xdf\x65\xe1\xe0\x2d\x5a\xfb\xca\x27\x69\xf1\x89\xd2\xb1\x58\xf6\x40\x61\x97\x2f\x3e\xb8\x6a\xb7\x2e\xbe\x51\xda\x6e\xf5\x36\xce\x85\xfd\x52\x3f\xd3\x7c\xad\x1b\x9b\x38\x8b\x34\xdc\x87\x8c\xed\xde\x7e\x17\x93\x84\x28\x52\x2b\x13\x5f\xc2\xcf\xaf\x2f\x13\x9b\x71\xbc\x19\xb1\xd8\x4c\x27\x48\xc6\x41\x32\x6e\x3c\xe1\x20\x19\x57\x4d\xf8\x8d\x48\xc6\x2d\xb4\xfa\x38\x16\xd5\x3a\xab\x4f\xf0\x8f\x94\x56\xea\x38\xaf\xc0\x57\x73\x8f\x04\xff\xc1\x7a\x57\xc8\xf1\xcf\x3b\xf8\x0f\x82\xff\xa0\xf2\x26\x79\xf3\x5e\x83\xe3\xbc\x1a\x0e\xe8\x34\x08\x6a\xc4\x92\xf9\x06\x35\xe2\xc8\x66\x1b\x0c\xec\xc1\xc0\x1e\x0c\xec\xc1\xc0\x1e\x0c\xec\x68\x53\x03\x7b\x03\x2e\x7b\x08\x73\x6a\x4b\x83\x88\xdf\x8a\xdb\xe0\x38\xe5\xe2\xc3\x7a\x0d\x82\x68\xbc\x64\xbe\x41\x34\x3e\xb2\xd9\xb6\xd0\x2e\xd2\x2e\x0b\x3b\x8d\xab\x0c\x22\x07\x84\xa6\x77\x23\x69\x0a\x4f\xef\x08\xda\x8b\x8f\x82\x9d\xbf\x1a\x42\x7d\xc0\x73\x0f\x78\xee\x01\xae\x25\xe0\xb9\xa3\x00\x48\x12\x00\x49\xda\x0c\x48\xd2\x60\x19\xdf\x02\x9e\xfb\x61\x2c\x0c\x6f\x28\x49\xd9\x09\x86\xb2\x10\xbb\xc1\xe5\xaa\xe0\x0d\xb0\x12\xa4\xb3\x84\xe3\x78\x19\x58\x8c\x93\x23\x7d\xc0\x98\x25\xa2\xa7\x69\xfb\x97\x45\xe5\xa9\xb5\x92\xa7\x1b\xab\x19\xf9\xa1\xcc\x07\xad\x51\xb8\xdc\xb4\x5b\xa1\x66\x15\x6b\xb7\xb6\x70\x43\x1f\x55\x40\xed\x61\x77\xf4\x9b\x2c\xda\x1a\x8e\xe9\xea\x63\x7a\xb8\xfa\x28\x55\x47\xb7\xb1\x21\x44\xfe\xed\x98\xce\xee\x81\x90\x8f\xdf\xde\x89\x0d\x08\x6d\x01\xa1\xad\x96\x32\xc7\x09\xe7\xdc\x1a\xc5\x2b\xd8\xd2\x02\xf4\x71\x80\x3e\xde\x25\x71\x82\xa5\x31\x58\x1a\x5b\x67\x69\x6c\x93\x0e\xb1\xc7\xd2\x29\xdb\x69\x13\x47\x65\x09\x08\xda\x44\xd0\x26\x2a\xa6\x16\xb4\x89\xaf\x50\x9b\x68\x07\x85\x83\x2a\x11\x54\x89\xa0\x4a\x04\x55\x22\xa8\x12\x3b\x27\x63\x50\x25\x5e\xa7\xac\x4a\x95\x3e\xd1\x30\x25\xf5\xa8\x94\x89\xa0\x48\x04\x45\x22\x28\x12\xa1\x70\xcc\xf2\x39\x85\xc2\x31\xa1\x70\x4c\x28\x1c\xf3\x06\x0a\xc7\x1c\x52\x84\xab\x41\x2b\x3f\x8e\x34\x9b\xa3\x10\xe2\x5e\x2d\xcf\xe6\xed\x89\x74\x21\x73\x28\x64\x0e\x05\x13\x65\xc8\x1c\x0a\x46\xb8\x60\x84\x6b\xb5\x11\xee\xb5\x2c\xeb\x07\x3e\x9e\x07\x12\x54\x8f\x24\xda\xf9\xaf\xc7\x20\x8d\x1e\x38\x3e\x21\x58\xe0\x82\x05\xae\x9a\x32\xc7\xe9\xca\x6f\x8d\x14\x70\x8c\x95\x63\x83\x06\xd0\x9c\x10\x21\x48\xa1\x39\xad\x42\x90\xc2\x12\xe2\x04\xfd\x28\xe8\x47\xad\xd3\x8f\x5e\x59\x71\x68\x6d\x88\x73\xd0\x20\xcc\x73\x41\x83\x08\x1a\xc4\x1b\xd5\x20\x5a\x43\xe1\xa0\x3e\x04\xf5\x21\xa8\x0f\x41\x7d\x58\x4e\x9c\xa0\x3e\x04\xf5\x21\xa8\x0f\x47\x13\xd6\x7c\x4c\x0a\x44\x50\x1e\x82\xf2\xd0\x6e\xe5\xa1\x35\x13\x0a\x01\xc0\x21\x00\x38\x04\x00\x7f\x35\x01\xc0\x6f\x54\x81\xdf\xad\xd8\xf6\x07\x4b\xa8\x77\x9e\x80\x91\x49\x22\xef\xbe\x4f\xf8\xf0\x61\x3e\x23\xfa\xbf\x97\x74\x4a\x98\x04\x4a\x50\x35\xf7\xc5\xb4\x9a\x0d\xb5\xb8\x95\xde\xdd\xf7\xae\x3f\x5d\xf9\xe5\x81\xde\x7d\x7e\xbc\x7a\xe8\xdd\x76\xee\xb2\xe5\xce\x66\xe5\x2f\xb1\x7d\xaf\x20\x69\xda\x93\x7c\x47\xb4\x4a\x0d\xcc\xe0\x5e\x61\x95\xca\xcd\x46\x76\xd7\xbd\xef\xde\xfd\x0c\xe5\x8d\x06\x97\xbd\xfb\xce\xf7\x57\x85\x7d\x5e\xf8\xbd\x73\xf1\xd3\x63\xef\xae\xfe\xf7\xee\x7f\xf7\xee\x1f\xee\xeb\x7e\xbd\xeb\x5e\x75\x3b\xf7\xf5\x6f\x7f\xec\xf4\xae\x1e\xef\xba\x4b\xe9\xb1\x74\xb4\xcb\x75\x2b\x09\x44\x82\x12\x1f\x28\xb2\xcc\x50\xe4\x34\x44\x99\x54\xec\xb8\x7c\x55\x5f\xe7\xe8\xd1\x9a\x2a\xa8\x6d\xdc\xdc\x1b\x5e\x43\x46\xc7\x8a\xa9\xc4\xc3\x84\xc4\x0b\x2d\x39\x1a\xd6\xb5\x84\x0b\x83\x7a\xc1\xd2\x93\xa4\x35\x2b\x8f\xcc\xf1\x41\x50\x74\x4d\x11\x16\x57\xf4\x61\xd6\xa1\xb6\x07\xa6\x59\x32\x7d\x26\x85\x9e\xa2\x54\x08\xc2\x54\x32\x47\xe4\x0b\x95\x4a\x2e\x34\xea\x96\xaf\xae\x59\xcb\x10\xb2\x06\x27\x58\xa2\x21\x21\xac\x38\x7e\x41\x12\x82\x65\xc5\x98\xed\xea\x37\x23\x4b\xb6\x56\xd6\xc8\x64\xee\xd8\x11\xa6\x49\x2a\x48\xe9\xb4\xf0\xe9\x0c\x0b\x2a\x39\xeb\x7e\xd1\x57\xb4\x3e\xc8\x37\xf0\x3a\x17\x9b\x9d\x98\xee\x4f\xfe\x0e\xbe\x2e\xfe\xf9\xe9\xa1\xf8\x57\xe1\xcc\x5f\x3d\x14\xff\x5a\xbe\xd7\xbd\x86\xcb\x3b\xfb\x14\x7d\x7a\x38\x47\x9f\x00\x62\x54\xa0\x87\x09\x36\x3b\xf6\xea\xe1\x1c\x5d\x11\x29\xe1\x9b\xfc\x65\x45\x55\x02\x73\xfb\x9e\x32\x2c\xe6\xc8\x4d\xdf\x54\xee\xc3\xd1\x04\x91\x8c\x34\x65\xe2\xb1\x7f\xa4\x0c\x2c\x12\x39\xf5\xae\xf8\x98\x46\x38\xd9\x8e\x88\x9d\xeb\x02\x1f\xb8\xb9\x5b\x4a\x0a\xff\xe9\x45\x5a\x74\xae\x2f\xa1\x2a\x9e\x1b\x6a\xc5\xcc\xaf\x89\xd4\x9b\x24\xe2\x2c\xb6\x3e\x35\x2d\xd4\xcc\x3d\x5d\xe5\x1f\x1c\x2a\x0b\xa6\x92\xb2\xb1\x6e\x11\x7d\x40\x37\x77\x7d\x76\x23\x62\x63\xdf\x25\x5a\xc8\x37\x7b\x8e\x4a\xc4\xb8\x42\x74\x3a\xe3\x42\x61\xa6\xb4\x7e\x03\xd2\x8d\xa5\x88\xe1\x00\x17\x7c\x3a\x4d\x15\xd6\x07\x6d\x81\xa8\xcc\x58\x79\xee\x89\xea\xc5\xe0\x08\xab\xa0\xa1\x11\x7f\xf2\xb9\xcc\x84\x6e\x5f\x8b\x5e\x45\xd3\x00\x8d\x17\x34\x74\xd7\x04\x16\x02\x17\x2f\xe0\x77\x54\x91\x69\xf9\xf9\x86\xd7\xee\xbf\x2b\xed\x1e\x17\x26\x2b\x82\x88\x8e\x88\x26\x54\x91\x48\xe9\x23\xb8\xd1\x9e\x78\xbc\xfe\xf1\xfa\xe6\x17\x5f\x30\x7a\xd7\xf9\x7c\xf9\x9f\x05\x18\xd8\xce\xdd\xe7\x85\x2f\x06\x3f\xff\xe7\xc2\x37\xff\xff\xa5\xfb\xa9\xdc\xd3\x82\xf9\xc2\x9b\xcb\x29\x68\x0a\x60\xea\x76\x53\x45\x74\x8a\xc7\x04\xc9\x74\xa6\x77\x80\x3c\x2b\xae\xaf\x96\x94\xaf\x38\x8e\x29\x1b\x9b\xe2\x6f\x57\x54\x11\x81\x93\xcf\x78\xf6\xd1\x99\xe5\x37\xa0\xce\xff\xb9\x2f\x14\x20\x7c\xf7\x6b\xe7\xb3\x5f\xc2\xf0\xdd\xed\xdd\xcd\xc3\xcd\xd2\x59\x17\x5a\x58\x3c\x46\xfa\xe7\x73\xf8\x7f\xf4\x01\xe9\xd6\x33\x81\x7e\x4a\x14\xd6\x8a\x0e\xfa\xc6\xd4\xcb\xca\x32\x61\x28\x4b\xe0\xd4\xcc\x04\x9d\x52\xb8\x52\x8c\x61\xf2\x5b\xa3\x33\x64\x4a\x51\x76\x6e\xcc\x0b\x60\x04\x70\x97\x32\x8b\xb1\x88\xd1\x3f\x64\xb9\x1e\x26\xd8\xc3\xcd\x17\x24\x46\xa7\x68\xa2\xd4\x4c\x9e\x7f\xf8\xf0\xf2\xf2\x72\xa6\x9f\xd6\x02\xec\x07\xfd\xe1\x94\xb0\xb3\x89\x9a\x26\xa6\xfe\xa7\xa6\xc2\x39\xba\x15\x5c\x5f\x21\x60\x77\x20\x82\xe2\x84\xfe\x93\xc4\x68\x68\xf8\x1f\x1f\xa1\xdf\x22\x2e\xc8\x59\xbe\x30\xd6\x56\x66\xef\x11\x6b\x4f\xfb\xa0\x1f\xaa\x60\x26\xe5\xf5\x44\x31\x89\x68\x6c\xc5\x0c\xc2\x22\x0e\x06\x55\xe3\x82\xd1\xed\xb9\x22\x63\x5a\x51\x9b\xa5\x2a\x27\xa7\xa7\x83\xe1\x98\x78\xe5\x3b\xad\x7c\x9d\x6d\x38\xad\xcf\xf5\x8c\x36\x9e\x4a\x22\xe0\x6e\xc5\x70\xab\xba\x47\x67\x7a\xc2\x11\x4f\xd0\x30\x1d\x8d\x88\xf0\xc3\x07\x4e\xb4\x92\x46\x25\x12\x24\xe2\xd3\x29\x48\x0c\xfa\xad\x54\x9a\x5d\x0d\x14\xb3\xa3\x3d\xeb\x33\x58\x7f\xad\xbd\xc1\x0e\x88\x39\xb0\x3a\x46\x48\x8c\x30\x9b\x9b\x6e\x86\xe9\xc8\x6f\xdf\xd4\xd5\xc5\x31\xa2\xaa\xcf\x3a\x49\x82\x04\x99\x72\x45\xbc\xf2\x69\xe0\xea\x2c\x12\x1c\x58\xa4\x20\xb3\x04\x47\x24\x36\xfb\x21\xe1\x11\x4e\xd0\x88\x26\x44\xce\xa5\x22\x53\xbf\x81\x6f\xc0\x04\xa5\x69\x46\x25\x8a\xf9\x0b\x4b\x38\xb6\xf3\x28\xbf\xf6\x6d\xf1\x34\x76\x5d\xcd\xd3\xae\x10\x5c\xc0\xff\xfd\x48\x59\xbc\x33\x0e\xf5\x78\xdf\xbd\xf3\xff\xbe\xff\xf5\xfe\xa1\xfb\x79\x3d\xee\x93\xed\x2c\x18\x1e\x98\x26\xce\xd1\xbd\x21\x02\x17\x5a\x22\x12\x35\x93\xfa\x6c\xb7\x52\xfe\x05\x8f\x37\xe4\xbe\x9f\x3b\xd7\x8f\x9d\x02\x47\xb9\xbf\xf8\xa1\x7b\xf9\x58\xd2\x07\xec\xfc\x0a\x32\xbc\xd1\x6a\xfd\xef\x2e\x7e\xe8\x5d\x5d\x0e\x2a\xf4\xe0\x77\x77\xdd\x8b\x9b\x9f\xbb\x77\xb9\xca\x5a\x49\xa2\xd2\x60\xca\xcc\xea\xc1\x30\xa5\x09\x8f\xd1\x70\x5e\x5d\xe1\x56\x4b\xce\x09\x78\xce\xf3\x1a\xcf\xa6\xd5\x73\xe0\x4d\xae\xd8\x70\xfe\xc6\x94\xc7\xe4\xc4\x3e\x03\xa5\x81\x8d\xcd\xc8\x48\xcc\xd5\x0d\xeb\xde\x31\xf3\xec\x2f\xa6\x6a\x6f\x46\xb8\x73\xd4\x41\x52\x3f\x98\xea\x43\x2d\xe8\x78\x0c\xf6\xd0\xd2\x50\x4d\x6b\xf6\x55\x20\x2f\xbc\x67\xd6\x7f\x26\x38\x9c\x73\xdd\xad\x35\xa4\x67\xc6\x16\xf3\x22\x94\x91\x2e\xb6\x28\x30\xd8\x51\x2a\x86\xe6\x16\x4b\x13\xa1\x96\x5e\xe6\x3c\x1a\x33\x98\x3e\x5c\xc0\xb6\xa4\x31\xe3\xce\x04\x79\xa6\x3c\xf5\x5e\xb5\x95\x8a\x0b\x2b\x5e\xd9\x7c\x4e\x00\x20\x9b\xb1\xf5\x94\x9a\xc9\xb6\x47\x65\x0b\x9a\x85\x3d\x43\x0b\x23\xc1\xa7\x15\x6d\x14\x8f\x49\xef\xe6\x5e\x09\xac\xc8\x78\x7e\x69\x59\xc6\xe6\xc7\xe3\xf2\xe6\x97\xeb\xab\x9b\xce\xe5\xa0\xdb\xf9\x54\x3c\xf1\xd9\x2f\xf7\x0f\x77\xdd\xce\xe7\xe2\x4f\x83\xeb\x9b\x87\x81\x7b\x62\xe9\x96\xaf\xe9\x60\xf1\x9e\x2e\x3e\x78\x8e\x34\xcb\x05\xd6\xf8\x42\x93\x44\x5f\x26\x1e\x7f\x1c\x92\x11\x17\x86\xcf\x4f\x5d\xa0\x89\x15\x61\x1c\x6d\xad\x2e\x56\x9a\xc5\x39\x18\xfc\xaa\x9a\x34\xc6\x7c\x25\x08\x9e\xc2\x3d\x81\x19\xea\xb2\xf8\xf4\x66\x74\x7a\x6f\xbe\x9c\x62\xf1\x44\x44\xf6\xea\x8b\xa0\x4a\x11\x56\x50\xe9\xb0\x1b\x72\xa6\x24\xe6\x1d\x9c\xa1\x3b\xcd\xf7\xf5\xf3\xd9\xa5\xa6\x37\x7b\x4c\x14\xa6\x89\xb4\x83\x2d\xd0\xf5\x1c\x5d\x61\x31\xce\xcd\x8b\xdf\xf0\xd1\xc8\x34\xf6\xad\x19\x86\xbe\xc3\x0a\xb3\xa8\xe0\xbd\x7a\x6b\xb8\x7b\x11\xfa\xb3\x0f\x67\xf2\xf0\xe2\xae\x7a\x9c\x6d\xb7\xa7\x1e\x6f\x81\xe2\x46\x63\x2f\xe8\x86\xf6\x97\x8a\xbd\x06\x13\x37\x3f\x2f\xbf\x64\xaa\xdb\x5e\xdc\x4e\xc5\x07\x2b\xb6\x93\xa9\xd0\xa2\x57\x7e\xa4\xb5\xcd\x8a\xbd\x44\xbe\x50\x6b\x30\xf0\xc7\x5d\xda\x42\x79\x33\x60\x35\xc6\xb3\x19\xc1\x42\x56\xad\x76\x51\x0c\xac\x59\x7b\xd3\x93\xdf\x87\x5d\x64\xd7\xcf\x09\xe2\x0c\x0c\x0e\x99\x10\x51\xda\x91\x0d\xf6\x80\x69\x6b\x61\x07\xdc\x42\xf9\xf8\x1b\x5b\xaa\xfd\x33\x95\x5a\x69\x34\x5f\x7e\x6f\x6b\xc8\x6f\xb6\x21\x3e\x76\x7a\x57\x25\xe1\x62\x70\xd9\xfd\xd8\x79\xbc\x5a\x6e\x26\x2c\xbc\x57\x5e\x62\x74\x8a\xf4\xef\xc5\x70\x00\x3a\x32\x77\x86\xab\x84\x6f\x54\x5a\xc2\xc0\x68\x65\xab\x54\x1b\x33\x7c\x4c\x66\x09\x9f\x4f\x09\x03\x13\x4f\xe1\x26\xd4\xf4\x1c\x61\x6a\xaf\x16\x6f\xb0\x60\xc5\xb1\x66\x37\xb8\xc6\x4e\x5d\xf9\x7d\x12\x67\x37\x6f\xb1\xfa\x7e\x89\x75\xdf\x1a\xa7\xa0\xfd\xcf\xbd\xc2\x6a\xc3\x33\xd6\xb9\x78\xe8\xfd\xdc\x2d\xea\x87\x17\x3f\xf4\x7e\xae\x92\x6a\x06\x9f\xba\xd7\xdd\xbb\xce\xc3\x0a\xe1\xa4\xd4\x64\x95\x70\x22\xf5\x80\xcb\x4e\x61\x2a\xb3\x40\xa7\xc8\xd4\xf0\x47\x54\x49\xf4\x4c\x25\x1d\xd2\x84\xaa\x39\xb2\x0e\xd6\xc7\x1e\x70\xd6\x67\x9c\xd0\x98\xaa\xb9\x13\x5f\x4c\xbf\xc5\x75\xd4\x9c\xd4\xb6\x6f\xcc\x0e\xbe\xdb\x15\xac\x7c\x66\x71\xdc\xa4\xcf\x11\xe8\xb6\xcf\xa0\xb4\x79\xaf\x31\x2d\x48\xb3\x31\x11\x66\x38\xe0\x54\xf2\xc7\xe2\xfd\xae\x47\xe5\x0b\x2b\x39\xd5\x32\xa1\x75\x4c\x18\xd1\x2c\xd2\xeb\xc4\x08\x52\x82\xb0\xf7\x5a\xe6\x9a\x25\x34\xa2\x2a\x99\xa3\x08\x6c\x58\x60\xce\x9c\x62\x86\xc7\x56\x38\x00\x35\xa7\xb4\x25\x7e\x4a\xc1\x00\x7f\x33\xb2\xa6\xfd\x07\x4a\x36\x3c\x66\x8f\xd7\x97\xdd\x8f\xbd\xeb\xe2\x16\xf8\xa1\xf7\xa9\x20\xc2\x7e\xee\x5e\xf6\x1e\x0b\xb7\xb9\x96\x64\x97\xcb\xf5\xe5\x66\x2b\x8e\x62\xf6\xd0\x39\xba\x34\xaf\x9e\x6b\xe2\xfe\x6e\x26\xa7\xb7\x8c\x34\xd3\xcb\x95\xdf\x12\x1d\xee\x5c\xa4\xa1\xfb\xd0\x65\x4a\x54\xfa\x25\x9a\x9a\x90\xac\x57\xa8\x60\x43\xaa\x8e\xc0\x58\xe8\xfb\xba\xec\x2b\x2f\x4f\xd9\x3d\x08\x21\xb2\x67\xb9\x65\xc9\x0f\xcd\x00\xa3\x41\x9d\x11\xab\xc2\x5b\x97\x33\xec\x9f\xc1\xf3\x3e\x4d\xa5\x32\x1e\x52\xd8\x9c\xe8\xe9\xef\x52\x13\x14\x3c\xa8\x67\xe8\x9e\x90\x3e\x73\xd6\x83\x31\x55\x93\x74\x78\x16\xf1\xe9\x87\xa7\x74\x48\x04\x23\x8a\xc8\x0f\x78\x46\xa7\x58\x4b\xd2\x44\xcc\x3f\x0c\x13\x3e\xfc\x30\xc5\x52\x11\xf1\x61\xf6\x34\x86\xc0\x1e\xe7\xe9\xfa\x90\x35\x3b\xe6\xff\x71\xf5\xd7\xef\x4e\xaf\xfe\xfe\xdd\xbb\x45\x0b\x59\xdd\xfa\x77\x59\x84\x67\x32\x4d\x6c\x20\xa0\xf0\x69\xe3\x8e\x7c\x4a\x56\xad\xf7\x75\x71\xb9\xb6\xd3\x5f\x2f\x6e\x1f\x0b\x16\xeb\xe2\x9f\x9f\xbb\x9f\x6f\xee\x7e\x2d\x70\xca\x87\x9b\xbb\xce\xa7\x02\x43\xed\xde\xfe\xd0\xfd\xdc\xbd\xeb\x5c\x0d\xdc\x8f\xdb\xd8\xde\x7e\x64\xfc\x85\x15\x49\x23\x1d\x07\x5c\xe8\xe9\x1c\x7d\xe4\x02\xfd\x98\xad\xe4\xe9\x10\x4b\xb8\x62\xdc\x9d\x25\x4f\xd0\x8c\xc7\xc0\x78\x11\x99\x4d\xc8\x94\x08\x9c\x58\x9b\x81\x54\x5c\xe0\xb1\xb9\xe9\x65\x24\xb0\x8a\x26\x48\xce\x70\x44\x4e\x50\x04\xbb\x61\x7c\x02\x8b\x02\xaa\x16\x1f\x97\xed\x7c\x77\x29\x53\x74\x4a\x9c\x0a\x6e\xff\x7c\x30\x8b\xb1\xc1\xe2\xdc\x3c\xfc\x50\x14\xf6\x3e\x5e\xfd\xfa\xd0\x1d\xdc\x5f\xfe\xb8\x94\x9e\xe6\xb5\xc2\xc8\xee\x21\xae\xea\x82\x27\xe9\x94\xf9\x9f\x37\x1f\x5b\xef\xfa\xa1\xfb\xa9\x3c\xba\x9b\xce\x43\x71\x67\xdc\x15\xe3\xf6\xde\x7d\x7f\x73\x73\xd5\x2d\x78\xba\xdf\x5d\x76\x1e\xba\x0f\xbd\xcf\x85\xfd\x73\xf9\x78\x07\x3e\xa0\xa5\xd3\x74\x23\xa8\x98\xa8\x9e\x96\x3f\xcd\x5d\xb3\xc2\x46\x9c\xa8\x63\xc3\xff\xcd\x59\x3e\xf5\xf0\x72\x4c\x94\x1b\x58\x75\x4e\x33\x93\x6a\x64\x46\x5a\xc9\x0e\x55\x71\x99\x50\x3d\x3b\x5e\xba\xd0\xcb\xb8\xf2\x43\x36\x04\x18\xd7\x99\x51\xb6\x71\x92\xf0\x17\x13\xa1\x3c\xa5\xfa\x56\x96\x04\x02\x95\xf5\x23\x32\xf7\x10\x9e\x55\x70\xbc\xe2\xb2\x90\x48\x10\xf5\x99\xa7\x4c\x6d\xbe\xe5\x3a\xd7\x05\xbe\xd3\xbd\xfe\x79\xf0\x73\xa7\xb8\x03\x7b\x57\xcb\x59\x8d\xdf\x44\xc5\x55\xdc\xb9\xfe\x35\xbb\x84\x21\x8e\xfd\x24\xd3\x50\x8d\xec\x1a\x25\x54\x8b\xbd\x11\xd6\xda\x6b\x02\x12\x0d\x22\x14\x4c\x0e\x53\x3d\x39\x88\x9b\x9d\x19\x7f\x92\xe1\x4f\x66\x90\xe7\xee\x43\xa9\x3d\x09\x74\x01\x6b\xaa\x4b\x13\x80\x76\xac\x56\xcd\x10\x61\xcf\x54\x70\x06\xc2\xf6\x33\x16\x54\x4b\xe3\xa6\x65\x3d\xd7\x73\xf8\xff\xf5\xda\x04\xc3\x68\x89\x71\xdd\x73\xa1\x2e\xb3\xf8\xe4\xcd\xac\x21\x55\x71\xba\x8b\x11\xba\xd5\x86\x8e\xc5\x77\x2b\x16\x67\xcb\x38\xe6\xe2\x84\x7f\x4f\x2e\x29\x4e\x34\x03\xd8\x9d\xbc\xd8\xb9\xbe\xef\x15\xe5\xc7\xa2\x9a\xe1\xf1\xe5\x8d\xe5\x45\x30\x54\x9a\x91\x3b\x65\xe2\xfe\xa7\x2b\xa3\x5d\xe8\x4d\x62\xcf\xad\xa7\x58\x80\x00\xe4\x6a\xab\xce\xb0\x90\xa5\x37\x24\x02\x24\xb3\x3c\x8e\x4c\xdf\x59\x10\xa5\xf5\xcc\x69\xdc\x67\xe4\xcb\x8c\x30\x09\xc1\x01\xe6\x3e\xcb\x7d\xed\xf2\x0c\xf5\x46\xc0\x12\xf4\xe3\x0c\xa5\xcc\x3a\xc0\xf4\x85\x6b\x06\x79\xa2\x45\x59\x3b\x84\x4c\x43\x04\xc3\x0b\x23\x2e\x06\x2c\x1f\x7c\x9f\xfd\x92\x39\xd1\xe0\xa7\x11\xd7\x0c\x48\xaf\xa2\x6d\xef\x1c\x61\x26\xe9\x09\xd2\x0a\x4b\x79\x4d\x21\x23\x42\x2b\x94\x36\x32\x4d\x73\x1a\xfb\xf1\xf0\xd7\xc0\x42\xf8\xb3\x7f\x19\x54\xdf\x05\xa5\xab\xa0\x46\x34\x4e\x8c\xc7\x64\xd0\xfc\x4e\x88\xb8\x20\xd6\xcf\xb2\xf6\x35\xb0\x8a\xb1\x3f\x60\xf9\xb4\xe0\x7b\xe8\x31\xa9\x30\x8b\xc8\x45\x82\xe5\x86\x41\x48\xce\xc6\x71\x52\x94\x38\xee\xee\x1e\x6f\x1f\x7a\xdf\xaf\xe0\xf2\xe5\x97\x17\xc3\x80\xa2\x24\x75\xee\xb9\xa1\xe0\x38\x46\x9a\x7d\x8e\xb9\x71\x05\x5a\xc1\xdf\x9c\x20\xb3\x26\x54\x7a\x71\xa2\x58\x3e\x15\x8c\xd4\x36\xcb\xc2\xda\x39\x7c\x57\x02\xb5\x84\x40\x91\xa6\x04\xf2\x4c\x1e\x6e\xa9\xc1\xb3\x68\xa2\xe8\xac\x75\x6b\x96\x60\x35\xe2\x62\x6a\xb8\x7c\x61\xd2\xa6\xf1\xe5\x8d\x52\xa6\x88\x10\xe9\x4c\x81\xca\xae\xc7\x5a\x96\x52\xf5\x92\x5d\xf1\xf1\x67\x22\x25\x1e\x93\x6d\x1c\xd0\x55\xca\xc3\xfd\xcf\xfe\x9f\xe0\x60\x6e\x22\xfb\x17\x46\xe8\x02\xfa\xdd\x7e\xba\x61\x1f\x4d\x20\xcf\x2d\x4f\x68\xb4\x61\xc0\xdd\xc7\x4e\xef\x6a\xd0\xfb\xac\x95\xf8\xce\x43\xf7\xaa\x20\x4a\xc0\x6f\x9d\x8f\x0f\xdd\xbb\x41\xf7\xbf\xbb\x17\x8f\x0f\x9d\xef\xaf\xba\x83\xeb\x9b\xcb\xee\xfd\xe0\xe2\xe6\xf3\xed\x55\x77\x45\x64\x4e\x6d\xe3\x8b\xd6\xd5\xf2\xa3\xe7\x0b\xdf\xc0\x0a\x6b\x5e\xe6\xdb\xcb\x20\x19\x0e\xd3\x04\x9c\xe0\xdc\x38\xc3\x31\x62\x3c\x26\xf0\xb5\x74\xd6\x19\x97\x6d\x72\x86\x7a\xea\x7d\x92\x20\x9c\x2a\x3e\xc5\xe0\xb5\x49\xe6\x7d\x86\x87\x9a\xb5\xe2\x24\xf1\xc2\xbb\x44\xca\x98\x66\xb1\xba\x31\x69\xe2\x8b\x13\xa2\xd9\xf9\xcc\xcb\x61\xb4\x7e\x83\x11\x65\x10\x40\x3c\xc5\xe2\xc9\xb8\x99\xf2\x2e\xf3\x43\x21\x11\x96\x7d\xa6\xc7\x45\xac\x61\xa8\x09\x85\xcf\x1b\x3d\x55\x4b\x9d\x29\x7e\x22\x9a\x2a\xd3\x34\x9a\xa0\x99\xe0\x63\x41\xa4\xb4\xb6\xe5\x08\x33\x13\x80\x60\x1f\xd7\xd7\x50\x9f\x31\xae\x49\xe1\x4c\xd8\x31\x99\x11\x16\x13\x16\x51\x93\xad\x08\xbe\xfb\xcc\xb4\x39\x16\x78\x36\x41\x92\x83\xd3\x1b\xc8\x0e\xf6\x2b\xf3\x92\xbb\xc9\xcc\x8c\xcd\xcf\xbe\x05\x5a\xa4\x9a\x4f\xdc\x80\x9c\x68\xa8\x0c\x2f\xbb\xcb\xd0\xb9\x5d\x8c\x1d\x70\x3a\x4b\x08\x74\x69\x49\x0e\x8b\xa1\x69\x5d\x58\x0f\xbd\x4c\x55\x8b\xa0\x2f\x6c\x37\x66\x2c\xed\x88\xce\x2a\x2c\xdb\xf6\x48\xa1\x1f\x30\x8b\x13\xdd\x8a\xf3\x61\x14\xcf\x22\x64\xd8\x74\xf4\xae\x71\xa7\x71\x9b\x5b\x34\xc2\xa9\xdc\xe6\x1a\x2d\xa5\x98\x1a\xab\xe0\x69\x1e\x14\x02\xdb\xdb\xe6\x97\x02\x75\x67\x9a\x45\xe2\x84\x5b\x2a\x99\xc7\x53\x1b\xb4\x0c\xa3\xa9\xb9\x66\x67\x82\xb2\x88\xce\x70\xb2\x91\xee\x57\xca\x31\xb0\xa1\xfb\xdf\xd0\x91\xde\x3e\xdf\x2e\xb8\x6d\x15\x11\x53\x48\x27\xb7\xc3\xcc\x96\x70\x0d\x4b\x92\x4d\xd6\x20\x32\x8f\x26\xc1\x82\xa7\xc6\x1f\x07\x74\x21\x71\xc5\x51\x3d\xab\x5a\x6e\x7d\x32\x70\x31\x00\x7a\x83\xc5\x36\x91\x3f\x75\xf4\x2b\xb5\x62\x7b\x37\xc1\x78\x38\xb9\xad\x6e\xb3\x6a\x05\xbc\x1f\xff\xbd\x6c\xef\x7c\xc6\x33\xbd\x67\xa2\x54\x2a\xf0\x14\x67\x73\xb4\x4a\x52\x29\x94\xdd\xf3\x9d\x67\x41\xed\xcd\x57\x23\x27\xa1\x0d\x80\x5a\xec\xa4\x10\x43\xe0\x21\x02\xd8\x3d\x3e\x4a\xb5\x2c\x8b\x30\x44\x21\xa0\x6f\xc8\xd9\xf8\x0c\xdd\xfc\xdc\xbd\xbb\xeb\x5d\x76\x4f\x50\xe7\xf6\xb6\x7b\x7d\x79\x82\x88\x8a\xbe\x75\x31\x8b\x36\x60\xa9\xcf\x14\xb7\xd2\xca\x1c\x4d\xf8\x0b\xf0\x46\x22\xc6\xa4\x30\x67\x17\xdd\x04\xa1\xca\x63\x2a\x95\x0d\x9f\xd5\x7c\x25\x1f\x96\x96\xf7\x2b\x77\x48\xaa\x26\xdb\x6c\x0d\x2c\x65\x3a\xd5\xba\xec\x80\xe2\xe9\x40\xf0\x64\x1b\xa6\x70\x09\x53\x01\x75\x39\x03\x53\xa0\x78\x8a\x74\xb3\x36\x14\x24\x73\x39\x66\x22\x9d\x16\x8c\x34\x5f\xd6\xf7\xa6\x77\x6f\x39\xef\x83\x8d\x47\xa3\x2e\x04\x02\xc0\x16\x6a\x58\x45\x6e\x36\x1e\x58\x4b\xfd\x00\x47\x91\x56\xb9\x77\x3c\xa9\xbc\xa3\xcc\x25\x60\x3b\xda\xdb\x34\x57\xed\x73\x37\xcc\x99\xe6\x60\x10\x0c\xac\xaf\x5c\xc9\x23\x9a\xb7\x5f\xd1\xef\x70\xbe\xd0\x2b\x6c\xd9\xb3\x3e\x7b\x94\x99\x49\xc5\x5c\xc2\x92\xc0\x4a\x4a\xf4\x32\x21\x70\x34\xe6\x68\x82\x9f\x49\xa1\x4b\x97\x43\xa2\x1b\x9e\xf3\x54\x54\x31\xba\x3e\xbb\x24\x33\x41\xb4\xa4\x5f\x76\xa0\x64\x7b\xfa\xae\xb8\x13\xc3\xbe\x0e\xfb\xfa\xe8\xf7\xf5\x45\x92\x4a\x45\x44\x47\x4a\x3a\x06\x43\xe2\x56\x02\x9c\x69\x6c\x30\xe3\x3c\x19\x34\xb0\x89\x34\xa7\x78\xc1\x13\x56\x08\xf8\x90\x06\xe9\x80\xa7\x20\x1f\x15\xae\x4d\xae\xef\x3a\x2f\x73\xd8\x0e\x6f\x09\x19\x9c\xcb\xac\xe3\x00\x25\xb6\x12\x71\x70\x55\x2b\xcb\x5a\x42\x7b\x17\x73\x2e\x8c\x7c\x93\xb9\xcb\xf2\x21\x96\x0e\x93\x13\x45\x28\x73\x64\xcb\x5f\x82\xfd\xac\x09\x6c\xe4\x8e\xdf\x53\xae\xb0\xfc\xf6\xac\xcf\xb4\x10\xf5\x44\xe6\xc6\xdc\xaa\xc5\x94\x3f\x6a\x59\xfc\x54\x12\x26\x21\xdc\xfb\x8f\xc6\x3d\xa7\xb7\xb8\x33\x57\x1b\xd5\x94\x4c\x67\x09\x56\x10\x74\x9d\xf5\x02\x21\xba\xb6\x51\x2b\x25\xe5\x01\xd0\x20\xe7\x9b\xb9\xd8\xdf\xcc\xf0\xc7\x44\x41\xe6\xb8\xa2\x0a\x74\xa6\x38\xd5\xe4\x59\x1c\xfa\x4a\xd3\x95\xd9\x15\x82\x83\x9f\x24\x4e\xb7\x63\xfc\x72\xb1\x8d\x95\x9c\x31\xd3\x16\xee\x6d\xcc\xfb\x07\x67\x37\x8a\x04\x67\xa5\x68\x18\xad\xcc\x99\x95\x1e\x1a\x76\xe0\xfc\xd7\x84\x9d\xbd\xd0\x27\x3a\x23\x31\xc5\x10\x01\xaf\xff\xfa\xa0\xe7\xf5\x1f\x17\x77\x37\xd7\x83\x3c\x93\xe7\xbf\xfa\xac\x93\x48\x9e\x65\x29\x20\xc6\x59\x16\x6e\x3f\x13\xc4\x89\x84\x76\x2e\x60\x75\xcd\xcd\x88\x7d\x56\x37\x82\x98\x47\xf2\x0c\xbf\xc8\x33\x3c\xc5\xff\xe4\x0c\x5c\xe9\x1d\xf8\x78\x91\xf0\x34\xfe\x05\xab\x68\xf2\x01\xce\xb5\xfa\x40\x9e\x09\x53\xc6\x4d\xa5\xc9\x15\x43\x4e\xb2\x84\x68\xfd\xff\xd0\x63\xce\x93\x8a\xa4\xd6\x64\x23\x32\x53\xe8\xff\x15\x64\xc8\xb9\xaa\xbe\xa4\xf8\x68\x24\xc9\x5a\x17\x52\xae\xa4\xdd\xdf\xa0\xbf\xff\xe7\x77\x7f\xd6\x5b\x68\x13\x1a\xf7\xee\x6f\x06\xfa\xfd\xff\xb8\xb4\xef\xcb\x35\xd8\x9d\x49\xa5\x95\xd6\xd5\x6c\xa8\x61\x02\xe7\x53\x06\xb7\x9f\x00\xe7\x05\xb0\x37\xd8\x0e\xf9\x3a\x56\x71\xb7\xcb\x42\xeb\xdb\xa9\x6c\x1b\x11\x13\x54\x6c\x6f\x8e\xe8\x14\x31\x8e\xa6\x26\xd6\x14\x33\xf4\xb7\x1f\xbf\xaf\x5e\xc0\x54\xd0\x8d\x3a\xa4\x16\x85\xc2\xeb\x52\xd2\x7f\x12\x89\xf4\xae\xd1\xbb\x98\x4f\x75\xd7\x82\xc8\x09\x4f\x62\xf4\x42\x40\x4d\xb2\x71\xa0\x99\x56\x2e\x48\x9f\xf9\x4d\x40\xc8\x21\xc2\x89\xe2\x63\x02\x77\xb5\x53\xd4\x14\x11\x5a\x54\x31\x59\x1a\x8a\x0b\x72\x62\x80\xd9\xee\xff\xea\x62\xab\x61\x9a\xf0\x93\x4b\x6a\xb1\x26\xb9\x78\x58\x3d\xf3\x51\xd9\xf4\x8a\xea\x6d\xf8\xe5\x45\xb6\x66\xdb\x6a\xd2\xd8\x24\x14\x6b\xc3\x2a\xaf\x4c\xf5\x60\x68\xc4\xd9\x20\xa1\xec\x69\xa3\xc5\x70\x89\xe1\x48\xb7\x60\x69\xa6\x5b\xcc\xec\xdc\xc6\x02\xb2\xc6\xf9\xf8\x98\x26\x89\x49\x6d\xf1\x97\x07\xe4\x2e\x43\x37\x10\x06\x66\x26\x07\x94\xc4\xd6\xef\x65\x35\x61\x41\x18\x04\xbc\xf5\xd9\x70\x6e\x7d\xb6\xf2\x04\xc9\x34\x9a\xb8\xcc\xbc\x88\x33\xa9\xc5\x68\x2e\x50\xc4\xa7\x53\xad\xf5\xc2\x92\x29\xce\x13\x69\xa3\xdd\xd9\xa9\xc2\x91\xea\xb3\xbc\xbf\x15\x27\xcf\x14\x65\xda\x2e\x75\xaf\xb9\x4b\x27\x2f\xfe\xb4\x54\xe0\xa6\xb1\x0f\x45\x01\x46\x30\xe3\x89\xf2\x40\x2d\xf8\xe2\x59\x32\x0b\x56\xa3\x19\xc8\x09\x17\x6a\x10\x57\xf2\x9c\x95\x9b\xa6\xcc\x08\x19\x39\x4d\x20\x68\x98\x3f\x6b\xe1\x9f\xbc\x64\xc6\xd7\x65\x43\xd0\xbb\x7a\xd9\x08\x9a\x1d\xa3\xa5\x23\x5b\x77\x0b\xd6\xd0\xca\x00\x93\x44\xc5\x98\xf0\x55\x63\xbc\x87\xb7\x2e\xf4\x4b\x4b\x89\x57\x3e\x77\x4e\x08\xe2\x71\x8e\xa1\x67\xee\x75\x9b\x11\xb2\x8c\xa6\x16\x3a\x61\x7f\x99\xa3\xcb\xa6\xf2\x58\xb4\xe4\xea\xb1\x80\xc9\x5e\x12\x90\x35\xb1\x18\x52\x25\xb0\x28\x00\xa0\x64\xfa\xa0\x24\x58\x40\x7c\x56\x9f\x19\x38\x3c\xa3\x29\xc4\x28\xa6\x12\x12\x44\xe0\x2e\xf5\x9c\x61\xa8\x99\x12\x58\x3a\xda\x79\x9e\xa3\x89\x3f\x87\xc0\xb2\x7c\x6b\x38\x66\xa7\x3b\xca\x60\xbf\xb4\x7e\xc6\xa3\x34\x17\xe4\x22\x90\x70\x2d\x54\x10\xa2\x4c\xd2\xf1\x44\x21\xca\xac\xdd\x11\x27\x63\x2e\xa8\x9a\x4c\xe5\x09\x1a\xa6\x52\x6b\xa1\x26\x58\xcd\xc4\xa3\x10\x15\x35\xe2\x42\xdb\x26\x11\xc7\xa5\x06\x17\x55\x94\x0d\xb6\x46\xb3\x43\xd9\x2d\xdd\x15\x2b\x36\x4e\x27\x83\x4f\x2c\xb7\x41\x89\xcc\x50\x37\x91\x89\x03\xe4\x0e\xb0\xea\xf7\x94\x48\x55\x77\x0e\x00\xec\x72\x67\x5e\x8a\x43\x54\xd2\x42\x26\x19\x54\x10\x17\xbb\x0d\x92\x57\x11\x70\xd3\x80\x52\x65\x4e\xa7\xe9\x4c\x55\x06\x6e\x2d\xba\x8a\xee\x3c\x28\xa3\x66\xc4\x86\x64\x2c\xd8\xcd\x00\x40\xd7\x67\xf7\x84\xd4\xe3\xd3\x2d\xac\xfd\x6f\x70\x94\x60\x0a\x36\xd1\x63\xf9\x96\xdf\xc6\x89\x7d\xd9\xbd\xbf\xb8\xeb\xdd\x1a\xc8\x89\x9b\xbb\xcf\x9d\x87\x41\x85\x5f\xbb\xe2\xa9\xcf\x9d\xbb\x1f\x2f\x57\x3f\xf6\xc3\x43\x31\x2b\xbb\xe2\x91\xbb\xfb\xe5\xc9\x1c\x0d\x86\x58\x91\x14\x56\xd9\xcf\x39\x9a\xcd\xd5\x84\xb3\x2c\x44\x21\x2e\xf0\xa6\x53\x64\x32\x82\x15\x84\x10\x09\xa9\x2a\x1c\x87\x0f\x10\x97\xb3\x5a\xc2\x2c\x2e\x96\x41\x97\xdb\xa9\x68\xb4\xc6\x89\xfc\x94\xf0\x21\xf8\xad\xad\xec\x63\x81\xe9\x96\x44\xa0\x6f\x19\xef\x73\x49\xe5\x2c\xc1\xf3\x85\x1e\x56\x5d\x39\xd7\x78\x4a\x20\xe2\x38\x87\xc5\x73\xc9\x22\x7a\x65\x20\x81\x29\xbb\xd7\xe9\x08\x32\x99\x14\xc5\x8a\xa0\x21\x51\x2f\x90\x37\xe7\xbe\xcd\x6c\xa9\x2e\x60\x44\x9e\xf5\x19\x98\x73\xfa\x9a\xc8\x71\x0a\xd1\x7e\xfd\x77\x27\xa8\xff\x2e\x26\xcf\x24\xe1\x33\xbd\xf2\xfa\x8b\x9a\x4b\xa6\x3b\xc5\x34\xb9\xe6\x2a\xb3\xcc\x6d\xb3\x9e\x82\x44\x74\x06\x92\xf9\x80\xe8\x76\x0f\x27\x78\x14\x76\xb2\x63\x67\x30\x06\x84\xe3\x58\x2b\xd9\xc0\xca\xdc\xf0\xf2\x10\x20\xe6\x4d\xbd\x50\xec\x72\x1d\x91\x22\x33\x7f\x9b\x1e\xfd\x36\x8b\x66\xcf\xca\x15\x60\xcf\x7b\x74\xc9\x6e\x7b\x91\x6b\xad\xe4\x47\x32\x87\x14\x8c\x5b\x4c\xc5\x86\xae\xd9\xaa\x98\xd7\xbd\x38\x69\xbb\x15\x1d\xb5\xc8\x5d\x5b\x4d\x87\xed\x1c\xb7\x59\xac\xde\xa1\xb4\x54\x17\xcb\x95\x75\xdc\x50\x6d\x7d\xac\x53\x52\x6b\x43\x18\x50\x59\xf1\x9a\x91\x68\x0d\x8d\x2b\x1b\xe0\xbd\x7e\x6f\xa5\xa6\x92\x89\x6b\x2e\xfe\x2e\x5f\x05\x9b\x1c\x5f\xce\xc7\x27\x2b\x47\x1c\x25\x5c\x16\xb1\x72\x1a\x0f\xfa\xc2\xbe\xba\x6c\xdc\x5d\x7f\xfb\x6a\xb9\x70\xad\x80\x86\x0a\xc2\x97\x30\x2e\xcd\x3d\xa3\xac\x87\xcc\x3e\x7d\x82\x28\x44\x5b\x82\x42\x96\xe4\xc8\x01\x2c\x46\xb9\x1b\xa4\xcf\xf2\x98\x15\x89\x5e\x48\x02\x61\x6e\x11\x9f\xce\xc0\xc4\x6f\x87\x6b\x5b\x22\xb1\x89\x18\x3e\x41\x3c\x55\xba\x31\x93\x93\xe3\x8c\xb8\x36\xe1\x27\x77\x7b\x18\xdf\x9b\x0d\x7e\xcf\xf0\xb2\xcd\x5e\x37\x77\x29\x65\xe8\x13\x51\xd0\x0a\x94\x59\xf0\x27\x08\x7a\x42\x39\x84\xb2\x9a\xf6\x5b\x9c\x28\x3b\x93\x35\x56\x3e\x07\x4e\xf9\x3e\xe1\xc3\xe5\x46\x02\x68\x1c\x3d\xde\xf5\x9c\x45\x32\x8f\x9f\xf2\x40\x99\x0b\x1e\xc5\xee\xed\x5d\xf7\xa2\xf3\xd0\xbd\x3c\x43\x8f\x92\x68\xf2\x64\xd3\x85\xfc\xea\x4c\x25\x31\x23\xb7\x48\x2c\x4c\x2a\x82\xeb\x0c\x21\x44\x88\x42\x16\xf4\x0a\xc6\x51\x84\x69\x59\xbe\xb1\x01\x24\x85\x5a\x43\x1d\x00\x0b\x95\xe7\x69\x23\xf3\x56\x9d\x40\x88\x93\x1a\x1c\x4f\x94\x9a\x19\xef\x74\x31\x32\x6f\xd5\xf6\x29\x46\xf4\xed\x7b\x32\x70\xb4\xd4\x84\x50\x81\x1a\x4d\xcb\x6c\xaa\x41\xf3\x39\x79\x21\xee\x9f\xf1\x6c\x79\xfa\x29\x7e\x29\x6c\x5a\x23\x0a\x7b\xbe\xfb\x7d\x9f\x03\xc7\xd6\x06\x86\x15\x6e\x3f\xc1\xdc\xa1\x65\x78\x6b\xc6\x37\x4d\xc6\x87\x74\x46\x32\x7f\x62\xa5\x41\xd8\x38\x56\x89\xe0\xec\xc0\x37\x94\xa1\xc2\x95\x78\x82\x46\xf4\x8b\x6d\x34\x8f\x6f\x77\x8f\x7a\x01\x0f\x35\xf1\x94\x13\xbc\x78\xa6\xd6\x10\x1b\x6e\xe1\xfd\xa5\x42\x24\x97\x5a\x24\x8a\xb4\xb8\x24\x48\xc4\x85\xbe\x29\xa0\xdb\xdc\x0b\xb1\x4a\x64\x50\x58\x68\xa2\x2c\x7a\x65\x96\x9d\xfe\xbc\xb4\x4a\x8c\x15\x39\xd5\xa2\xd7\x8a\x04\x68\x9b\x23\x03\xd9\x34\x58\x79\x70\x60\xf9\xcd\x33\x24\x63\xcc\x5c\x68\x76\xcd\x70\xdd\x95\xb7\x05\xab\xd2\x2a\x10\x86\xf4\x30\x90\xaf\x20\xf5\xa7\x30\x0e\x39\x03\x7a\x2e\x1d\x87\x8d\x7e\x69\x03\xd9\x5e\x70\x16\x8c\x53\x33\xd8\x74\x16\xb7\x69\xb0\x09\x96\x0a\xd9\x31\xd5\x99\x22\x3c\x15\x71\xbf\x46\xd8\x82\x6e\xdf\x54\x79\xd3\x5b\xa8\xa8\xc5\x12\xf0\x8c\x48\x87\x9b\x62\x50\x62\xb4\x4e\xe3\x04\xe1\x0b\x58\xa1\xec\x6c\xdf\x19\x29\xcb\xdd\x12\x3e\x33\x81\x20\xfd\xc5\xa6\xcf\x50\x87\x2d\xe0\x65\xb9\xb8\xac\x02\xbd\xcc\x9d\x84\x93\x17\x3c\x97\x68\x26\x0c\xb4\x8c\x89\xdc\x77\x93\x07\x0d\xac\xf8\x52\x16\x0a\xa1\x5c\xea\x04\x02\x5b\xcc\xea\xa0\x39\x27\xf7\x0e\xf6\xe0\xca\x2b\x45\x95\x67\x02\x79\xde\x5c\x6e\xab\x68\xc0\xea\x14\x19\x44\x13\xcc\xc6\x64\xe0\x8c\xac\x9b\x68\x4b\xba\x9d\x0b\x68\xe6\xd2\xb6\x52\x7d\x39\xdd\x1a\x85\xc9\x96\xb5\x31\x8f\x66\x06\x44\x7d\x08\xa4\xc2\x63\x82\xcc\x88\x1a\x99\xa5\x0b\x11\x63\x16\x6c\x18\xf4\x04\xdb\x6a\xb7\x18\x45\x5f\x27\xbc\x43\xe8\xd3\x15\x1e\x92\xe4\x75\x22\x27\xa0\x6b\x6b\x9c\x07\x6f\x9d\xc9\x06\x20\xe8\x05\xec\xf9\x25\x96\x61\xad\xf7\x22\xad\xca\x0d\x58\x36\x4f\x38\x72\xf6\xa4\x6d\x33\x51\x57\x02\x65\x93\xa9\xd6\x15\x46\xf1\xaf\x3d\xaf\x80\x48\x95\x81\xcd\xbf\xfe\xca\x36\xe5\xcd\x06\xe2\xd5\x31\xa9\x19\xc7\xd6\x85\x4c\x56\x4e\x65\x63\x90\x81\x86\x35\x0b\x7b\x23\xc4\x38\x23\x88\xca\xfc\x61\x55\x4c\x87\xca\x20\x7a\xb4\x88\x6f\x8c\x2f\x59\xf1\xb1\xac\xa6\xd4\xbe\x2d\x2d\x39\x78\x42\x66\x1b\x70\xf9\xdd\x8c\x68\x45\x15\x8b\x39\x40\x7c\x1a\x3e\x5c\x94\xe9\x56\x8e\x73\xe7\x02\xf7\x83\x43\x70\xf5\x22\x75\x15\x47\x20\x46\x96\x06\x87\x0c\x0e\xaa\x7d\xc8\xbe\x64\x61\x6a\xfa\x2c\xb3\x6c\xc0\x46\xa4\x12\x4d\xf1\x0c\x7c\x7a\x8c\xab\xfc\x2d\x03\xbb\xa4\xb2\x25\x3c\x71\x82\xb8\x34\xa5\xc1\x56\x53\x80\x8b\xf1\x36\xa1\x1a\xcd\xcb\x3f\x34\x37\x2c\xb9\xcb\x3f\x5f\xd5\x22\xb8\xa6\x03\x06\x1e\xd3\x67\xc2\xdc\x89\x3a\x71\x27\x52\x93\xc4\x4d\x39\x99\x9f\x62\x08\x72\x26\xb1\xef\x77\x59\xce\x0f\x8d\x39\xa8\x0d\xd6\xd0\xe6\x24\x7b\xa8\x0c\xe2\x31\x10\x6d\x05\x6c\x7d\x17\x96\xee\x9f\x11\x8b\x1a\x6c\xf2\xd0\xb1\x44\x7f\x64\x5c\xfd\xd1\xc3\x55\x76\xa6\x13\x78\xd5\x19\xc0\x4e\x16\xea\xe0\x00\xcb\xb0\xdb\x16\x61\x0f\xdf\x6b\x25\xe5\xb7\x8d\x4c\xc8\xc3\xee\xf7\x2a\x0b\x77\x17\x73\xf0\xea\x0a\x89\x85\x78\x01\x54\xbe\x94\xca\xe6\x56\x53\xab\x30\x3f\xe9\x05\x33\xab\x5c\x15\x20\x90\xad\x45\xa3\xc0\x80\x05\x3c\x82\x6d\x76\xdb\xb4\x71\xdc\xd6\x0a\x08\xe6\x6a\xab\xc8\x26\x59\xa6\x75\x5a\x81\x28\x06\xce\xd9\x22\x1c\x35\x18\xc3\x67\x7d\xf6\x91\x0b\x2b\x00\x48\x5b\xe5\x60\x88\xa3\xa7\x53\xc2\x62\x84\x53\x35\x31\x58\xbf\xd6\xab\x31\xb7\xbb\x41\xcb\x39\xb0\x6d\x32\x20\x0f\x2a\x23\x2c\x62\x57\x6f\xe3\x99\xbb\x51\xf4\x99\xd7\x08\xd4\x51\x80\xea\x59\x50\xd6\xb8\x4e\xd1\x25\x52\x6b\x77\x75\xb4\xa8\xaa\x6c\xbb\x50\xd7\x76\xf9\x39\x2b\x54\xea\x85\x0a\x10\x10\x5e\xc5\x47\x8b\xd4\xe9\x39\x5b\xa7\xd3\x2e\xf5\x7e\x5e\xf4\x81\x9c\x58\x7d\xc6\x18\xc4\xec\x0c\xb4\x9c\xf5\x9d\xe3\xb5\x05\xcc\xe2\x51\x2a\x20\x58\xb8\xaa\xcd\x6f\xa2\x09\x4d\x72\xcf\xc9\xb7\x27\xd9\x30\x75\x93\x09\x79\x26\x89\x41\xcc\x8f\x04\xe4\x05\x18\x9b\xe5\x77\xe8\x7f\x9b\x6a\xaf\xe8\xcf\x7d\xf6\x09\xd8\x70\x92\xcc\x01\xcf\x33\x6b\x19\xab\x52\x33\x4f\x95\x03\x50\x36\x11\x09\x15\x07\x62\xd6\x7a\x82\x9f\x49\x9f\xb9\x66\xfe\x37\x7a\x42\x7f\x42\x7f\xae\x53\x2e\x5d\x78\xff\x9e\xad\x2c\x1f\xbd\xe0\x79\xef\x96\xb3\x8c\xd2\xf2\x1b\x67\x84\x29\x98\x40\x2b\x70\x3d\x32\x58\x6e\xca\x9e\x79\xb4\x90\x43\xe2\x9f\x5a\x2c\x08\x53\x03\xc6\x63\x32\x20\x15\x0e\xd5\x25\x4c\x42\x0b\x01\xd7\x3c\x26\x2b\xdd\xa1\x19\x33\xfd\x05\x0c\x47\x32\x1d\x66\xcb\x01\xf0\x02\x59\x2e\x79\x66\xfb\x28\xee\xb4\xea\x91\x67\xd8\xb7\x9b\x8c\x7b\x53\x57\x6e\x2e\x36\xe2\x1c\x7c\xb7\xda\x9d\x98\x60\xe5\xa4\xc9\xf2\x71\x2c\xbb\x21\xf4\xc3\x7a\xe6\xf6\xb2\xf2\x50\x7d\xa1\xf2\x8a\xa0\x63\xaa\xb5\x87\xe6\xee\x62\xe0\x84\x9b\xf8\x52\x0c\xc4\x69\x23\x67\x4a\x4e\x0a\x07\xf3\x72\x9a\xed\xbf\xdc\x05\x3a\xe4\x69\x59\x7d\xb0\x04\xa0\xd2\x0f\x36\xb0\x9a\xc2\x5c\xf3\xe1\xb1\xc9\x3f\x24\x13\x6a\x32\xfe\x3b\x17\x57\x48\x9f\x0e\x3e\x35\xb0\x58\x40\xb4\x54\x4d\xb8\xa0\xff\x5c\xb6\xb7\xb1\x50\x74\x84\x23\x35\xd8\x49\x15\x99\xfa\xcd\xd4\xb1\xfd\xf4\xea\x2b\xd5\x2d\x20\x1c\xe0\x67\xe2\x05\x20\x42\x78\xa1\x6d\x45\x66\x8e\xdc\x32\xbf\xe5\x02\x31\xfe\x92\xc3\x62\xb9\xf7\x01\x09\xda\x4b\xdc\xc0\x5a\xe5\x9a\x41\x04\xb1\xa4\xb0\x3f\x01\xa4\xea\xbd\x32\x49\x99\x00\x70\x6e\xe0\xa5\xf4\xf6\x9c\x60\x16\x27\xee\x0a\x41\xdc\x44\xf4\xcc\x5f\xf0\x7c\x2d\x9f\xba\x1f\x57\x99\x67\xe9\x99\xe5\x2f\x2a\x41\xc0\x03\x8c\xa4\xa6\x0a\xaa\x66\x95\x22\x8c\x86\x29\x00\xeb\x6a\x9a\x8c\xd2\xc4\x54\xe3\x88\xb8\x88\xcf\xfa\xcc\x06\x54\x7b\xbd\x69\x11\xd0\x69\x4d\x58\x65\x0d\x52\x8b\x3f\x6a\xeb\x7d\x18\xb3\xdc\x52\xb9\xfe\xa7\x94\xa4\x3b\x4a\xab\x7c\xd5\x40\xf4\x07\x3c\x96\x79\x64\xb9\xa1\x8d\xbe\xf2\x72\xfa\xfe\xae\x67\x2a\xbd\x44\x64\x67\x2e\xce\x70\xbd\x8c\x9d\xc5\x14\xb1\x5d\xcb\x4c\x77\x67\xea\x19\xec\xc0\x4e\x77\x88\x20\x9d\x45\xd1\xb3\x82\xab\xdb\xed\xf7\x9c\xa5\xe5\xa2\xc3\x18\xbf\x5c\x61\x88\x92\x50\xb7\x47\x3b\xd8\x06\x77\xc7\xa2\xae\xb2\x34\xd4\x3d\xb7\x8a\x65\xb7\x45\x45\x86\xbc\xe2\x90\x4b\xf3\x22\x28\xc0\xfe\xcd\xf3\x87\xb3\x7a\xcd\xee\x16\xf6\x79\x8c\x16\xfe\x8c\xb6\x00\xc1\x3a\x8e\x84\xf3\xea\xab\x73\x0d\xbb\x8e\x6d\xa8\xd8\xf5\x62\x30\x46\xdd\x89\x30\x2c\xa9\xad\x47\x62\x11\xf3\x67\xe5\x61\xc8\xca\xbb\xbc\x8e\x55\x38\x93\x18\x0f\x77\x32\xb2\xed\x38\x88\x70\x34\xa9\x9d\xd4\x90\xf3\x84\x60\x56\xa7\x14\x54\xfe\x5c\x3e\x22\x06\xb1\x16\x58\x77\x92\x00\x6c\xb3\x23\x81\x2d\xf5\x99\x6b\x45\x2c\x06\xb8\x7d\xc3\xc3\x4d\xc0\xa7\x1b\xa8\x22\xcc\x19\xd4\x28\x1b\x27\xa4\x4c\x2b\x5b\x17\xe1\xc4\x76\x92\x44\x69\xe2\xd5\xfa\x9c\x11\xa1\x47\xad\x49\xfc\x4c\x98\x56\xc5\xec\x38\x9c\x87\xea\xc5\x65\xb9\x67\x15\xbe\x4e\xb2\xae\x9d\x93\x14\x52\x49\xe3\x3e\x83\x83\xcb\x8b\x87\x55\xef\x55\xa9\xb5\x37\xdf\xdc\xb7\xf1\xe9\xf4\x84\x88\xb5\x8f\xe7\x7d\xd1\xf6\xbf\xf6\x99\x34\x7d\x0f\x20\x70\x64\x6b\x7f\xa9\xe7\x53\xcb\xf1\x37\xcc\xc2\x3a\x8c\xb5\x03\x79\x06\x20\x14\xa7\x18\x4b\xec\x45\xe2\xd4\x61\x7c\xed\xf5\x2e\xc9\x6b\x9a\xb8\xdb\xa0\xe1\x50\x96\xc6\x1f\x34\x8c\x26\x00\xa3\xef\xb2\x73\x7b\x65\xa5\xfa\xa2\x1f\x3e\x4b\xda\xca\x63\x57\x6d\xd5\x5e\x25\x30\x40\x4e\x00\x50\xc0\x2f\xc6\x70\x41\xa5\x11\xee\x5d\xed\x92\xe9\x4c\xcd\x6d\xa9\x3b\xb8\x17\x0b\xf2\x3e\xc0\xf8\x55\xf9\xfc\xcb\x77\x64\x5c\xf0\xfa\x57\x75\x06\x1d\x59\x6b\x4d\x65\x93\x8e\xd0\x3e\x2c\x4c\x09\x86\xa3\x2e\xc4\xc7\x54\x0d\x1e\xe0\xa4\xd6\x44\xb8\x03\xa6\x09\xca\x51\x0e\xbd\x61\x11\x7d\x95\x48\x89\xe6\x5d\x38\x49\x4a\xf3\xc2\x90\xe3\xae\xb2\xca\x81\xc3\xbc\xbc\x71\xf3\x08\x84\x04\x0f\xc9\x5a\x31\x07\x57\xe6\x85\xa5\xbb\x08\x1e\x81\x70\xfd\xd9\x2c\x99\x37\x4b\x13\xf0\xb5\xdf\x4a\xe4\xbb\x55\x03\xf3\xf1\xf2\x96\xde\x4d\x45\xcc\xb9\xcd\x86\x28\x49\x94\x0a\xaa\xe6\x03\x6b\x4b\x6d\xce\xb4\xee\xed\x9b\x17\xf6\xc5\x26\x86\x8a\x73\xe4\xfa\x73\xb6\x5b\xb8\xa7\x04\x35\x65\x91\xec\x14\x9a\x2c\x37\x4e\xd5\xa4\x12\x11\x6b\x19\x61\x1d\x24\x57\xb3\xa1\xea\x2e\x36\x1d\x9e\x2d\xb7\x32\xe0\x23\x07\x76\xd5\x9c\xb0\xe5\x3a\x34\x6b\x18\xa1\x1d\xa6\xf6\x4c\x50\x2e\x6c\xb9\x97\x26\x91\x8a\x53\xfc\x65\x30\xc3\x02\x27\x09\x49\xa8\x9c\x6e\x6e\x32\xff\xeb\x5f\x96\x8e\xf6\xc2\x94\x25\x32\x83\x9d\xe2\x2f\x74\x9a\x4e\x11\x4b\xa7\x43\x2b\xe5\x62\xf9\xe4\x23\x9a\x3a\xfc\x05\x03\xcc\xe5\x06\x58\x40\x81\x10\x1e\x46\x6d\x9f\x79\x68\xe5\xd6\x54\x81\xa3\x09\x25\xcf\x80\xa5\x2a\x18\x91\xf2\x0c\x5d\x73\x45\xce\xd1\x67\x3c\x7b\x00\x41\xcd\xd4\x09\x1d\x1b\xa7\x03\x96\x48\x4b\xad\x29\xa3\xea\xa4\xcf\x2c\xc4\xb9\xa3\xca\x87\x88\x33\x03\x73\x1b\x01\x61\xb3\x26\xc0\x8a\xee\xf0\x5e\x95\xcb\x56\xa5\xb2\x86\xd8\x02\xbf\x0c\xbc\x90\xe4\x81\x49\xf9\x58\x63\x1f\xdf\xe1\x17\x13\x84\x7f\x89\x15\x36\x25\x80\x97\x49\xee\x36\xca\xcd\x96\x85\x32\xe8\xce\x2e\x1a\x88\x5b\x88\x91\xac\xa0\x9d\x09\x39\xfe\x86\x9e\x91\x33\xf4\x7d\xc2\x87\xf2\x24\x37\x55\x99\x1f\x25\x51\xf2\xc4\xf8\xfd\xe0\x6f\x93\xdf\xf7\xad\xa3\x7e\xce\xf7\xa1\x96\xe3\x88\x7e\x31\xc8\x26\xf2\xaf\xe7\x1f\x3e\x4c\xe7\xa7\xc3\x34\x7a\x22\x4a\x7f\x02\x99\xa2\x92\x42\x0e\x16\x0c\x57\x81\x8c\xad\xa2\xce\x22\x40\x59\xa3\x1d\x69\x73\xa5\x24\x01\x30\x7c\x7d\xa5\x67\xd5\x72\x1d\x9e\x15\x67\xd5\xa5\x40\xed\x94\x45\x5a\x77\xbc\x0a\x28\xda\x87\xd1\x56\x4c\x35\x60\x1f\xbc\x7b\x94\xe0\x71\x49\x65\x59\x43\x49\xb9\x99\x52\xbb\x8b\xf4\xdc\x21\x88\x46\x9f\xb2\x62\xe8\xe0\x7b\xe7\xe5\x05\x6f\xad\xf5\x62\x9d\xf5\x59\x47\xa2\x17\x62\x8a\xfc\x42\xa2\x29\x38\x7d\x52\x2a\x27\x59\x9a\x29\x98\xa1\xa1\x51\x83\x71\x6c\xa0\x30\xac\xe2\xe8\x34\x2b\xe7\x16\xb3\x1a\x28\x4e\x24\x39\xd1\x0d\x83\x49\xd5\x45\x87\xa2\x17\x81\x67\x33\x22\xfa\xcc\xe2\xd5\x02\x2a\x3b\xe7\x36\xf2\xa7\x2e\x45\x20\x68\x94\x87\xd5\x28\x3d\xda\x93\x62\x16\xea\xaa\xf3\x0d\x49\xab\xcb\x28\x5c\x95\x87\xe9\xc8\xa7\x65\xd1\xa6\xe1\xfb\xaf\x6f\x36\x6e\x38\xe6\x55\xda\x79\xa7\x94\x7b\x01\x35\xc4\xa7\xa0\x40\xca\xbc\x54\xaa\xb3\xf5\x65\xea\x7b\x41\xcc\x01\xb8\x73\x78\x39\xe6\x44\x7a\x46\x7c\x94\xd9\xe2\x12\x3a\x22\x5a\xfa\xe8\x33\xbd\x8d\x7d\x87\x83\x41\x4d\x77\x20\xea\xba\xd3\x48\x70\x29\x6d\x3a\x85\x69\x67\x79\x52\xdc\x16\x05\x1a\x0d\xf4\x7b\xef\xe6\x7a\xb0\x58\xaa\xd1\xfb\xcd\x15\x6d\xb4\x3f\x56\x22\x27\xd4\x36\xb5\xb2\x44\x63\x4e\x8b\x35\x8a\x34\x7e\xb8\xb8\xea\x65\x95\xc9\x4a\x5d\x2f\x56\x69\xf4\xe1\xf2\xeb\xeb\x34\x2e\xce\xd8\xab\xd8\x58\x6a\x62\x49\xcd\xc6\xd5\x8b\x55\x0c\xe2\xde\x06\x0b\xb1\xb4\xf4\x2b\xf9\x43\x71\xcf\xac\xca\x35\xd8\xd1\x32\xd5\x5c\x2b\x11\x08\x8c\xfb\x0e\x5c\x00\xc1\x4b\x3f\x25\x15\x9e\xce\xfc\x3c\x5a\x07\x06\x6b\xa7\x69\x8e\x5a\xdd\x25\x78\x50\x90\xfa\x08\x9b\x20\xa1\xf2\xe0\x16\x96\x62\x3d\x8f\xd7\x83\xc5\xbe\xdf\x45\x6c\xfa\xe1\x12\xd3\x93\x79\x1e\x0c\x29\xad\xec\xe6\xea\xaa\xd7\xd8\xfd\x87\x24\xc3\xf9\xaf\x5d\xd0\x6d\x33\x4f\x33\x3c\x30\x41\xb0\xb4\xee\x6f\x48\xd0\x2c\x25\x6f\xad\x61\x1e\xce\xc6\x6c\x52\xbc\x4f\xb3\xca\x1a\xde\x55\x63\x8b\xc5\x45\xee\x20\x52\x21\xc8\x33\x11\xb0\x77\x6c\x28\x15\x2b\x1e\x55\x9c\x08\x82\xe3\xb9\x47\x91\x2c\x8e\xc3\xf4\x0c\xe6\x31\x49\xa7\x5a\x81\x07\xd5\x84\xf1\x53\x3e\x73\x3a\x4b\xe1\x29\x28\x8b\x42\x47\xfa\xc6\xf2\xa2\x40\xf4\x1b\xec\x94\x7c\xa1\x52\x69\xb9\xa2\x22\x04\xd6\x35\x02\x12\x0f\x14\x4b\x9b\x10\x7b\xc3\xf5\xdf\x75\xbe\xbf\xb9\x7b\xe8\x5e\xf6\xdf\xe5\x29\x17\x2e\xa7\x30\x83\xf9\x72\x55\x1b\x38\xeb\xb3\x2c\x4e\x39\x43\xb5\x86\xb5\x44\x38\x8e\xf3\xf8\x68\xab\x44\x1a\x99\x6d\x29\x47\xf6\x4e\xc5\xca\x08\xe5\x25\xcd\x3c\x42\x62\x59\x5b\x4f\xd6\x12\xd7\x59\xe1\xe4\x98\xf4\xb8\x25\x79\x4c\x3b\xba\x6c\x7c\x40\x5e\x65\x74\x6d\xa2\x1c\x62\x24\x23\x2f\x4e\x57\x82\xdb\xf9\x03\x36\x97\xf0\x7a\xdc\xce\x2d\xc8\x06\x8b\xfa\x91\x7e\x21\xf1\x5d\x8d\x54\xb5\x93\x34\xa5\x46\x01\x96\x95\xab\x90\x32\xba\x8e\xc6\x9f\x4d\xe5\x51\xbf\xd7\x9c\x2d\xdd\xe4\x38\x7b\x39\x66\x2e\x00\xe6\x2a\x84\x51\x44\x84\xc2\x94\xa1\x11\x1c\x6c\x16\xcd\x11\xa0\xb0\x10\xf0\x61\xff\x05\x4d\x29\x03\x38\x88\x65\xa4\x7d\x2c\xce\x63\x0d\xa1\xf5\x73\xef\xfa\xf1\xa1\x20\xaa\xfe\x70\xf3\x58\xac\xd4\xdf\xf9\x75\xa9\xac\x5a\x6a\x61\x59\xb0\x90\x37\xc5\x3c\xb5\xd4\x42\x07\x67\x94\xa9\x9c\x68\x32\x57\xe4\xf1\xee\x6a\x2b\xf9\xae\xda\x59\x56\x0b\xfc\xee\x4b\x57\xd5\x30\x17\x4d\x5e\x8d\x49\xb4\x0a\x9a\xb6\xf9\x3e\x32\x51\x50\x9a\x0e\xd6\x9a\x68\x61\xeb\xb0\x44\x33\x2c\xac\x1f\x2a\x36\x01\x50\xc5\x72\x6f\x46\xf3\x5a\x06\x0b\xf2\x89\xa8\x9f\xf5\xd5\xc7\xd9\x2e\x92\x4b\xac\x28\x0b\xfe\x51\x32\x78\x36\x0d\xaf\x71\xd2\xec\x50\x96\xe4\x2f\x39\x61\x19\x7a\x40\xb6\x07\x1f\x4c\xe3\xcc\x94\xfd\xef\xe8\xe6\x80\x22\x2e\x4c\x53\xab\xa4\x9c\xe9\x1d\x69\x30\x72\x1d\xb0\xae\xd7\x1c\x1f\x99\x97\x1b\xc2\x0c\x7a\xc9\x02\xba\xad\x9c\x94\xa8\x73\xdb\xab\xa0\xf5\x55\xd9\x85\xf4\xb6\x6a\x14\x25\x99\x37\x6b\xd7\xc8\x57\x5e\xce\x69\x2b\xa0\xae\xec\x4c\xb7\xc3\xb6\x32\x4e\xff\xdb\x62\x24\x41\x1b\x20\x98\xab\x54\x86\x42\x2e\xf9\x0a\xb4\xe5\xf5\xd2\x2b\x73\x32\xac\x89\x64\xe5\x0f\xc8\x66\xd7\xf8\xe8\x4d\x8b\xa1\xdb\x27\x3e\x9a\x13\x37\x55\x90\x6d\x6c\xc1\xce\x10\xae\xf2\xd9\x34\x81\xb8\xfa\xd9\xec\xe8\x0c\x01\x05\x30\x5d\x5c\x95\x4d\x17\x72\x6d\x01\x09\xfc\xe9\xfa\xbb\x6d\x3d\x54\xac\x7c\x7c\xce\xfc\x6d\x01\xc6\xf1\x0c\x5b\xbb\x03\x28\x51\xae\xfc\x45\x55\xb5\xc4\xb3\x3e\xf3\x02\x56\xa4\x51\x7b\xf4\x19\x71\x15\x67\xa0\x8c\x31\x03\xb4\x72\xc8\x7d\xca\x84\x9f\xc2\x0a\x94\x71\x0f\xd4\xa4\x58\x33\x66\xa1\x1f\x7b\x3a\xe5\x04\xbb\xec\x52\x67\x41\xb1\x71\x80\xbe\x7d\x09\xda\xf3\xaa\x44\xd8\x8e\xc1\x1c\x0d\x46\x0b\xec\xd5\x20\xf4\x10\x09\x62\x4e\x24\x7b\xaf\xb2\xfc\x5d\x9a\xcc\x5d\x48\x75\xc9\x3d\xa0\xa5\x3a\x4c\x6d\xcb\xcb\x0f\xf8\x0e\x20\xb7\xd6\x55\x1c\xbc\x63\xb5\xd2\x4c\xe5\x7c\xbc\xb0\x13\xfc\x58\x24\xe8\xb4\xce\xaa\xfe\x65\x46\xa2\x4d\x70\x81\x6e\xb1\xc0\x53\xa2\x88\x58\x16\x8e\x54\xac\x10\x0e\x22\x8e\x5b\x41\xdb\xaf\x59\x45\x53\x3e\xa5\x5c\x67\x27\xd3\x6e\xaf\x56\xe1\xfc\x64\xb3\x58\x0b\xd2\x4c\x4f\xe3\x67\x6b\xf9\x5f\x73\x16\xb6\x9f\x7c\x1a\x36\xda\xca\x83\x75\xda\x76\x4e\x87\xc1\xb7\x79\x58\x40\x8a\x29\x84\x0b\xb5\x04\xd8\x66\xf5\x28\xeb\x10\x6d\x56\xf1\xd2\x9d\xf0\x6e\x97\xe1\xe0\x32\x93\x4b\x87\xaa\x90\x3b\x01\xbb\x04\x54\x2a\x03\xee\x52\x8d\x4a\x03\x42\x4b\x55\x84\xa4\xe7\xf6\xb3\x98\x85\xb9\x41\xd7\x4a\x56\xe5\x8a\x61\x25\x72\xad\xe0\x71\xbb\x42\xec\x08\x12\xcd\xae\x25\x9a\x55\x5b\xb9\x10\x5d\xab\x77\x27\x11\x25\xf0\x20\x5b\xc9\xdb\xa2\x3e\x14\x27\x08\x29\x5d\xf6\x8a\xb4\xe5\x80\xe1\xea\xa7\x2c\xfb\xab\xc8\xc1\xdd\xa6\xf6\xb7\x6a\x55\xae\xea\x99\xe7\x82\x02\x0f\x54\xe2\x4b\x03\x36\xae\x06\x46\x6b\xc2\x20\x8d\x95\xbf\x77\x6d\x1c\x58\x90\x33\x3e\xe7\x29\x7a\xa1\x72\x82\x14\xef\x33\x88\x13\xcc\xbc\x01\x8a\x23\xf3\xe0\x09\x3c\x05\xd8\x16\x32\x1d\x4e\xa9\x42\xd8\x9b\x61\xc1\x24\x79\x62\xcf\xb3\x7e\x01\x66\x5c\x09\x5f\x50\x85\xbb\xb4\xe2\xd0\x6c\x60\x5f\xcb\x1b\xd9\x16\xa1\xc0\x8b\x69\xde\x2f\x46\x81\xa7\xf1\xf8\x1a\x66\xe5\x99\x0b\x20\x05\xa8\xda\xda\x60\x91\x60\x01\xae\x97\x4a\x55\xba\x5b\xac\xa1\x67\x05\x40\x41\xbe\x10\x8d\x10\x0a\xf2\xc7\x77\x01\x51\x50\x57\x7b\x6e\x59\xca\xaa\x7b\xa5\xc6\xfe\xed\x52\xa1\x15\x77\x81\xf3\xbe\xa4\x74\x5b\x2b\x29\xb5\x0d\xaa\x2e\x4f\x08\xd8\x3c\xbc\xbc\x2e\x7a\x19\xce\x78\xc4\x59\x4c\xd7\x88\x17\x86\xfa\x62\xc3\x74\xd4\x61\xf3\xd5\xc8\x47\x53\x3f\x50\xdf\xda\x4b\x3c\x49\xa4\x1a\x73\x73\xa5\xca\x9a\xb7\xef\xef\x74\x2f\x25\xb4\x08\x46\x44\xca\xb7\x13\xe3\x0a\xf2\x7e\x22\x95\xcc\x2b\x72\x51\xfb\xac\x5a\x4a\x5a\xce\xb7\xb7\x4d\x23\xd9\x29\xec\x9e\xc7\x23\xdc\x2c\xac\xd5\xed\x97\x2c\x10\xcf\x28\xf4\xc4\x82\x6c\x94\xc4\xe0\xdc\x0d\x59\x17\x40\xa5\x85\xa3\x4d\x72\xcd\x2b\x38\x47\xf5\xd0\x17\x92\x3c\x56\x9e\x5d\x2b\x18\xec\x50\xfd\x5c\xb8\x41\x1a\xe7\xc4\x64\x72\xbc\xbd\x31\x6c\x50\x77\x9c\xd9\x1a\x4a\xee\xe4\x4d\xca\x1b\x03\x9c\xed\xce\x40\x78\xcb\xc8\x14\xba\xf1\x13\x70\x41\xdb\xb1\x63\x13\x8e\x93\x41\xc3\x97\xd6\xa4\x30\x63\x13\x52\xb9\x97\x59\xaf\x5b\x93\xda\xf3\x89\x0a\x1b\x93\x4c\x7d\xeb\x06\x14\xa3\xb6\xa1\x9c\xa5\xdb\x22\x13\x40\x53\x16\x13\xc1\x08\x56\x93\xc3\x65\x82\x5c\x6c\x6b\x42\xf7\xc6\xb7\xdf\xac\x10\x3b\x52\x5c\x4c\x0e\xd9\x66\xb8\xc5\xe2\xfc\x2b\xc7\xa9\x1f\x6f\x62\xcd\xb2\x01\x12\x59\x79\xea\x05\xf5\xb6\xc2\xb4\xe9\xe1\x0f\xad\xb3\x4b\xb7\x4a\x16\xa9\x56\x39\xf7\x93\x36\x53\x61\x9b\x5a\x48\x98\xd1\xa7\xdd\x2f\xea\xbd\x82\x24\x6f\x22\x3f\x65\xff\x29\x13\xcb\xca\x87\xa7\x5e\x16\x05\xd4\x70\x57\x98\x32\xcb\xbd\x96\x25\x4e\x68\xb9\x77\x8a\xab\x72\x25\x5a\x9f\x85\xf3\xe6\x93\x70\x42\x4a\x46\x48\xc9\xa8\x58\xa3\x90\x92\x81\x50\xdb\x52\x32\x56\xa9\xa0\xcb\x8c\xb4\x99\xdf\x10\xca\xbc\x16\x6a\x2b\x99\xf5\x5d\xa1\x47\x6e\x9e\x76\xe0\xec\x9c\x7e\xcc\x96\xfd\xc6\x7e\x51\x19\xb6\xb5\xf0\x5a\x79\xb6\xbe\xcd\x95\xcd\xcb\xae\x0b\x2c\xe2\xc4\x42\x10\xda\xa0\xea\xa2\x8d\x6c\x99\x39\xb7\xcf\x7e\xe0\x2f\xe4\x99\x88\x13\x84\x15\x9a\x72\xc0\xb5\xca\x63\x78\xe0\x20\x14\xb0\xf4\x4d\xac\x06\x46\xd7\x78\x4a\x62\x53\x6a\xd3\x0b\xbd\xb4\x46\x65\xeb\x0e\xae\x42\xda\x05\xd0\x58\xb3\x0c\x2e\xb6\xa3\xcf\x4c\x38\xa4\x09\xc1\x03\x59\x81\xba\x89\xc1\x86\xf9\x63\xe6\xac\xfe\xe3\x19\x7a\xd0\xf7\x13\x95\xc5\xf1\x7a\xc0\x7b\x75\x63\xeb\xb3\xb1\xe0\xe9\x2c\xb3\xf3\xf1\xa1\xa9\xb9\x6c\x22\xb4\x16\x9d\xd5\x30\x18\xe7\xa9\x8e\x70\xac\x75\xf1\xe5\x1b\xe7\x55\x22\x65\x37\x82\x59\xf2\x37\x90\x3e\x86\x59\xf8\x9f\x0d\xc7\x37\x3e\x66\x0f\x5c\x66\x59\x05\x80\x3d\x39\xc0\x2f\x89\x04\xab\x50\xe6\x19\x28\xe4\xba\x17\xf1\x14\x2a\xc7\xb9\xcc\x6e\x9b\xf9\x56\x9c\xff\xa1\x1a\xaa\x21\xef\xdc\xc6\xa5\x99\x44\x5a\x7b\x4f\xec\xcd\xa2\xdb\x38\xc2\xb7\x8e\x5f\xdc\xa6\x62\xc6\x41\x12\x4b\xe6\x0e\x5a\xc2\x82\xfc\xcd\xf8\x2c\x35\xb1\x77\xd4\x0f\xc5\xaa\xdc\xd9\x54\xaa\xcf\x58\x45\x13\xcd\xb9\x73\x54\xb6\x1d\xc5\x24\xe6\x5c\x79\xbf\x56\xde\x8a\x19\x5c\xf8\xbd\xd7\xb8\x3d\x96\xed\x1e\x2f\xc6\x30\x0b\xe4\xcc\x24\x89\xa9\xee\xcf\xb8\x06\x6d\x25\x75\xcf\x2e\xea\x5e\xb1\xbf\xe8\x89\xae\xda\x45\xab\xc6\xdf\x6c\x6f\x15\x4b\xbd\xed\x3c\xda\x71\x0b\x98\x9b\x4b\x0b\x2a\x96\x3f\x68\x4b\x03\xd7\x84\x28\x08\xba\x59\xa6\x92\x2d\xcf\xf0\xac\xc5\x91\xcc\xe2\x3a\xc5\x33\xad\x44\x28\xae\x6f\x49\x31\x36\x72\xac\x89\xe5\x45\x18\xa5\x82\xba\xb3\x5f\xca\x5b\xaf\xdf\x1d\x60\xa1\xfc\xe0\x97\xf2\x8a\xb0\x57\xe5\xd0\x04\x25\xe0\x48\xa5\x38\x0b\x9e\x84\x3d\xe1\xaa\xef\x9b\x1c\x7d\xe7\xfc\x17\x4e\xbc\xab\x58\xd3\x95\x1b\x7b\x8b\x55\xc6\x55\x18\x8c\x8d\x4e\x1a\x65\x63\x0f\xc0\xb1\xda\x4a\xdc\xa4\xe8\x46\xe5\x9b\xcd\x0a\x87\x54\xbe\xea\x64\x9f\x4d\xde\x5d\x02\x30\xd5\x28\x64\xbd\x8d\x15\x13\xbc\x4c\x00\x1b\x2a\x6c\x65\x37\x1f\xd8\xd3\x76\x04\xb0\xc7\x14\x42\x19\xb0\x93\xe5\xbe\xf1\xcb\x26\xe8\xa1\x7d\xfb\x5f\xf9\x8f\xa0\xbf\xdb\xe2\x2c\x15\x0f\xf6\x19\x17\xf6\xd1\x93\xec\x39\xfd\x58\x8e\x4f\xac\xa5\xc4\xc5\x37\x73\xf4\x51\x51\xc4\x29\x04\xb4\x16\x8b\x33\x67\xe0\xa9\xb3\xb2\x16\x7a\xf0\x4f\xe9\x90\x08\x46\xf4\x9c\x1c\xae\x43\xc6\x83\xa7\x98\xe1\x31\x80\x61\x9f\x40\xd0\x21\x48\xd9\xb9\x06\x65\x4e\xa2\xa9\x0f\x0a\x4c\x56\xf3\x78\x9b\xca\x9c\xd7\xc9\x86\x3e\x8d\x04\x6e\xb1\x78\xf3\xc8\x95\xea\x43\x7b\x67\xfb\xdf\x4c\xd1\x78\xe8\xdc\xff\x38\xb8\xeb\xde\xdf\x3c\xde\x5d\x14\xb4\x8d\x8b\xab\xc7\xfb\x87\xee\x5d\xe5\x6f\x79\x1a\xf0\x4f\x8f\xdd\xc7\x9a\x9f\x5c\x03\x57\x9d\xef\xbb\x85\xa2\xf3\x3f\x3d\x76\xae\x7a\x0f\xbf\x0e\x6e\x3e\x0e\xee\xbb\x77\x3f\xf7\x2e\xba\x83\xfb\xdb\xee\x45\xef\x63\xef\xa2\xa3\xdf\xf4\x9f\xbd\xbd\x7a\xfc\xd4\xbb\x1e\xb8\x88\x6e\xff\xa7\x5f\x6e\xee\x7e\xfc\x78\x75\xf3\xcb\xc0\xeb\xf2\xe6\xfa\x63\xef\x53\xd5\x2c\x3a\xf7\xf7\xbd\x4f\xd7\x9f\xbb\xd7\xcb\x8b\xdb\x57\x53\xa3\xb6\x6e\xb6\x77\xff\x7a\xb6\x2e\x4f\xba\x1b\xce\xed\x99\xa0\xff\x04\x97\xcb\xad\xd9\xa2\xa7\x27\xee\x93\x29\x45\x7f\xaa\x39\xb7\x73\xe7\xe5\x4c\xaf\xcf\x32\x9f\x70\x26\x0b\x28\x3c\x96\x2e\xab\xbb\x30\xda\x73\xd4\x81\x43\x06\x7a\x4e\xa1\x53\x48\x1a\xc9\x46\xea\xa2\x08\x60\x1f\x26\x74\x4a\x21\xa0\x00\x9d\xa2\xf2\x82\x17\x1b\xb4\x73\x82\x21\x58\x77\x63\xbc\xec\x34\xc8\x72\xc2\x38\xec\x94\x73\xe4\x2e\x16\x62\xac\x20\x06\xd6\x77\xce\xf0\x94\x46\xe5\xec\x16\x40\xb6\x45\x39\x8a\x4b\xb9\xc5\xc2\x06\x2b\xb6\x3c\x21\xe8\xc7\xbf\xe7\x83\x02\xc7\x8b\x35\x18\xa4\x0b\x15\x28\xed\x0f\x22\x35\x54\x5d\xb5\x3d\x0b\x3d\xb9\x63\x6e\x2d\xe2\x70\x6e\x6d\xa5\x7b\xf0\x92\xa5\xcc\x43\x72\x2b\xb8\xcc\xf4\xf1\x36\x33\x2a\xed\xf1\x73\x74\x0f\x28\x32\x32\xb7\x38\xe8\x55\x9c\x25\xe9\x98\x32\x44\xa7\xb3\x04\x78\x8c\x31\x43\x0c\xc9\x04\x3f\x53\xee\x0a\xae\x98\xba\x34\x40\x47\x2b\x11\xa2\x53\x54\x7b\x50\xce\x51\x27\x8e\x65\x91\xc1\x15\x76\x8e\xe3\xa2\xa7\xc5\x61\xfb\xe0\x6b\x9a\xb1\x5a\xb6\x59\xda\x47\xf9\x91\x03\x8a\xed\x1e\x27\x67\x91\x1d\x16\x45\x86\x2d\xa4\x16\x4d\xc1\x81\xdb\xca\x83\x8d\x64\x98\x07\x2c\x9f\x1c\x6b\x5e\x25\xc7\x38\xc4\xa2\xed\x7a\xb4\xd0\x45\x4d\x3b\xcd\x28\x3b\x80\x83\xb6\x59\x9f\xb5\x80\xdb\x2b\xba\x74\x33\x4e\x4a\xa5\xee\x1a\xf7\x57\x28\x95\x57\xd9\xd9\x4e\x9d\x54\xd5\x42\x24\x1c\xc9\x41\xb6\xff\xd7\x98\xc7\x2d\xbc\x7a\x93\xbd\xb9\x54\xd2\x1c\x78\x74\x5b\xd7\x75\xb5\x90\xff\x6c\xdd\x57\x4b\xf7\xe1\x8e\x90\xb3\x9a\x4b\x91\x50\x2a\x84\x46\xe0\xa5\xc4\x94\xd9\x02\x52\x24\x73\xa3\xb9\x82\xed\xfa\x1c\x67\x25\x15\xf1\x90\x3f\x17\x74\xe2\x29\x91\x12\xd7\x60\xc1\x78\x96\xbc\x6d\x18\x43\x76\x42\xed\x8b\x0d\xf7\x93\x3b\x93\x0f\xfa\xad\x65\x32\xfa\x9d\xaf\xd0\xbb\x89\x6a\x19\x36\x76\x41\xcc\xe8\xc6\xa4\x32\x6a\xfe\x72\x92\xc7\x00\x71\xe1\x85\x46\xd5\x79\xad\x1a\x5a\x03\xcb\x04\xab\xac\x0b\xe6\x7b\x1e\xd7\x0f\x1d\xf2\x5a\xdf\x18\x6c\xdc\xba\x83\x70\x91\x3e\x6b\xec\xba\x82\x9b\xd6\xaf\xd8\x1e\xf1\xe9\xd4\xc8\x05\x05\x13\xf0\x09\xc2\x26\x83\x34\x97\xa6\x64\x1a\x4d\x8c\x73\x4c\x5f\x19\x27\x7d\xf6\xe2\x2d\x48\x21\xc6\xba\xe3\xb7\x04\x40\xad\x5f\xf4\x71\xa3\xcf\x85\xc8\x75\x10\x19\x29\x84\x51\x7b\x1b\xc1\xf8\x31\xf3\x82\x67\x2b\x36\xb8\xb7\x5e\x5b\x6c\xf5\x0d\x6a\x6b\x96\xe8\x5b\x57\x61\x33\x9b\x9b\x57\xd8\x72\x0b\x05\xbf\xe9\x10\xbc\xda\x9a\x55\x23\xd8\x41\x69\xcd\x83\x22\xa7\x67\x99\xb0\x26\x71\x7a\x3a\xb4\xf0\x1f\x7a\xba\x8e\xda\x7f\x72\x33\xfa\x93\x51\x84\xd3\x1a\xbc\x18\xaf\xb5\x0c\x3c\x1d\x9d\x6a\x99\xd5\xe1\x18\xd8\xf8\x11\x89\x4e\x0d\x20\xe3\x7b\x08\x62\xed\xdc\xf6\xde\x9f\xa0\xf7\x7e\x22\xdf\xfb\xa3\x31\x5d\xe4\xc7\xdf\x52\xcd\x16\xf7\x04\x5d\xae\x90\x4b\x52\x3c\xf4\xb0\x53\x4a\x7c\xc0\xee\x18\xcb\x06\x50\x1d\x17\xd0\x6f\x16\xde\x01\x8f\x3e\x94\x8b\x34\x4e\xef\x2c\x92\xdd\xfa\xcd\x8c\x84\x4d\x65\xc5\xca\xc5\x7d\x36\x9c\x97\x3d\x63\x27\x99\x6b\xac\x31\x8f\xd8\xba\x04\xa2\x6e\x6f\x31\xef\x7c\xc7\x11\xd6\xcb\x6f\xa3\x15\x99\xec\x9d\xac\x4e\x4f\xce\x43\xeb\x42\x3b\x42\x6a\x42\xd5\xac\x0a\x66\x3e\x47\xcc\xca\x45\x59\x25\x7d\x1d\xdb\x76\x6b\x10\xcf\xdf\xa9\xa2\x88\x4d\xe5\xa8\x11\xed\xc3\x2e\xdb\xef\x2e\xdb\x45\x2a\x4b\x71\x70\xeb\x5f\xdf\x17\x46\x8a\xf4\x9a\x71\xe6\x5e\xad\xca\x64\x0c\xbe\x50\x27\x73\x75\x79\xeb\x35\x1d\xe5\x1e\x4d\x56\x7b\xca\xef\x4d\xb4\x85\xf1\x55\x2f\x8e\xb5\x3c\xd4\x8e\xb2\xd5\xa5\x38\x35\x69\xab\x8a\x4e\xc9\x89\x29\x67\x96\x47\x88\xd8\xf3\x0a\xdb\xcd\x04\x76\x4d\x08\x15\xae\x13\x0b\x1e\xb9\x16\xce\xc1\x9a\xba\x40\xdd\x1e\xd9\x22\x3c\xe7\xba\xf3\xb9\x7b\x39\xe8\x5e\x3f\xf4\x1e\x7e\xad\x00\x06\x2d\xfe\xec\xb0\x41\xbd\x07\xee\x7f\xbd\x7f\xe8\x7e\x1e\x7c\xea\x5e\x77\xef\x3a\x0f\x2b\x70\x43\x97\x75\x56\x87\x49\x99\xca\x2a\xe5\x71\x1d\x5c\x4a\x67\x64\xae\xe8\x7d\x11\x3d\xd4\xeb\x84\x92\x1a\x04\x51\x83\xe9\xc0\x62\x22\x50\x4c\x9e\x49\xc2\x67\xb9\x51\xb7\x92\x60\x1e\xb4\x68\x45\xfb\xcb\xe0\x45\xa1\xcd\x32\x8d\xcf\x91\xa9\x8d\xe8\x95\x87\xce\x1a\x04\x91\x0f\x0b\xc2\xde\x2b\x44\xbe\xcc\x12\x1a\x51\xe5\xe5\x7c\x72\x61\x9d\x3b\xc6\xe7\x0a\x21\xbd\x2b\x36\xd7\xce\x42\x78\x76\x6e\x71\xf0\xc3\x0f\x16\x6d\x0d\xd9\x89\xca\xa0\xee\x56\x56\x86\xda\x81\x59\xa1\xc6\xd3\xbe\x80\xc4\xb7\xc1\xe8\xf6\x61\x9c\x58\x4c\x6c\xb2\x79\x9b\x35\x28\x7d\xd5\x83\x5c\x7d\x1b\x2e\x0b\x2e\x2a\x9c\xeb\xe5\xd1\x45\xcd\x76\xea\x2b\xc7\x08\x15\x0a\xd1\xee\x00\x52\xc5\x06\xfc\xaf\x19\xe5\xb1\x50\x08\x88\x99\x40\x5d\x8c\x04\x99\x72\xa5\x15\x30\x13\x46\x71\xa2\x85\x2a\x8a\x13\xfa\x4f\x00\x1f\x13\xe4\xcc\x0b\x3b\x71\x90\x6d\xb9\xf3\xc2\x02\x83\x9c\xf5\xd9\x65\xf7\xf6\xae\x7b\xa1\x19\xd2\x19\x7a\x94\x80\x2b\x56\x98\xfa\xa5\xdd\xde\x46\x1c\xf3\xc3\x3f\x28\x93\x8a\xe0\xba\x08\x3a\x22\x04\x17\xcd\xf9\x43\xd6\x5f\x17\xde\xab\xde\xde\xf0\x5b\xc1\x32\xe6\xcc\x0f\xd7\xb5\x55\xc4\xbd\x44\x8b\x9d\x27\xb2\xdd\xe1\x97\x02\x45\x7c\x5c\x15\x90\x44\x8a\x54\xdf\x23\xb5\x01\x99\xb5\xf9\xfc\x0a\x7d\xde\xc2\xbb\xcb\xe6\xf9\x00\x71\x89\x52\xe5\x30\xaf\x06\x09\x36\x2b\x67\x54\x9a\x67\xad\xa8\x28\x5e\x03\x83\xa5\xb4\xf5\x87\x64\x8c\x19\x12\x29\x63\x25\xdc\x5f\xdf\xce\xb7\x18\x69\xb4\xee\x51\xd5\x34\xc3\x53\x9e\x32\x53\x8f\x57\x8f\xaa\x62\x30\x72\x46\x98\x5a\x31\x98\xd7\x42\xd8\x29\x0d\xb5\xbd\x20\x3b\x15\x03\xad\xc3\xd9\xa9\xf2\x66\x41\xa9\xf2\xf5\xae\x65\x17\xc9\x58\x70\x69\xe9\x43\x95\xdd\xcf\xd5\x5a\x36\x96\x4f\x5b\x77\xf7\x80\xe5\xd3\xea\xae\x62\x12\x3d\xad\x7b\xd9\x94\xd3\x59\x13\x5b\xe9\x7d\xc1\xd8\x37\xd7\xbf\xda\x9a\x3b\x50\xe0\x3f\x7a\x42\x3f\x3c\x7c\xbe\x42\x23\xaa\xe5\x5e\x7d\xad\x5c\x63\x2d\x63\x3f\x8a\xc4\x59\xa5\xad\x65\x37\x15\x49\x76\xf7\xc2\xc2\x3b\x51\xca\x93\x12\xf4\x8d\x86\xc7\xc4\x99\x9a\x85\x85\x51\x2c\xd5\xdc\x11\x98\xc5\x7c\x6a\xe6\xf1\x41\xa6\xa3\x11\xfd\x72\xa6\xb0\xf8\xb6\x86\x1e\x26\xa6\x63\xf0\x0f\x3e\x1c\xe8\x11\x6d\x79\x11\x57\x35\x87\x6c\x01\xf2\x8c\x6c\x76\x66\x97\xe6\xd9\xff\xc3\x87\x00\x11\x00\x28\x07\xce\x37\x68\xe3\x24\xec\x23\x6e\x27\xe5\x15\xb9\x0b\xe8\x35\x11\x17\x82\x58\x64\x01\x53\x34\x76\x86\x85\xa2\x60\xad\x75\xe8\x37\x85\xb2\x07\xf9\x12\xf9\x25\xf2\x27\x38\x87\x18\x1f\x12\x02\xee\xa5\x19\x4d\xd6\x53\x7a\x2f\x0a\x9e\xd1\xd2\x09\xb4\xe1\xba\x16\x10\x15\x0c\x32\x2b\x45\xac\xee\x33\x61\x6a\x27\xfa\x09\x34\x51\x81\x75\xd0\xcc\xc7\x61\x6a\xb7\xf6\x2e\xf3\xcb\xcd\xc5\x41\xfb\x31\x55\x4a\x60\xb8\xe7\x6d\x76\x99\x75\xe8\xd7\x85\x19\x3c\x37\xf6\x5c\xc3\xa3\x8b\x74\x59\x91\x4f\x60\xa9\x9d\x97\xc6\xcf\x63\x81\x5d\xcd\x86\x0d\x11\x9a\x24\x31\x56\x0c\x0f\x19\xc4\x2a\xa7\xe5\x35\x37\x7d\xea\xbd\x55\xea\x72\xe5\x92\x6f\x00\x47\x54\x68\xe6\x13\x81\x3c\xd8\x5d\x44\xef\xaf\x03\x78\x00\x03\x79\x14\x09\xc4\x9d\x2f\xb5\x62\x99\xfa\xf1\x9a\xf3\x65\x92\x1d\x6e\x20\xa3\x9b\xc1\x68\xa1\x91\xcc\x04\x89\xf4\x55\x76\x8e\x6e\x13\xa2\x25\xaf\x54\x4b\x5f\x69\x92\x38\xe8\xb6\xe5\xd2\xe1\x5a\x70\x83\x7b\x9f\x97\xa7\x7b\x2c\x99\x98\x83\x2e\x5c\x3e\x33\x8f\x06\xbb\x87\xa9\xf0\xe8\x0b\x26\x64\x30\x24\x16\xb5\x48\xe0\xf0\x73\x13\xb5\x0b\xa6\x24\x5c\xb8\xc8\xe8\x3f\x35\xfb\x15\x44\x4e\x78\x6d\x66\xa8\x3f\xdb\xfd\xcc\xc1\x91\x72\x8f\x93\x70\xf7\x61\x5d\x30\x7a\x03\xb9\xa6\x74\x07\x16\x44\x9c\x26\xbe\xd8\x3c\xf6\xc4\x02\xe9\xda\xbb\xd5\x0e\x0d\x6e\xc9\xdc\xd4\xe6\x83\xda\xe5\xae\x8b\x5c\x99\x99\x1b\xdf\x6b\xf6\x7a\x6e\x40\xce\xf3\x28\xa8\x92\x79\x39\x41\xa4\xef\xda\xba\x25\xd6\xf3\x1c\xa4\x62\x2d\x1c\x8f\x1c\x8e\x7e\x1d\xce\x6d\x33\x78\xf2\x61\x69\x22\x54\xb3\x4b\x5b\x25\x04\xc4\x68\x1b\xe8\x24\x0b\x10\x7f\x76\xdb\x18\x32\x56\xaa\x78\xf5\x4c\x79\x5b\xb7\x1a\x48\xc9\xb9\x28\xb3\x2f\xef\x5a\x61\x07\x16\x26\x10\x40\xe3\xd6\x07\x8d\xb3\x25\x63\xb2\xbd\x07\x10\x8f\x4a\x00\x5a\x42\xee\x40\x2b\x0b\x0e\xd6\xe8\xbd\x2a\x5d\xac\xb0\x3a\x8d\x72\xc3\x0a\x6f\x68\x5e\x72\xb9\xa5\x07\x4e\x4f\x66\x3e\x80\x6c\xdb\x6d\x62\x80\x0a\xf3\x37\xde\x03\x68\x93\xc4\xc8\x40\x3e\x18\x48\x6b\x4b\xbb\xcc\x73\x32\xc3\x82\x30\xd5\x67\x77\x7a\x14\xe6\x8d\x3c\x12\xc3\x45\x01\xb9\x32\x03\x50\x8c\x78\x84\xb0\x7d\x0b\x88\x5e\x17\x86\x27\x07\xe6\x21\x50\x4d\xf7\x88\x4c\xf0\xbd\x79\xc6\x00\x45\x58\xa0\x24\x3d\x55\x3a\xca\xd5\x78\x2d\x40\x46\x13\x0a\x38\x0d\x31\x91\xf6\x42\xa2\xca\x02\x71\x64\xe2\x77\x4a\x1c\xb0\x36\xbc\x96\xf1\xaf\x2a\x86\xed\x0c\x05\xcc\x19\xe8\x64\x9f\x79\x7d\x2c\xc1\x61\x35\xca\xfa\x86\xaa\x04\xac\x33\x8d\x33\xc7\x17\xfc\x69\x56\x88\x0b\x3a\xa6\xcc\xab\x86\x65\xa7\x37\xc5\x33\x30\xef\x9a\x33\xc8\x47\xd9\x9d\xf6\x60\x73\x1c\xce\x60\xc4\xff\xf7\xbf\xff\xe7\x8c\xd6\x79\x3f\xe4\xc0\x52\xa0\x0d\x2b\xb9\xde\xb2\xf8\x2b\xef\x41\xaf\xd4\x40\x7a\x78\x3a\xad\x2c\xe4\x6d\xe4\xdf\xda\xcb\x4d\x6f\x1a\xae\x26\xc6\xdd\x5b\xdc\xee\xe0\x1b\x11\xe9\x92\xb3\x61\xae\x98\xd7\xa5\x25\x95\x90\x9b\xa0\x47\x62\x4e\x72\x66\x20\xf0\x2b\xcd\x2f\x98\x69\xfa\x2c\x7f\x45\x1a\x10\x19\x83\xdb\x6b\xbe\xc8\xa9\xd3\x90\x30\xcb\x78\x7f\x1e\x29\x91\xbb\xc3\xbd\x58\x68\x57\x17\xc5\xc4\xb0\xea\xf6\x4b\x37\x6d\x89\x73\x7b\x00\x96\xdb\xc4\x8c\x4e\xb0\xdc\x5f\x68\x4e\x65\x3d\x2f\x63\x4d\xf7\x85\x87\x55\x41\x3a\x66\x90\x26\x45\x56\x2f\x48\x2a\x89\x30\x9c\x2e\xc3\x10\xb3\x3b\xc1\x87\xe7\x84\x08\xd1\x15\xbe\x46\x32\xc5\x74\xad\x6c\x06\xfd\x7c\x35\x78\x68\xc1\xd9\x80\xc7\x44\x0c\xe2\x54\x2d\x1c\x8b\x65\x19\x06\xfa\xa5\xcb\x54\xcd\x57\xb7\x2f\x13\xbc\x58\xcf\x68\x19\x60\xab\x7e\xbe\xa6\xd9\xd5\x12\xb3\x17\xe2\x53\x94\x9a\x6b\xe0\x50\x49\x09\x0e\xd5\x46\xbc\x16\x4c\x24\x70\x03\x33\x05\x38\x84\xb9\x26\x65\xaf\x68\x03\xda\x0e\x23\x47\xc3\x34\x37\x29\x65\x65\x30\xe2\xb3\x3e\xfb\x68\xea\xc8\x80\x96\x67\x06\x10\x41\xba\x11\xf9\x32\xe3\x92\x14\xf2\xdf\x2a\x4a\x5b\xd8\xc4\x57\x3b\x8c\x6a\x61\x3d\x7f\x69\x7b\x59\xfd\xd5\x81\x6d\x17\x17\x7c\x71\xca\xd5\x3b\x70\x2b\x71\x30\xa2\x33\xaa\xf7\xce\xa0\xf2\xa4\xed\xaf\xbc\x72\x1e\xd3\x05\xe0\x61\x2a\x99\x9f\xa0\x6c\x7a\xa5\x0d\x91\x90\x67\x02\xe6\x74\x18\xa3\x5f\xc0\xa4\x68\xd7\xab\x61\x27\xab\x0e\x50\x9e\x7c\x0a\x6c\x01\xc5\xe5\x11\x14\x53\xf4\xaa\xf6\x62\x31\xf9\x68\xeb\x3c\xb9\xaa\xc0\x94\x35\xc4\xf3\x8e\x5f\xc8\x65\x4e\x14\x22\x5f\x14\xb1\xa5\x5e\x1f\x5c\x26\xe3\x62\xf2\x03\xaa\x4e\xc6\xaa\x97\x1d\xf7\x5e\x74\xbb\xe3\x12\xdf\x5d\xaa\x66\xec\xae\x7c\x9b\xba\x38\xc1\x2c\xb6\xf9\xb8\x56\xc9\xd0\xc2\x16\xcc\xce\x18\xdd\xb2\x4c\x05\x9b\x55\xea\x21\xe0\x9b\x36\x0d\x54\x3f\x5c\x64\x4e\x61\xd4\x2a\x0b\x84\x57\x70\xa1\x25\xf7\x94\x29\x9a\xe8\xcd\x61\xc7\x20\xd1\x08\x22\xe3\x2c\xba\x23\x44\xb6\xd7\x01\x08\x52\x29\x29\x1b\x0f\x2c\x25\x5d\x6a\x69\xb3\x8b\xa1\xb8\xa7\x3e\x9b\xa6\xcc\x97\xdf\xbb\x86\x96\x1b\xd5\xcd\xb6\x06\x70\x37\x97\xd4\x0a\x1a\x07\xe3\x6e\x32\x16\x95\xcf\xe5\xc2\x0e\x68\x6c\x48\x41\x4d\x45\x71\x98\xe8\x3a\x76\x77\x90\xe9\x16\xc1\x2f\xf2\x2b\x44\xda\x44\x55\x93\x7e\x06\x91\xfa\xaa\x26\x13\x57\xd6\x66\xe0\xf6\x58\x26\xa2\xd9\x7a\x66\x19\xce\x40\x29\x99\x17\xbb\xee\x6c\x3a\x02\x4e\x92\x21\x8e\x9e\x32\x2d\x2c\xb3\x45\x70\xe1\xea\x41\x68\xb9\x12\x0a\xde\x99\xcd\xa5\x07\x1a\x81\x74\xe3\x7b\x0b\x0d\xfc\x91\x1d\x76\xde\xb9\xa1\x9a\xc5\x95\x33\x78\x57\x66\xf4\x26\xb7\x21\x26\xb3\x84\xcf\xa7\x35\xf7\x59\x39\x81\x71\x9b\x48\x9d\xba\xfc\xc9\x9d\x5e\x65\x25\xa6\xb7\xf6\x65\xb6\x90\x0d\xb5\x03\x30\xae\x35\xb8\xe4\xa7\x84\x0f\xc1\xa4\x6a\xcd\x0f\x2e\xc3\xc7\x4b\xf5\x28\x9f\xe7\x75\xf3\x8e\xca\x27\x92\xca\x59\xa2\x95\x99\xfa\x1e\x4c\xce\xc9\x7e\xd7\xcd\x20\x24\xac\xb6\x0e\x36\x8f\xd6\xae\x7c\x7d\x1f\xb0\xcf\x57\x4e\x12\x30\xcf\x1a\xfe\x55\xb2\xb2\x99\x54\xc3\x33\xe3\xa4\x56\xbc\xcf\x14\x1e\xbb\xc5\xb5\xc2\x25\x7f\x61\x44\xc8\x09\x9d\x15\x0a\x61\x6e\x1d\x1e\x6e\x77\xb4\xfd\x8f\x09\x86\xae\x6c\xb3\x85\xa9\x5b\x0f\x7c\x76\x6a\xd0\x59\xf4\xee\x94\x33\x1c\xe5\x36\xd9\x28\xc1\x52\xd2\xd1\xdc\x03\x55\xc9\xe2\x7c\x21\x75\xad\x68\xc4\xf0\x2a\xdf\x55\xb1\x39\x43\x9d\xdd\xa0\x0a\x6c\x9f\x51\xf9\x58\x3c\xfc\x34\xf6\x41\xf7\xf4\x6d\xb6\x08\xbd\xe3\xe4\x04\x4b\xf5\x5a\xf0\x60\x03\x9f\xb0\x19\x0a\x40\x53\xbc\xa6\x3d\xef\xa4\x8a\x34\xcc\x85\x8d\x94\xa3\x85\x65\x72\xb4\xa5\x99\xd5\xe1\x32\xa4\x15\x1f\xbe\x48\x15\x72\x58\x61\xe7\x69\x9d\xd1\x99\xc4\xf5\xb9\xcc\x50\x5a\x00\xcc\x22\x7f\xf9\x04\xc9\xad\x40\xd9\x9a\x6c\xca\x4b\x92\x90\x9d\x04\x9b\x6f\xb0\x43\xcb\x91\x1c\xde\xde\x5c\xba\x2f\xf3\xb2\x14\xab\xed\x2a\x1b\xc4\xc0\xd7\x60\x24\x55\x0f\xfd\x17\x33\x50\x1b\x06\x5f\xb5\x8a\x60\x13\x05\x2a\xaf\x1e\x6d\x9b\x76\xb9\x17\x5a\x62\x86\x6f\xf7\x7b\x3e\xc7\xc2\xa6\xce\x67\x9c\xc9\x89\x6d\xdc\xe7\xaf\x1c\xaa\xbe\x30\xae\x4f\xa4\x49\x58\xcd\xca\xd3\xb7\x11\xef\x5d\xbc\xa1\x9a\xed\x0b\xeb\xb8\x56\x1c\x8d\x09\x20\xf1\x50\x16\xd3\x67\x1a\xa7\x38\x39\xaa\x3d\xb1\xb3\x44\x9b\x1d\x51\xbf\x9a\xc3\x34\xb2\xf4\xe4\xf1\xa0\x44\x49\x77\x1f\x2d\x60\x7e\xda\xc5\x69\xe1\x12\xb4\xe3\x58\x1a\x85\xe1\xcd\x4b\x6c\x5b\x43\x63\xd8\x91\x59\x80\x88\x20\x4a\x16\x2e\xd9\x7c\xec\xbb\x97\x26\x0d\x8d\x63\xfb\x46\x06\x07\x51\x80\x61\xc3\x05\x34\x4b\xb3\x46\xaf\xcf\x75\xcb\x47\xeb\xad\xcb\x9d\xeb\x9f\xb1\xf2\xa8\xf2\xd3\x15\x84\xe1\x36\x9c\xd3\xe6\xf2\xb0\x03\xa0\x6d\xa1\xf0\x53\x77\x0c\xdb\x79\xff\xb6\x40\x38\x5e\x10\x09\x76\x27\x22\x1f\xd1\x36\x69\x85\xa4\xbc\xb0\x14\x87\x92\x97\x4f\x1d\xb6\x57\x8e\x94\xd5\xde\x25\x6a\xc7\x49\xbe\xb3\xee\xc7\xfd\x5d\xf0\xab\xf7\xcb\x4e\xf6\x07\xc0\xdc\x62\xc8\xc7\x4f\x6d\xb9\x1f\x38\xbc\x5e\x0c\xe7\x82\xcf\x6b\x45\x74\xac\x1d\x5e\xa3\xb8\xd8\x05\x72\xee\x63\x79\x6d\xf2\x65\xe3\xc5\xdd\xe7\x56\x5b\x77\x2c\xbb\xd0\xd1\xf6\xec\x3d\xb4\xbb\xd1\x7b\x21\x04\xa9\x37\xbb\x45\x2b\x20\x9d\xdc\x92\xed\xf2\x90\x55\xd5\x68\xdc\x1e\x3e\xc2\xe5\x96\x0e\x66\x82\x8c\xe8\x97\x8d\x54\x81\x5b\x78\xd5\xaa\xd7\x9a\xcc\xa5\xaa\x8f\xe0\x16\x84\x2a\x91\x5e\x20\xad\xa5\xb4\xad\x0c\xd7\x67\x79\x46\xae\x4d\xc7\xd5\xc2\x30\x17\x85\xaf\x36\x85\x3e\xdd\x7d\x85\x4a\xb3\xae\x13\xa5\x66\xf2\xfc\xc3\x87\x31\x55\x93\x74\x78\x16\xf1\xa9\xc9\xff\xe0\x62\x6c\x3e\x7c\xa0\x52\xa6\x44\x7e\xf8\xcb\x9f\xff\x9c\x2f\xf1\x10\x47\x4f\x63\x03\xe7\xb4\xe8\xef\x2c\x2e\x39\xc1\x72\xbb\x88\x32\x97\x3a\xb9\xe7\x14\x7a\xaf\x1b\x97\xb4\xac\xdf\x91\x0a\x4f\x67\x7e\x08\xb2\xa9\xf1\x28\x15\xce\x2b\xcb\x40\x3e\xac\x9e\x26\x9a\xe0\xd9\x8c\xb0\x7a\xb3\x8b\x49\x70\xde\x82\xf5\xb8\x14\x69\x3b\x42\xf2\x65\x96\x60\x56\x84\xfd\x80\x32\x69\x82\x44\x84\x29\x0b\x49\x91\xd7\xa6\x87\xdd\x68\xa0\xa7\x0c\xff\x5f\x2f\x05\x16\xe6\x48\x65\x5e\xff\xd0\x0d\xc7\xd6\x22\x76\x15\x6a\xb1\x47\xba\x72\xfd\xe7\x9c\x76\xc4\x51\x6d\x59\x72\xec\xbd\xad\xf5\xb6\xcd\x0e\x8a\x04\x67\x03\xf2\x45\x33\x39\xb9\x29\x50\xdc\xa3\x24\x12\x75\x7e\xb9\x47\x72\xce\x14\xfe\x72\x8e\x3e\x53\x06\x02\xec\x0f\x3c\x15\x12\x5d\xe2\xf9\x29\x1f\x9d\x4e\x39\x53\x13\xf4\x19\xfe\xdf\x7e\xf5\x42\xc8\x13\xfa\x95\x60\x61\xf9\x83\xad\x1f\xe9\x4a\xd8\xc1\x16\x12\x29\x93\x88\x3c\xeb\x13\xfa\xe7\xff\x85\xa6\xa6\xe5\x73\xf4\xdd\x87\x3f\xff\x2f\xf4\x47\xf8\xdf\xff\x83\xfe\x58\x63\x69\x58\x0f\x6a\x0e\xca\x8c\xdf\xd5\x86\x11\x00\xa5\xe4\x22\xc9\x57\x35\x7b\x21\x78\xbe\x52\x95\x2d\x3f\xd1\xe8\x89\x8f\x46\x03\xbd\x31\x4c\x02\xe9\x00\x6f\x65\x76\xf0\x51\x83\xa9\x2d\x14\x6f\xca\x4e\xe6\x05\x9f\x6c\xa7\x06\x69\xc4\xb1\x6b\x99\xe6\xe6\x09\x08\x5e\x2b\x94\x1e\xa7\x12\xde\x22\xb1\xe6\xaa\xeb\x9c\x0e\x67\x5d\x74\xa0\x03\xce\x82\xe4\x23\xf3\x38\x81\xb8\x10\x70\xea\x47\x4f\x9b\x00\x33\x4b\xc8\xca\xe3\xb0\x10\xd6\xfd\x66\x62\x75\x61\x6a\xaf\x15\xa7\x2b\x17\x3a\x5f\x1d\xa2\x7b\xcf\xc5\x56\xfa\xd6\x13\xa9\x4d\xa1\x59\x51\xdc\xcc\x15\xdc\xc6\xbe\x51\x43\x71\x24\xb9\xc8\xd0\xbb\x8d\x5d\xc4\x96\x40\x5d\x6d\x45\xa5\xc2\x04\x35\x36\x3b\xf4\x7a\xea\x97\xd9\x2b\xab\x86\x09\x11\x8e\xee\xe9\xbc\xb8\x23\x8c\x56\x8b\x48\x9a\x25\x56\x8c\xb8\x02\x64\x73\xd5\x82\xde\x67\xb8\x2a\xd0\x38\x84\xdb\x42\xde\x10\x73\x92\xad\x05\xae\xa8\x5e\xcf\x54\x44\xe4\x82\x6f\x17\x6e\x9d\x50\xb6\x90\xa7\x51\x1b\xdc\x56\x2f\x93\x5f\xd9\x0a\x71\x0e\x87\x9a\xc7\xb9\xb2\x60\xdc\x12\xb6\xf6\x8a\x07\x80\x5b\x9c\x0d\x00\x29\xee\x02\x63\x75\xa1\x22\xc8\x16\x5c\xdb\x18\xae\x73\x86\xe7\x0a\xca\x94\xea\xc8\x08\xac\x79\xe1\x92\x98\x49\x08\x27\xdb\x7a\x1c\x5e\x6d\xa4\x3c\x46\xad\x50\xa5\x18\x46\x02\xf9\x96\x1b\x62\xe4\x9a\x32\x65\x27\x48\x60\x08\x06\x56\x13\xdd\x9e\x24\xe2\x74\x84\x23\xca\xc6\x27\x1e\x3c\x2a\x40\x95\xf8\xd7\x41\xd5\x26\x7d\xc0\xf2\x69\xb7\x01\xae\x5b\x57\x9b\xa5\x71\x5e\xf1\xd0\x02\x1a\x19\xc7\x0a\x5d\xc0\x86\x54\x58\x3e\xd5\x21\x7a\x2d\xc0\x09\x2e\x19\x5d\x46\x0a\x07\x42\xb8\x6c\x7c\x0e\xfa\x80\xf8\xfa\x14\x54\x2a\x71\xf5\xcf\x2d\xb8\xa8\xcb\x34\xc5\x19\xfa\x4f\x19\x55\x77\xc9\xf8\xe5\x84\x0b\x35\xd8\x10\x8f\xb8\xec\x52\x61\xe4\x34\x01\x20\x21\xfe\x4c\xc4\x33\x25\x2f\x45\x58\xdf\x75\xf6\xa2\x31\x9a\x79\xf1\x94\x80\xfb\x3a\x9d\x71\x48\xdd\x1a\xa1\x29\x66\x73\xc3\x28\x35\x73\xc1\xf2\x49\x66\x55\x97\x91\x9c\xe2\x24\x39\x41\x82\xa4\xd2\x54\x23\x97\x24\x19\x9d\xba\x02\x30\x31\x4a\xf8\x98\x46\x38\x41\xc3\x84\x47\x4f\xd2\x64\x56\xb2\xb1\x61\x52\x33\xc1\x23\x22\xa5\x27\x59\xe5\x28\x0a\x36\xb7\x15\x4a\x2e\x2b\x22\xa6\x94\x51\xa9\x68\xe4\x44\xa6\x1c\x0c\xc5\x14\xfe\x8f\x30\x98\x84\x21\x53\x18\x86\xab\x25\x3d\x62\x40\x61\x53\x66\x4b\x85\xc1\x75\x6d\xb1\x1e\x5d\x72\x42\xdd\x01\xda\x01\x74\xa5\xdb\x21\x03\x55\x3c\x90\x2b\x8e\xd4\x85\x7d\x0d\x8e\xf1\xb2\x2d\x70\x57\x3c\x51\xd9\x86\xcc\x4e\x5a\x01\x4e\x0b\x72\x19\xb2\xd4\x8b\x82\xe4\x92\x65\x24\xb4\x0c\x49\x0f\x86\x5c\x83\x9f\xb7\x6a\x4f\x6b\x2a\x82\xc8\x03\xd5\xe9\xca\x5e\x7b\xca\xa2\x24\x8d\xb3\xb2\xaa\x5a\x04\x78\xd6\x9b\xc4\x91\x47\xd3\x5e\x0b\x0a\x27\x08\x4b\xf4\x42\x92\x44\xff\xd7\x64\x5e\x9c\x66\xe5\x42\x34\x4b\x36\x25\x5d\xa0\x13\xc7\xa5\xeb\x76\x54\xeb\x50\x51\x6f\xb1\x9a\x18\xac\x89\x29\x57\xa6\xa2\xad\x41\x45\x75\xf6\x2d\x03\xa3\x39\x4c\xf8\x10\x4e\x3a\x00\xa6\xba\xfc\x6a\x2f\x9d\x33\x8d\x22\x42\x62\x12\x9b\xfa\x9c\x19\x98\xa7\x3d\xa2\xdf\x56\xc3\x77\x16\x28\xd2\x02\xb0\xd4\xb2\x61\xad\x16\x32\xb5\x58\xdd\xf0\x0c\xdd\x96\x00\x81\x3c\xca\x8c\x70\x19\x1e\xee\x64\x61\x09\x5f\x07\x60\xb5\x34\x89\xfd\xad\xd0\x9a\x00\xab\x85\x3e\x77\x00\xb0\x5a\x9a\x67\x4d\xce\x08\x1f\xef\x35\xd7\x5d\x4f\xea\x8a\x37\x4f\x40\x34\xc0\x74\xe6\xee\x2c\x6c\x41\x77\x20\xe7\x55\x1b\xb1\x5d\xe0\xb1\xa5\x1a\xa0\xaf\x0b\x1e\x5b\x1a\x4c\x9b\xc1\x63\x4b\x43\x6d\x2f\x78\x6c\xc5\x40\x1b\x80\xc7\x1a\xe7\xfe\x40\x6f\xea\x66\x4c\x01\x12\xaa\x86\xe9\xe8\x1e\x20\x06\x96\x8e\xf1\xc2\x04\x0e\x98\x6b\xcc\xdd\xd1\x36\xbe\x08\x46\x6b\x73\x6f\xeb\xc2\xb1\x4a\x4e\x88\x75\xf7\x5e\xe6\x7d\x33\xa0\x23\xeb\x9a\xdd\x4f\x7c\x6b\x37\xd8\x21\x23\x3c\xb3\x58\x06\x75\x25\x8e\xda\x93\xb5\xbd\x19\x2e\x2f\x60\x5f\x16\x58\x7e\x23\xe4\xba\xcf\xa5\x6a\x21\x13\xfe\x62\x2b\x76\xc1\x36\x34\x9b\xb2\x76\x0b\x42\xa7\x03\xab\xb4\xd5\x51\x8e\x32\x45\xc6\x65\x9d\x36\x3f\x34\x94\xa9\xbf\xfe\x65\x25\x27\x32\xd0\x9e\x4e\x3d\xf4\x6a\x76\x64\xce\x0e\xfb\x1b\x89\x51\x34\xd1\x5a\x91\xd4\xea\x8b\x9e\x8e\xb9\x59\x25\x9a\x62\xea\x14\xa9\x54\x1a\xd7\x12\x95\x7d\x56\xc0\xc2\x3d\x43\x1f\xa1\x0c\x32\x9e\xce\xb4\xfe\x95\xcd\x8f\xea\x9d\xd4\x4f\xbf\xfb\xee\xaf\x04\x7d\x87\xa6\x04\xb3\x82\x0e\x0b\x6a\x93\xbe\xfa\x00\x3b\x52\x4d\x48\x9f\x55\x2e\x05\xea\x7e\x31\xb5\xcd\x5c\xbc\x61\x8f\x8d\xb8\xd3\x89\xa1\xbc\x27\x8e\x26\x48\xa6\x43\x53\x9f\xda\xb3\x61\x38\x41\xfa\x8a\x8f\xc1\x51\x0d\x37\xb2\x1b\xf4\xb2\x53\xb8\xdf\x18\x00\xeb\x6e\x6c\x7a\x1b\x77\xe0\x1e\x39\x95\xa4\x80\x29\x56\xe1\x34\x33\x9c\xcf\x3f\xf8\xd2\xe0\x0d\x9d\x18\x1f\x82\xd6\xcf\xb0\xb5\xec\x6b\x59\x1a\xc2\x89\xc1\x4b\x96\x26\x58\xd8\xa3\xdf\x67\x5a\xd1\x10\xe4\x99\xf2\x54\x26\x73\x14\x73\x46\x4e\x60\x27\xa4\xd1\xc4\x38\x56\xb5\xce\x82\x6d\xa1\x94\x67\x2a\x53\xad\xd0\x42\x5b\xae\x2e\x8b\x54\xd8\x60\xa1\x4d\x28\xf4\xa3\xd5\x6f\x02\x6f\x29\x2f\x3f\x12\x35\xd3\xa2\x7c\xb8\xe2\x12\xcf\x6f\x08\x57\x5c\xd8\x55\x01\xae\x38\x83\x2b\x5e\xa4\x4b\x1b\xe1\x8a\x4b\x6b\xde\x0c\xae\xb8\x6a\xc9\x37\x80\x2b\x2e\x34\xf3\x66\xe0\x8a\x4b\x14\x7d\x33\x70\xc5\xa5\x79\x05\xb8\xe2\xb7\x07\x57\xbc\x25\x20\x6f\x35\x2f\x36\xb8\x5e\x8a\xb2\xf9\xda\x9b\xec\xbd\x44\xbd\x1b\xbd\xc1\xa2\xa7\x62\x50\x5b\x76\x5d\x6d\x0f\x02\x5c\xcd\x84\xd6\x03\x01\xae\x54\xd5\xeb\x59\xdd\xb6\xc0\x62\xa0\x18\x1c\x18\x04\xb8\x30\x81\x10\x5f\xb9\x7e\x7c\x65\xe5\xe6\xb3\x7d\xeb\xe1\xb9\xa0\xcb\xf2\x85\xdc\x10\x06\xb8\xb0\x3e\x8d\x22\x31\x41\x74\xdf\xc1\x4e\xdc\xaf\x34\xff\x50\x38\xe4\x2b\x65\x79\x9f\x8a\xd2\x02\x92\x6b\x09\xcf\xa1\x14\x1a\x25\xdc\xf7\xff\x87\x9d\xbb\x41\x64\x70\x89\xbc\x99\x5f\xc5\xec\xc5\x06\x5b\xb5\xf1\x0e\x75\x5a\xe9\x6e\x12\x85\x5d\xf2\xe6\x9a\x2e\x66\x37\x88\xfb\x19\x89\x6a\x6c\xcc\x74\x4a\x77\xd5\xec\xaa\x8b\x2c\xc3\x60\x03\x85\x7c\x21\x2f\x55\x5f\x4f\x66\x38\x46\xc6\x2f\xa5\x03\x03\x4a\x8a\x79\x73\x4c\xa5\x12\xb5\xb1\x4d\x0b\x23\xdc\xc6\x55\x3a\x4b\x1b\x07\xc4\x78\x54\x1d\x6f\xf6\xda\x94\x4c\xb9\x58\x15\x58\x55\xf9\xa6\x2d\xb1\xb4\xc9\xab\x64\x36\x21\x53\x2d\xc9\x0c\xd6\x6d\xa4\xe9\x7a\x67\x49\xcb\x36\x77\xcd\x04\x3a\x16\x36\x81\xe7\x08\xd5\xcf\xc6\x06\x09\xb5\xf1\x72\x6f\xbb\xcc\x16\xab\x75\x4d\x87\x90\x03\xf1\x5e\x6e\x70\xb3\x0f\x15\xdc\xdd\xb0\xbf\x2b\x63\x3a\xb2\x90\x9a\xd5\x51\x1b\x4b\xe2\x35\x96\xe1\x9d\xe5\x6f\xd9\x02\xe4\x6b\xb8\xf2\x8b\xde\x79\xcd\x09\xfd\xea\xd3\xeb\x07\x78\xd4\xa0\xf5\x2e\x92\x07\x22\x73\x24\x11\xa7\xbe\x66\x50\x18\xcc\x22\xbd\x0a\xbb\xc4\x69\x94\x5b\x6c\x92\x54\xd4\x46\x99\x36\x31\x68\x47\x2a\xc5\x09\x68\x12\x7e\xd5\xd4\xf2\xa2\x0e\xe7\x15\x69\x8f\xcd\x3c\x26\x94\xa9\xff\xfc\xdb\x5a\xab\xa9\x55\x2b\x4b\x37\xa8\xf4\x86\xa3\x88\x48\x63\x63\xb7\x51\xc8\x78\xc8\x9f\xa1\xc8\xdb\x36\xab\xaa\x8f\xb2\x9e\xb7\x66\xf0\x19\x04\x76\x9c\x6f\x75\x23\x2e\x4c\x04\x4f\xc7\x13\x67\x43\xd2\x67\x46\x4f\xad\x6a\x2d\x7f\x5e\xb0\x91\xaf\xbd\x96\xdf\xa7\x34\xd9\xcc\x42\x77\x5f\x28\x7f\xf7\xa9\xf7\x80\xe4\x24\x3b\xad\x43\x68\xb6\x72\x61\x17\x07\xdd\xbc\x4f\xfb\x6e\xe6\xaf\x81\x6e\x4e\x1c\xec\xeb\x88\x27\x09\x78\x1a\x24\x99\x3e\x13\x51\xdd\x3d\x4c\xf8\x81\xae\x87\xd8\x98\x0d\x00\xde\xce\x13\x23\x1a\xc9\x5f\xb7\x46\x34\x94\xc8\x8d\xbe\x1c\xb4\x60\x42\xd5\x38\x23\xac\xca\xc6\xf6\xcb\x62\xe5\xa1\x23\x0b\x18\x74\xd1\x63\x3b\x0b\x1a\x74\x24\x39\x70\xe0\xe0\x8a\x79\xb4\x35\x78\xb0\xc4\xec\xb2\x58\xbe\xfc\x9a\x71\x81\x43\x46\xf1\xe9\x68\x12\xf7\x59\xa7\x90\x4f\xe1\x2a\xb4\x0f\xe7\x79\x40\xb6\xd1\x21\x7c\x66\x06\xf5\x5d\xac\x61\x05\xdc\x68\xfa\x13\x68\x3a\x06\x34\xd9\x84\x14\xba\xb0\x41\x88\x26\x27\xf1\x29\x8e\xe6\x51\x42\x23\x4f\x67\x1e\x0b\x3c\x9b\x54\x71\x3c\xb7\xf2\x01\x75\xe8\xb5\x50\x87\xea\x0a\xa1\xad\x13\xb7\xed\xf6\x15\xc3\x53\x12\xd0\x90\xda\x88\x86\x74\x92\xe1\x6d\xb0\xbc\xa4\xdc\x2b\xc2\x38\x2c\x9e\xfb\x00\x89\xf4\x0a\x90\x48\x9b\x1c\xfe\x1c\xef\xa8\x70\xec\x03\x4c\x53\x13\xe2\xbd\x3e\x4c\x53\x26\x04\xb4\x0a\x79\xa7\x9e\x1f\xbc\x32\xa2\xcb\xe2\xc0\x5e\x13\x96\xa9\x42\x5c\x5a\x47\x6e\x5c\x86\xcb\xb4\x6c\x5f\x34\xa2\xcb\xeb\xa2\x24\xad\x47\x99\xb5\x00\x90\x2a\xef\xce\x96\xc0\x21\xd5\x2f\x43\x4b\xce\xcd\x2e\xb3\x7a\xd6\xab\xd9\xeb\x67\xf6\xac\xa3\x60\xae\x97\xe4\x93\xed\x87\xe3\x4a\xf4\xc9\x8b\x1b\x6e\x96\xec\xd3\x71\x3e\x78\x22\xd0\x84\x27\xb1\x03\xe1\xc8\xa8\x95\x75\x90\x65\x42\x64\x04\x72\x8b\x71\x3f\x23\x91\xd1\x36\xf3\x42\x7c\xcb\x52\x7a\xb2\x45\x84\xe1\xee\x80\xd1\xec\xc2\x8a\x92\x71\x92\x4d\xec\x27\x2b\xa5\x0b\x59\x34\xff\x2f\x19\x63\x81\x42\xe0\x35\xa8\x1e\xe6\x4a\xbb\xf7\x8a\xc1\x2d\x13\x3d\x3c\xe3\xa8\xa8\x2a\xb1\x6b\xf6\x19\xfc\xfa\x4c\x9d\x21\x06\xfb\x3d\x2e\xf5\x52\xba\xd9\x35\xf2\x54\x96\x37\xcb\x06\xc1\x70\x0b\x15\x13\xb7\x07\x47\x9a\xe2\x2f\x83\x19\x16\x38\x49\x48\x42\xe5\x74\x6f\xc1\xd0\x17\x45\x77\xb5\x3e\xab\x82\x1b\x13\x19\x4b\xa7\x43\xb3\x15\xdd\x40\x6c\x91\x4d\xc5\x91\x48\x99\x0f\xed\x96\x2d\x4c\x56\xc4\x33\x85\x7b\x01\xac\x6a\xd1\x04\xaa\x25\x8f\x30\x15\x8c\xc8\xda\xda\xb4\x24\x4a\x05\x55\xf3\x81\x2d\xf5\xdb\xfc\xc0\xdd\xdb\x37\x2f\xec\x8b\xcb\x3d\xfc\x0e\xd5\xc0\xf5\x97\x95\x16\x9e\x11\x01\xe5\xb9\x5c\xa1\x29\xaf\x9c\xb1\x45\xad\x20\x59\x8d\x2f\x08\xff\x5e\xb8\xb6\xeb\x02\xa7\xf1\xcb\xc0\xcb\x28\x1b\x44\xe5\xcd\xb1\xea\xb0\x56\xe1\x6e\x2d\x9b\xe4\x9e\x91\xa7\x6a\xbc\xe8\x7b\xa8\xee\x63\xd3\x46\x4c\xd3\x7a\xc0\x9e\x2b\x1c\xec\xb5\xf9\xc2\x78\x29\xff\x15\xc5\x6e\xbc\x71\x5a\xac\xa3\xaa\xe0\xab\x25\x83\xed\x78\x6f\x35\x18\xb1\xd7\xc9\x8e\x86\xad\x0f\xba\x10\xe9\x4c\xd1\xe1\x22\xb4\x8f\xe3\x06\x3b\x28\xdd\xdb\x49\x20\xcd\xdc\xb9\x59\x0a\xdd\x9a\x7a\xbe\x05\x4e\x6c\x67\xa7\xe5\x7f\x8b\xa3\xe6\x10\x92\x0c\xc2\x94\x9f\xc7\x78\x33\xa5\x4a\xb9\x44\x09\x63\x80\xd7\xbb\xb3\x68\x9b\x7e\xef\xc2\x5d\x30\x54\x58\x36\x26\xaa\xb3\x3e\xeb\x48\xf4\x42\x10\x23\x16\x42\xa3\xa2\x76\x72\x66\xd5\x87\x9a\x6b\x43\xa2\x7b\xca\x62\x73\xb4\xf0\x40\x95\xcc\xca\xfe\x99\x3e\x46\x38\x91\xe4\x44\x37\x0c\xd5\x82\x15\x87\xe0\x57\x8c\x5e\x04\x9e\xcd\x88\xe8\x33\x9b\xc5\x02\x0e\x27\xce\x13\xd3\x7e\x5d\x88\xaf\xa5\x01\x19\x44\x38\x9a\x1c\x68\x8d\x30\x24\x23\x45\x13\x12\xbb\x7c\xe9\xe2\xf2\xb8\x79\x1b\x83\xfd\x1a\x8b\xd5\x1b\xb9\xb2\x75\x27\xb6\x93\x24\xd2\x1c\x25\x2b\xef\x3e\x23\x42\x8f\x5a\xef\xe1\x67\xc2\x10\x1d\xb9\x71\xd8\xd8\x25\xf4\x02\x9e\x39\xbd\xf5\x9f\x31\x4d\x0c\x00\x81\xeb\xda\x09\x81\xc6\xfd\xd0\x67\xc6\xdd\xcf\xa2\x42\x86\x2e\x65\x54\x4e\x34\xa7\x4e\xc1\x27\x0b\x6a\x46\x5d\xe2\x10\x7b\x5e\xe7\x34\x77\xf5\xe3\xcb\x39\xe8\x33\x15\x9c\x4d\x21\x49\xc8\xe2\x52\x39\xf2\x49\xa2\xb2\xe3\x51\x99\xe2\xb9\x52\x22\x8e\x63\x59\x34\xbe\x1a\xb5\x92\xfe\xb3\x60\x76\x39\x2d\x64\x45\x46\x1e\xac\x12\x04\xb1\xba\x8a\x7e\xcb\xe4\xdf\x90\xda\xb1\x98\xda\x51\x4d\x9b\x36\xa6\x77\x64\x87\x78\xdd\x14\x8f\xba\xe5\xdf\x85\x64\xbb\xc3\x54\x8f\x57\xce\x89\xd8\x4f\x3a\xc4\xeb\xe6\xaf\xec\x23\x75\x25\x24\x78\xbc\x62\x82\x47\x63\x4b\x6d\x31\x36\xbd\xfe\xd8\xae\x95\x1c\xb1\x02\xcc\xaa\xaa\x97\xcf\x44\x09\x1a\xc9\x5d\xf0\x07\x39\xc3\x0d\xa3\xfa\x40\x0b\x9c\xad\x90\x9a\xf4\x03\x99\x13\x14\xe2\xe4\xb2\x0a\x97\x43\x41\xf0\x53\xcc\x5f\x16\x6c\x75\xd2\x47\x13\xf9\xcc\xb5\xd8\x23\x48\x44\x25\x29\x44\xf2\x50\x89\x18\x91\xd6\xd8\x89\xfb\x6c\x42\x89\xc0\x22\x9a\x40\x76\x67\xbe\x30\x26\x4b\xd8\x00\x3a\x99\x58\x0e\xdf\xdb\xb5\xc6\xa2\x37\xa0\x7b\xd9\xc2\x94\xe1\xf3\xd9\x35\xd7\x23\x99\x9a\x57\x32\x61\xc6\x4a\x19\xbe\x49\xae\xd1\xf2\x6f\x9b\x88\x90\x11\x7b\xaf\xc9\x08\x59\x30\x95\xf7\x46\xc3\x84\x84\x7c\x37\x84\xa4\x84\x3d\x25\x25\x54\x90\x78\xbd\xc4\x84\x8d\x4c\x7e\x87\x8f\x99\x76\x3d\x1f\x22\x6e\x7a\x55\xd0\x5a\x3a\x1c\xec\xfd\xe8\x55\xce\xb9\xe9\x09\xfc\x25\xdb\x14\x46\x22\x16\x7a\x9f\x0d\x49\x1c\x03\xa7\x55\xdc\x56\x68\xcf\xf7\x8e\x33\x0f\xe8\xbb\x17\x4b\xbd\xd9\x71\xc2\xd9\x58\xd2\xd8\x80\xcd\xcc\x30\xd4\x2a\xf6\x8d\x17\x00\xae\x00\xeb\x9b\x24\x44\x38\xaf\x84\x40\xdf\x48\xca\x2c\x9a\x64\xf6\x5d\xcc\x89\x64\xef\x95\x31\x16\x60\x36\x47\x4f\x8c\xbf\x24\x24\x1e\xc3\x0a\x95\x07\x73\x8a\x28\x39\x41\x54\x65\xaf\x09\x40\x63\xe0\xa9\xea\xeb\xb1\x43\xac\x9d\xd1\x00\x88\x7d\x57\xd8\xea\x15\x1e\x07\x96\xdf\x9e\x21\xd4\x63\x68\x84\x23\x75\x82\x64\x3a\xcc\xdb\x8f\xb9\x29\x2e\xaf\xb5\x6f\x6f\xe2\x79\x23\x21\x66\xbe\xa2\xf3\xea\xb3\xe1\xb8\x83\xde\xae\x9d\x84\xe2\xad\x62\x0b\x9f\xf1\x36\x10\xab\x9f\x53\x69\x83\x30\x10\x67\xd9\xd1\xb7\xf0\x52\x19\x46\x36\xe0\x9d\x1a\xbc\x69\xc6\xe3\x5a\x5b\x67\x69\x2a\xeb\x8e\x25\x0f\x04\xb5\x82\x92\x75\x54\x41\xbb\x86\xdc\x5a\x6a\x92\x4a\x10\x3c\xb5\xce\x01\x7d\xd5\x80\x58\x63\xc2\x40\xf5\xe8\xa9\x30\x12\xe6\x3a\x4b\x7c\x45\xd9\x93\x5e\xdd\x1c\x15\x9c\x03\x5e\xb2\xee\xb9\x6a\xd1\x66\xfa\xc6\x23\x17\x9c\x19\x07\xe1\x56\x72\x27\x1d\x33\x9c\xac\x69\xe3\x58\xa0\xdc\xa2\x4f\xcf\xc9\x59\x56\x5c\xd0\x52\x84\x31\xf6\x21\xd3\xe3\x5a\x36\xa4\xd2\x7c\x7d\x79\x0f\xa3\x98\xcc\x08\x8b\x09\x8b\xe6\xb0\x45\x18\x20\x07\x09\x86\x13\x84\xe1\x3d\x9c\x9c\xa1\x4b\x93\x5f\x94\x49\x78\xf6\x5a\x87\x0b\x7d\x8a\x19\x1d\x69\x3d\x01\x8c\xb0\x76\x94\x7d\x66\x86\xe9\x7c\x20\x24\xb7\xae\x66\x14\xab\x5a\x19\x7d\x83\x5c\x6f\x89\xca\xcc\x8a\xef\xa3\xe5\x17\x0e\xf4\xb6\x6c\x75\x74\x73\xae\x06\x83\x4c\x87\xa7\xf0\x77\x21\xe1\xce\x01\x15\xe5\x28\x3a\x24\x21\x60\x0e\xb4\x1e\x2f\xb8\x18\xeb\x80\xf5\x76\xe1\xb7\x5b\x91\xc7\xe2\xf5\x51\x50\x6a\xa6\x94\xd1\x69\x3a\xf5\x9c\x77\xa6\x62\x43\x64\xed\x97\x26\x13\x65\xa6\xf5\x80\xc8\x81\xb7\x23\x7d\xb9\xb2\x39\x1a\xd3\x67\xc2\xfa\x6c\xc6\x29\x53\x67\xe8\x9a\x2b\xe2\x95\xc8\x30\xd0\x59\x7c\xa6\xe8\xd4\xa0\xbd\x0a\xa2\xcf\x81\x01\x05\x07\xa0\xcd\x09\x56\x27\x28\x4e\xe1\xa8\x32\xa2\x34\xeb\xd0\x37\xae\x82\x95\x81\xf8\x70\xd1\x67\xe6\xa6\x1b\x61\x9a\xa4\x82\x58\x99\x15\x9b\xbc\xa0\x7c\xc8\xf9\xc8\x2c\x12\x9c\x37\x89\x29\x1d\x4f\x94\x5e\x22\x2d\xe3\x59\x7f\xe3\x44\x73\x23\xde\x67\x43\x82\x30\x9a\x71\x49\x15\x7d\xce\xfc\x97\x74\x84\xb0\x94\x60\x41\x39\x43\x97\x05\xfb\x3f\x95\xa0\x7a\xd7\xc5\x15\x53\x36\xb0\xb6\xe7\xfa\x7c\xa4\xad\x17\xb2\xd0\x8b\xa5\x32\x1e\x4a\x9e\xa4\xca\x77\xc1\x56\xaf\x6d\x6e\x1a\x77\x85\x0b\xc0\x40\xcc\x47\x7d\xe6\xf6\xb5\x3c\x43\x1d\x89\x24\xd7\xab\x24\xcd\x52\x46\x82\x2a\x22\xa8\x41\xb1\x22\xca\x2c\x42\x76\x4e\xb3\x33\x30\xc5\xe2\x49\x8b\x50\xbe\x05\xde\x60\xaa\x16\xac\x1d\x43\x23\x21\x01\xac\x97\xbf\x1c\x60\xfa\x47\x8c\xb3\x53\x46\xc6\x78\xd5\x8a\xf4\x59\x61\x49\xd0\x37\x74\x94\x2b\xa4\x75\x3e\x47\x8f\x76\x03\x88\x7c\xaa\x5b\x25\xd3\x71\xdd\x22\x8d\x12\x8e\x57\xb8\x8d\x47\xf9\xa1\x47\xff\xe0\x43\x33\x46\xad\xf7\x73\x05\x52\xa0\x56\xaf\x46\x5c\x90\x09\x66\xf1\x89\x5b\xac\xe2\xd8\xe0\x66\xb4\xa6\x36\xa7\x8c\x81\x24\xe8\x40\x94\x89\xc1\xa2\xc2\xcc\x5b\x0b\xab\xb8\xd9\xa5\xc8\xd7\x61\xad\xbb\x22\x6b\x0d\x6a\xbf\x18\x03\x84\x61\x79\xf3\x99\x3d\xe2\x92\x4e\x67\x49\x9e\xd3\xe5\xd9\x46\x47\x5a\xc4\x72\x3c\x92\x3f\x83\xe9\xca\x69\x6d\x70\xab\xdb\x95\xd3\xfb\xac\x62\xe4\x19\x23\x85\x5b\xc3\xd9\xbc\x4c\x19\x50\x8f\x85\x7d\x23\x89\xfe\x53\x91\x5c\xed\x33\xc2\x7a\x9f\x39\x11\xe4\x5b\xe0\x32\xb6\x59\xcf\x78\xa6\x45\x68\x03\x73\x6b\xe9\x87\x22\xe3\xe4\x2e\x9c\x13\x7b\x18\xdc\xa3\x95\x17\x95\xa2\x5a\xcc\xfe\x9e\x02\x42\xd7\xe5\x96\xb0\xfb\x94\xc5\xa4\xb6\x98\x55\x23\xae\x51\x77\xb7\x18\x86\x3a\xd8\xb4\xfe\xc4\xc3\x84\x48\x82\xd4\x4b\x06\x14\xa7\xf5\x2a\x30\x59\x0a\x92\x90\x67\x9c\xdf\x71\x59\x5f\x96\x5d\x46\x58\xd6\x94\x87\x01\xb4\x35\x3d\xfe\xcd\x13\xa7\xb3\xf1\xdd\xe8\xa1\x3c\xe3\xc4\x26\xae\x58\x5f\xb9\xac\x5f\xb0\xde\xe5\x56\x31\xa4\xb6\x95\x2a\x7a\xd6\x8b\x18\xae\xef\x1f\xc9\xbc\x9a\x22\x2b\x40\x0c\x97\x65\xa3\x67\x64\x5f\xc3\x56\x7d\x9b\xbf\xb3\x48\xe3\x5a\xca\xfd\x58\x98\xf2\x2b\x24\x51\xdd\x2e\x54\xe0\x86\x8f\x32\x1d\x8d\xe8\x17\xd0\x6a\xdd\x4d\xe2\x34\x8f\x48\x70\xa9\xb9\x18\xc8\x2a\xc8\x2d\x9e\x71\x24\x6f\x93\x50\x55\xf9\xa6\xd6\xb2\xd6\xde\xd1\xb5\xd4\xfe\x29\x25\x62\x2b\x7a\x67\x5b\x75\x9d\x70\x44\xef\x94\x54\xeb\x88\xae\x51\x85\x1b\xc6\x24\xf9\xad\x3e\xe0\x1a\xd2\xad\x86\x3f\xaf\x7c\x6d\x68\x98\xef\xfa\x03\xf1\xb9\xf6\x52\x9b\x5a\x1e\x9f\x96\xc5\x26\xbb\xa2\x44\x9a\xbf\x9d\x58\x80\x75\x6c\x03\xa7\xb2\x54\x67\xec\x62\x46\x8c\x63\xc4\xd4\xac\x52\xb6\xd2\x82\xa7\xa9\xdb\xc6\x28\x1b\xf7\x99\xa3\xad\x3c\x41\x26\x4c\xbc\xc4\x50\x0b\xd8\xf6\xd8\x7b\x35\xdb\xd8\xcd\x4c\xaa\xc6\xaf\xce\x88\x94\xfa\x62\x94\x4a\x60\xca\xac\x0f\xc7\xd1\x47\xf6\x19\x3a\x2d\xc7\xa9\x9f\x80\x1d\xe1\xc4\x65\xbb\x9e\xe4\x03\x94\x7d\x66\x32\x4e\xd0\x5f\xd0\x37\x0a\x8f\xcd\x2d\x01\xe8\x95\x38\x01\xdc\x4b\xd0\x12\xac\x56\xee\x25\x07\x64\x27\x92\xc6\xdf\x9e\x2f\xeb\xd3\xd8\x10\xbe\x81\x66\xe0\x90\x6b\x1a\xe6\x04\xa2\xa3\xfc\x0f\x12\x7f\xbb\xac\xa5\xfc\xa5\x27\x32\x3f\x29\x13\xb9\xfe\xde\x78\xc0\x5b\x45\x68\xee\xeb\xe2\x80\x41\x37\x77\x52\xe2\x21\x49\x7e\xce\x27\x8a\x96\xb2\xa2\xef\x29\xc3\xdb\xf1\xa0\xca\xe1\x35\x8b\x40\x1f\xce\xeb\xea\xd6\x55\xb0\x9e\x8d\x11\x57\x3a\x46\x96\x25\x48\x77\x67\x25\x76\x57\xd5\x10\x43\xd4\xe3\x84\x24\x33\x14\xd3\x11\xb8\xde\x14\xec\x97\x0c\x3c\xd6\xd4\xfb\xd1\x0a\xcd\x34\x65\x06\x08\xd8\x44\x7d\xbc\xd8\x93\x6e\x59\x46\xde\xf8\x59\x9f\xf5\xd4\x7b\x89\xa4\x12\x9c\x8d\xb5\x32\x1d\x3f\x53\x99\x17\xb2\xd3\x07\x32\x9d\x12\x61\xbb\xa0\xd2\x48\xdd\xb6\x08\x14\x76\x17\x9b\x1e\x9b\xbe\xfa\x40\xf0\x71\xc5\x16\xf5\x97\x46\xaf\xd0\xa3\x94\x2e\x6a\xaa\x22\xec\xdd\x2e\x6e\x89\x77\x1e\xd8\x74\xf9\xb3\x6f\x9d\x44\xd3\xdc\x90\xe9\xf8\xe5\x87\xb2\x19\xd3\x52\x7d\x89\x09\x73\xed\x0b\xa1\xe9\x45\xe0\xaa\x26\xa4\x26\xcd\x49\xf7\xe3\x1c\x5b\x66\x70\x6b\xa9\x58\xa5\x09\xda\x51\x1b\xed\xc9\x37\x61\x12\x0a\xf6\x0f\xa9\xb0\xa2\x91\xbd\x05\xb8\xb0\x56\x5c\xab\x57\xd7\x2f\xed\xb6\x3a\x89\x8c\x70\xb2\xb8\xc2\x4b\x7c\xea\xe6\xf9\xe5\x86\x4e\x7b\xdc\x4c\xdb\x4b\x81\x5d\x22\x9e\x24\xeb\x94\xa9\x2b\xcd\xfc\x22\x7f\x7d\xf9\x88\xf2\x7e\xf4\x02\xb8\xb5\x80\x53\x63\x0c\x14\x38\xb1\xee\x22\xa9\xec\x2a\xf9\x0f\x99\x4b\x6d\x6e\xd5\xc7\x3e\xe3\x23\x28\x64\x98\xd4\x45\xae\xcf\xfe\x3f\xf6\xde\xb5\xb9\x71\x23\x49\x17\xfe\xbe\xbf\xa2\xd6\xfb\x46\x74\xf7\x59\x8a\x72\xdb\x67\x26\xbc\xda\x70\xc4\x4b\xab\xd5\x36\xc7\x6a\x49\xa3\x8b\x3d\x7b\x86\x13\xec\x22\x50\x24\x31\x02\xab\x60\x14\x20\x35\x67\x77\xfe\xfb\x89\xca\xcc\xba\xe0\x46\x02\xa2\xd4\xf6\xee\x99\x0f\x33\x6e\x11\x40\xdd\x2b\x2b\x2b\xf3\xc9\x27\x73\xb5\x49\x86\x64\xd2\x40\x30\xf7\xb5\xf5\xfd\xef\xf1\xa4\x58\x84\x00\x98\xdf\x70\x79\x51\x8d\xc0\xc9\xc0\xc9\xa4\xb6\x63\x0f\x6d\x78\xf6\xa4\x01\xdf\x87\x7c\x99\xb0\x0d\xc2\x8e\x68\xf4\x80\x53\x5b\x40\x70\x2a\x0c\xf2\x23\xdf\x7a\xfa\x9b\xae\x1c\x09\x72\xd0\x72\xb8\x33\xaf\x4f\xe5\x52\x0d\xd8\x9c\x9e\xae\x86\x76\x1f\xb7\x6b\x36\xd8\x7f\x0e\x89\x81\xb3\x8f\x63\xda\x67\x3f\x9e\xb6\x2d\xea\xc1\x3b\xd3\x8e\xe0\x4b\xfa\x61\x43\x21\x12\x7c\xf3\xf7\x21\x67\x6b\x75\x6b\x05\x25\x32\x68\xce\xee\xa1\xfa\x50\x59\x87\xcf\x3e\x46\xb5\x72\xe0\x99\x0f\x18\xbb\x6a\x2f\xf5\x33\x8c\x19\x6d\x92\x5e\x83\x75\x20\x3f\xd7\xb0\x5c\x0f\xb6\x46\x97\xd9\xe1\xe0\x9b\xdc\xbe\xc1\x00\x69\xa6\xe9\xd6\xe0\xa3\x4f\x88\x06\x60\x99\xa4\x42\x8f\xd9\xb4\xc5\x89\x6b\x83\xf0\x1d\x68\x1c\xc3\x01\xad\xf6\x54\xe6\x49\x90\xfc\xdd\xea\x48\x2c\x81\x24\x74\x21\x90\x25\x70\x5a\x80\xfb\x74\xad\x1e\x31\x02\x2f\x4f\x8c\xcc\x42\x65\xb5\x00\x97\x96\x91\x05\x09\x79\x84\xd0\xa1\xe6\x3e\x50\x18\x17\x61\xae\x39\xce\x19\x16\x5a\x20\xea\x53\xfa\x1c\x69\x3c\xfb\x73\x1c\xd8\x5a\x6f\xcd\x17\x7d\x2e\x05\xf6\xdd\x03\x5a\xe7\xb4\xfc\xe1\xf6\xc8\xf7\xf0\xa9\x35\xec\x72\xb6\xcc\x05\xdc\xb2\x37\x8e\x37\x0d\x13\x27\x28\x05\xe7\xdd\xcd\xbb\x1f\x8f\xef\xa6\x4c\x14\x11\x4b\x93\x7b\x31\x93\x91\x7e\x80\x4b\xdf\x2f\xa5\x28\xcc\xcf\x1d\x46\xa0\x64\x23\xa4\x06\x49\x90\x14\x3d\xef\x6b\x76\x60\xcc\x7f\xdf\x55\xbf\xef\x73\x2b\x77\x5c\x9f\x66\xed\xda\x9c\x86\xb0\x4c\x21\x6d\x1b\x0e\x6d\x8b\x5d\xf3\x3b\xf4\xb7\x9e\xb5\x65\x3c\x7f\x42\x48\xb4\xfc\x6b\x29\x07\x2a\x5d\xa7\xfe\xa3\xa0\x15\x1d\x3a\xdd\x26\xe3\x90\xcf\x64\x58\xac\x35\x7e\xd3\x5a\xfa\x3e\x21\xe2\xa9\x6f\xac\xff\xdc\x27\x87\x67\x45\x2e\x04\x88\x10\xb7\x9e\xe8\xac\x27\xb6\x35\xd7\xb1\xe0\xa3\xf1\x4c\x7e\xb0\xa8\x3a\xff\xab\xf6\xbe\x86\xcd\x22\x48\xf3\x52\x2d\x05\x8a\x8d\x13\xed\x7e\x80\xa4\x7d\xba\x4c\x0b\xcc\x5a\xbc\x4c\x24\x4f\x5d\x43\xf1\x49\x9b\x94\xc8\xb9\x8c\xd6\x87\xba\xc9\x93\xe5\x5c\xa4\x43\x34\xd1\xe9\xf2\x2c\xd5\x66\x7d\x47\xf7\x1d\xbb\xf3\x29\x79\xb9\x7d\x67\xd0\xe3\x44\xb9\x3d\x99\x77\xb3\xf3\x14\xb3\x06\x0b\x06\x38\xac\x7a\x84\x3c\x92\x80\x99\x59\x24\x4d\x1d\x61\x58\x18\x9a\xea\xc2\xce\xa0\x16\xc6\x8b\x99\xcc\x4b\x09\x09\xc5\x1c\x2a\x93\x33\x9f\x13\x26\xb2\x18\x09\x42\xac\xac\x8c\x98\xc0\x94\x2b\xf8\xb2\xb9\x9f\xa9\x52\x83\x3f\x6a\x23\x0a\x73\x40\xbd\x86\x5c\xff\x08\x8b\x1e\xb1\x2c\x4f\x36\xe0\x52\xd6\x6f\x5a\xa6\xee\x94\x17\x3c\x55\xab\xe7\xb6\x2a\x3d\x31\xc4\xc6\x36\x83\x4d\xdf\x99\xc1\x5f\x09\x29\x72\xe8\x28\xd8\xb2\x5b\xb7\x70\x0f\x2b\x77\x87\xe4\x06\x4f\x22\x39\x7f\xb5\xb3\x58\xf0\xb2\x50\x1b\x73\xbf\xe5\x69\xba\x1d\xa1\xd7\x59\xb0\x35\xd7\x6b\x3b\xd1\xe8\x30\xec\x73\x36\xd1\xe0\x9e\xf2\x68\x2d\x6e\x0a\x5e\x94\xad\xc8\xac\x5a\x2b\xbf\x10\xb2\xdc\x7c\x71\xc2\xfe\xec\xfb\x78\x3a\x39\xfd\xe1\x6c\xfe\x6e\x7a\x33\xf9\xee\xfc\xec\x5d\xd0\x1f\x7a\xf2\x61\x7a\x73\xd3\xfc\xf5\x87\xe9\x6d\xf3\xc7\xab\xcb\xab\xbb\xf3\xc9\x6d\x5b\x29\xe7\x97\x97\x3f\xde\x5d\xcd\xdf\x4f\xa6\xe7\x77\xd7\x67\x2d\x9f\xde\xdd\x76\x3f\xbc\xf9\x71\x7a\x75\xd5\x56\xea\xd9\x4f\xd3\x53\x53\x1d\xfd\xfe\x97\x60\xdb\x81\xeb\xdc\x8c\x40\x47\xff\xea\x3b\xf3\x88\x55\x5f\x3c\x61\x77\xf5\xbc\x57\x14\x88\x85\x24\x62\x8f\x5c\x1b\xe1\x06\x71\x80\x60\x82\xf5\xa3\xd5\xf5\x29\x62\x95\xa3\xb5\x60\xa9\x52\xf7\x65\x46\x32\x0f\xad\xed\x52\xa1\x45\x48\xe8\xa0\xb4\x1f\xa6\xb7\x27\xcd\xfc\x5b\xae\xb0\x80\x2e\xd5\x19\x97\x1f\x39\x52\x07\x80\x9c\x05\x23\x8b\xcd\xcb\xe4\x5d\xd7\x41\x0d\x6e\xca\x76\xd5\x83\xa5\x71\x59\xd4\xaa\x89\x63\x4f\xb2\x04\x1d\x0b\x0a\xae\x4e\xf8\xae\xd1\x74\xc3\x81\x89\x47\xd9\x42\x44\xbc\x44\x44\xb7\x39\xc0\xf2\x5c\xe5\x61\x83\xfd\x42\x79\xbe\x42\x69\x81\xb5\x16\x58\x9b\x33\xd3\x71\x7d\x9f\x64\x59\x65\xda\x69\x21\xee\x9f\x79\x48\xf5\xf6\x90\x44\x85\x88\xbf\x68\xea\x45\x3e\x06\x1f\xf5\x66\xb3\xab\x4d\x93\x83\xbd\x9e\xc8\x15\xda\x12\x6c\xd2\xbd\xf5\xd6\xa1\x90\x00\xf4\xea\x61\xc0\x90\x04\xc4\x9c\x35\x2e\x29\x5a\x02\xb0\x22\x5e\xb0\x47\x01\x74\x34\x25\x65\x1d\xc5\x3b\xbd\x91\x19\x50\x1d\xe2\x01\x6c\x0e\xe1\x0a\x4d\x4d\xa7\x90\x7f\x0e\x45\xde\x7c\xaf\xc5\x30\x27\xde\x5e\x4e\x91\x77\x58\x28\x48\x7d\x1b\x2f\x00\x2d\x7e\x4e\xa7\x5f\xcb\x49\xb7\xe7\x10\x32\xc7\x41\x9f\xf6\x58\x1e\xb7\x4a\x72\x94\xfe\x0d\xab\x24\xd0\xd8\x3b\x56\xb7\x2a\xe6\x5b\xb3\x38\x00\xf4\xa0\xcb\x2c\x53\x79\xc1\x3a\xca\x40\x08\x24\xb6\x0f\xce\x32\xea\x87\x13\x91\x50\x88\xd1\x5c\x74\x4b\x1e\xb6\x7e\xd4\x52\x34\xae\x81\xef\x2c\x08\x2e\x82\x0b\xa6\xcb\x99\xb9\xa9\x5c\xd5\x2b\x2b\xb4\x4d\xa9\x3e\x24\x3a\x33\x33\x8a\x43\xdf\x14\xce\x6d\xb5\x5f\xda\x12\x5a\xa7\x3c\x15\xcb\x62\x3e\xd0\xd9\x05\x25\xca\x2e\x36\xc0\x64\xb5\x7e\x86\x12\xfb\xdf\x3e\xbe\x22\x50\xb4\xb9\x72\x04\x96\x87\x5c\xa9\x02\xf5\x5e\x7f\x37\x62\x76\x34\xc1\x6c\x41\x95\x52\x1c\xbd\x53\x2e\xcd\x5d\x02\xb1\x64\x2e\xe4\x7c\x3c\x93\x67\x00\x3e\xf5\x17\x1c\x1b\x5e\x0f\xb7\x8b\xbd\xf7\x8a\x4a\xc2\xfa\xcf\x1a\xe9\xd2\x9d\x1d\xc0\xaf\x7b\x84\x2c\x8a\x74\xeb\x38\xa2\x62\x56\xf9\xae\xcf\xee\x41\x6b\xba\x55\x2d\xb1\xc3\xb8\x75\x74\x21\x32\xb2\xf8\x63\x3f\x3d\x4a\x1a\xbc\xcd\xa6\xaa\x31\xfb\xd9\x5a\x94\x20\x68\xc8\x05\xd1\x58\xdc\x6b\xca\xb7\x96\x50\xbc\x6d\x60\x9f\x83\xa3\xfb\xb9\xc3\x88\x76\x0f\xb0\x23\xe3\x6c\x19\xe5\xca\xc5\x5e\x4a\xb4\xf4\x0e\x80\x21\x9d\xba\x8f\x6e\xc4\x6e\x44\xe5\x7b\x48\xe1\x4e\xa8\x74\xd0\x59\x64\xba\xfd\x67\x9c\x2c\x64\xf1\xb0\x20\x0d\x4a\xa9\x4d\x9e\x59\xb3\x7f\xc0\xb3\x88\x24\x1f\x6c\x99\xa4\x29\xe8\x01\x63\x36\x91\x5b\x4b\x82\x61\x8e\x42\x0b\x4e\x4d\x56\x52\xed\x8b\xcf\xef\x58\x4c\x51\xb0\x98\x6e\xba\x17\x13\xe2\x3f\x3c\x07\xd2\xf3\xac\xa8\x67\xe0\xc3\x33\xb2\x85\x37\xb3\xa9\xf4\x67\xc1\x1b\x60\x14\x08\x4f\xf3\xcf\x15\x59\xd6\x68\x6e\xf0\xe1\xdf\xdb\x9b\xfe\x7d\xc9\x73\x2e\x0b\x88\x97\x22\xa5\x35\x17\x41\xd8\xb6\xf8\x04\xd8\x56\x89\x06\x66\xf8\x29\x9c\x5c\x0b\x25\x40\xf8\x59\x12\x8f\x58\x32\x16\x63\xc8\xec\x9b\x1b\x5d\x62\xe1\xdf\x5c\x1b\xcd\x61\x26\x1b\x71\x20\x63\x36\x49\xb5\xa2\x2f\x84\x8c\x52\xa5\x01\xda\xbb\x08\x49\xd7\x61\xe5\x93\xbb\x6a\xb1\x85\xfb\x0d\x4c\xa5\x2f\x5e\xd1\x83\xe0\x43\x48\x50\x0b\xbe\xf6\x14\x76\xba\xff\xfd\x5f\x15\xd1\xc7\x76\xe1\x2f\x5e\x30\x15\x58\xe3\x18\x7a\xb1\x49\xc2\x34\xd3\xbb\x26\x08\xde\x80\x89\xf1\xf1\x39\x01\x7b\x1d\x7b\xcd\x0b\x96\x0a\xae\x0b\xf6\xf6\xcd\x20\xcc\x89\xed\xa0\x97\xae\xb4\x7d\x7d\x90\xbd\x0d\xd3\x0c\x95\x3b\x57\x31\xe4\x1d\xe6\x79\xc1\x38\x93\xe2\x31\x8c\xca\x51\x10\x48\x65\x93\x09\x8b\x80\x17\x04\xb1\xf8\xc8\x6a\x04\x91\xae\x78\x65\xea\x90\x23\x36\x55\x06\xb9\x65\xa9\x59\x2d\x2b\x6b\xe4\x50\x6d\x00\x63\x37\x2f\xf9\x80\xc9\x35\x2f\x66\x92\x24\xab\x85\xa3\x04\x21\xf2\x93\x34\xad\x06\x29\x72\x88\xc3\x95\xa6\xc3\xa6\xf5\xf1\xd8\x0d\xd0\x05\x5c\xbf\x5c\xa4\x58\xc5\xfe\xe7\x37\x0b\xc6\x32\x38\xae\xc8\xb0\xec\x56\x6d\xa7\xcd\x6e\xfd\x19\x95\xe0\x96\xea\xcf\xd5\x2a\x89\x78\xda\x43\x19\x16\x6d\x4d\xde\xb3\xb1\x9a\xbe\x82\x1d\xba\xf1\x73\x57\xd0\x5f\x55\x6e\xb7\xbb\xc3\x31\xfb\xa8\x5a\xcc\xf8\x1d\x93\x1b\xe8\x16\x87\x5c\xc0\x5d\xc8\xe2\xe7\xf2\x24\x57\x9a\x3e\x8d\x81\x30\x61\xbf\x14\xf4\x04\x04\x56\x74\x60\xdc\x5a\x1c\xc4\x43\x07\xe1\x97\x04\x22\x45\xc1\x47\x6f\x76\x78\x74\xb3\xff\xde\xdd\x1f\xf9\xfe\xdb\xae\x38\x70\x5d\xb3\xe3\xdd\xca\xde\x24\xfe\x2b\x8f\x20\x4a\x12\x6a\xb2\xf1\x99\x4d\x32\x4b\x9b\x02\x85\x83\x93\xa0\x55\x3d\xcc\x72\x15\x09\xad\xc7\xec\x0c\x0e\x1a\xfa\x93\xf1\xa5\x75\x74\x04\x2f\xcf\xa4\xb9\x99\x58\xee\xbb\xa0\xfc\xea\x12\x6f\xdb\x01\x48\xa4\x7b\x90\x8f\x68\xb3\x3f\xbf\x5d\xd7\x6d\xc2\xf2\xf8\x42\x19\x90\x12\x8b\x9d\xad\x4e\x58\xac\xa2\x7b\x91\x1f\xe7\x22\x4e\xf4\x09\xf8\xec\x8b\x4e\x67\xe1\xc6\xdc\xb6\x0f\xd6\x34\xba\x00\x08\x7b\x08\x05\x4e\xb1\x7e\x0a\x29\xb0\xa1\x49\x23\x96\x2c\xe1\x3a\x61\xe3\x59\x31\x80\xcb\x52\x05\x0a\x59\xe4\x5b\x44\x3b\x5b\x53\x56\x6d\x20\xec\x4d\xc3\x28\x6d\x5d\x91\xd8\xf9\x73\x60\x7b\x9e\xd8\x6d\x8c\xf8\x21\x20\x03\x76\xaa\x50\x14\x07\x86\xe2\x22\xe3\xc5\x5a\x03\xed\x47\x75\x0c\xe8\xd2\x05\x9f\x9a\x11\xe2\x19\xe0\x20\xd0\x4a\xe1\x3f\x72\xe4\x14\xba\x48\xd2\x74\x26\x31\x70\x03\x18\x3a\x5e\xb5\xb2\x0b\x99\x4f\x47\x8c\xc7\x31\xfb\xff\x5e\xbf\x3f\xff\x8f\xdb\xb3\xf9\xf4\x02\x6c\xde\xd3\xf3\xb3\x37\x23\xf7\xe3\xe5\xdd\xad\xfb\x15\x2d\x2c\x0f\x22\x67\x1b\x7e\x0f\x57\x3c\xa9\x05\x05\x9e\x8a\x99\x0c\x5b\x6a\x79\x97\xcc\x13\x2d\x2c\x82\x96\xd4\x14\x47\x3f\x4d\x73\xd8\x45\xda\x4a\x74\x9c\x03\x2e\xbf\xd7\xee\x93\xdd\x6b\xd0\x2e\x1e\x57\x85\x55\x03\x21\xbe\x98\xeb\x80\x88\x87\xee\xbe\x7e\xc1\x09\xb9\x4a\x64\x17\xce\x4f\xc8\x87\x97\x54\xe2\x7f\x14\x5b\x00\x9a\x5f\xf1\x24\xef\xbd\xf6\xda\x99\x14\xed\x8e\x31\xf7\x74\xae\xeb\x9b\x4a\xa3\x2e\x8c\x91\xca\x9d\x58\xd2\x36\x12\xdd\x5f\xbd\xbb\x44\xcd\x29\x3e\x15\xb9\x65\xf8\x72\xb1\xb0\x96\x06\xd3\x1d\x34\x7e\x0d\xce\xe4\xed\xe5\xbb\xcb\x13\x26\x52\xbe\x50\x10\x06\x49\x50\x23\x5b\x04\x0d\x58\xa4\x36\x41\x41\x15\x76\xb7\x11\xcb\x3c\xbb\x5b\x68\x44\x1b\x63\x19\x7b\x58\xde\x32\x95\x37\xb9\xd1\x9e\xf7\x0a\x48\x9d\xbd\x52\x79\x9f\xe3\xdf\xbc\x86\x71\x21\x99\xb9\xc8\xd5\x24\x2f\x9d\xcd\x4b\xc1\x81\xf9\x83\xdc\x42\x64\xcb\x27\x60\x6c\x9a\x56\x72\x71\x9b\x8d\xa3\xc7\xe4\xda\xf7\x6f\x2a\xc9\x7e\xfc\x46\xb3\x45\x59\xcc\x64\xb5\x0c\x25\xd9\xe4\xe7\x1b\xf6\x1d\x2f\xa2\xf5\x9b\x99\x84\xb8\xc4\x1f\xbf\xe9\xa0\xa1\x1c\xcc\xec\x6c\xc6\xe4\x1d\x2f\xf8\xb9\xe2\x71\x22\x57\x6d\xb4\xce\x3e\xf7\xe0\xd9\xed\xe4\x84\xd9\x14\x30\x3e\x8a\xb6\xb0\x74\x2a\x41\x41\x20\x90\xa1\x23\x56\x8a\x80\x28\x97\x55\xea\x5b\xbc\x99\xc1\x81\x35\x93\xb7\xc8\x67\x6d\xa4\x6a\x52\xb0\x4c\x51\xfe\x4b\x73\x2b\x43\xa6\x6f\x6e\xa3\xcb\x45\xba\x65\x66\x74\x60\x19\xbb\xc9\x20\x7d\x0c\xf4\x99\xa6\xb0\x9f\x49\xb8\xa0\xbb\xb8\xde\x54\x45\x3c\x05\xac\xdf\x51\x60\xd3\x33\xd7\x76\x55\x02\xb7\x0e\x80\x6c\xe4\xb6\x0a\xc9\x75\x74\x4f\x4e\x29\x0b\x27\x0a\x0c\x00\x30\x8f\xe4\x87\xdc\x28\x23\x71\x90\xc7\x16\x8c\x6f\x29\x8e\x8e\xf9\xd0\xf1\xda\xe2\xb0\x98\xa7\x2e\xe4\x5d\x95\xd2\xf2\xb8\x45\x60\xbe\x97\x5b\x80\x85\x43\xc2\x3a\x05\x90\x12\x2f\x9d\x69\x51\x36\x66\xd1\x9d\x89\xc1\x67\x33\x89\x08\xc4\xca\xbc\x84\xcc\x87\x41\xed\x4a\x02\x40\xb2\x19\x67\x5f\x66\x04\x98\x24\x5d\x3f\xcb\xc5\x91\x8b\x1e\x8f\x2b\x63\x6a\x4e\xd8\x31\xbb\x0e\xaf\xd7\xb1\x8a\xca\x8d\xcd\x4a\x01\x91\xe7\x84\xac\xa3\x43\xd4\xad\x10\x3c\xd8\xf7\xad\x78\x60\xb8\x2b\x04\x50\xef\xf4\xbe\x1f\xe3\x82\x99\x84\x9f\x36\x35\xf5\x6e\xc5\x17\x64\xc7\x61\x68\x38\x2c\x68\x9e\x55\x4b\xaa\x94\x76\x30\xa7\xc3\x85\x67\xce\x57\x39\x28\x5b\xe2\x53\xa6\xc0\xc8\x8d\x81\xd5\x2a\x7e\xa5\xd9\xf4\xca\x68\x40\xe6\xc6\xeb\xf6\x60\xa9\x0b\x04\xad\x61\xfc\x33\x7c\x8d\x61\x08\x23\xf6\x25\x9b\x95\x5f\x7e\xf9\x75\xc4\x3e\xd9\x7f\xfc\xfe\x77\xbf\xfb\xfa\xf7\x43\xc2\x54\xec\x85\x1c\xca\xf5\x63\xe4\x52\x91\x56\x55\xa2\x70\x06\x9a\x92\xea\x80\x59\xa0\x0d\xd8\x35\xfc\x4f\x61\xc8\x0e\x30\x49\x7c\x45\x3b\x5c\x87\x3b\x93\x55\xb6\xa6\x47\x12\x68\x51\x8c\xaa\x12\xc2\x29\xbb\xa4\xd1\xff\xf3\x0e\xa2\xd7\xb9\xd9\x2a\x4f\xc3\x4e\x25\xa9\x53\xaf\x4d\x21\xec\x35\xd9\xff\x0a\x70\x20\xbe\xb1\x07\x9c\x4a\x63\x91\x63\x9b\x9c\xc9\xce\x19\x12\x41\x38\x88\x4f\x59\xaa\x62\x4b\x2d\xef\x79\x14\x12\x50\x10\xce\x3e\x71\x23\xb9\x47\x44\x41\x4a\x71\xab\xe0\x79\x59\xf2\x48\x50\x8c\xf5\xeb\x4f\x27\xe6\xb7\x11\xdb\x9e\x00\x38\x75\xc4\xfe\x76\x42\x4c\x83\x3c\x2f\xe6\xe6\xa7\x37\x56\xd7\xa6\x22\xa0\xd1\x89\x66\xaf\x8e\x1f\x78\x7e\x0c\xe2\xf9\x18\x5b\xf4\x8a\x24\xab\xcb\xa9\x1c\xea\xe6\xa9\x52\xf7\x04\xdc\x6d\x7c\x78\x6c\x49\x6b\x61\x79\x3b\xbf\x09\x4e\xbd\x23\x35\x2a\xd8\x11\xbc\x20\xd8\x38\x5b\xb0\xf1\x5f\xb5\x92\x6c\xbc\xe5\x9b\x94\x7e\xb5\x4f\x09\x57\xcc\x35\xc5\xda\xc5\x0e\x23\x94\x6e\xd1\x52\xfa\x5d\xaa\x16\xd0\xab\x0f\xb6\xa7\x88\xcc\x85\x86\xfa\xd3\xc7\x1f\x58\xd4\x11\x4b\xe2\x01\xdc\x8b\x1b\x55\xe0\x2b\x14\x36\xdb\xec\xd5\x27\xd7\xa4\x3f\xa1\x5f\x18\x06\xc5\x06\x07\xa2\x71\xd8\xa1\xe2\x4c\xa1\x9f\xd8\x6b\x12\x41\x6f\xcc\x19\x43\x30\x68\x1c\x86\xb6\x0a\xb6\xae\x82\xff\x08\x2a\x48\x24\xc3\x70\xcf\x1d\x5f\xfe\xed\x78\x3c\x1e\xbb\xaf\x81\xf1\xe7\xff\xb0\xa4\xd0\x22\x5d\x62\x49\xf6\x04\xdb\xce\xe4\x07\x9b\xb4\xca\x1a\xaf\x3d\x1d\x76\x96\xab\x42\x45\x2a\x65\x47\xde\xa0\x1b\xab\x48\xb3\x7f\x31\x6a\x6d\x30\x94\xf0\xa3\xb9\xc7\x75\x50\xe8\x63\x96\x8c\xcf\xb4\xa9\xc8\x20\x5e\xdf\x56\x21\x03\xae\xbb\xd8\x72\x1d\x06\x39\xc3\x5a\x30\x2b\xe7\x98\x58\x72\xf3\xdc\xbc\x2c\x3e\x15\xf0\xa8\x83\x84\xb8\x15\x22\xdf\x7e\x52\x36\xc4\xad\xe7\x22\xc6\x65\xdd\x31\x00\xc4\x15\x4a\x92\x01\xfb\x39\x0a\xdd\x27\xe6\x70\x91\x61\x1a\x25\x5d\x6e\x36\x3c\xdf\x1e\xfb\xdd\xd6\x5c\x9c\x9e\xa5\x16\x64\x4c\x6a\x07\x00\x5c\xb8\x29\x6d\x2d\x42\x31\x90\x7a\x69\x4f\x34\xb7\x77\x23\xc8\x83\x1d\xb0\x3d\x09\x19\xa9\x98\xd6\xb5\x8f\x6a\xad\x6a\x2c\xee\x9d\xa6\xae\x62\x11\x31\xda\x1b\xe3\x64\x81\xf4\x67\xf4\x86\xfd\xb8\x43\x7c\xab\xb9\x2e\x8c\xa0\x5c\x0d\x70\x8f\x4e\x2f\x6f\xec\x37\xfd\x0f\x5d\x18\x87\xaa\xca\xce\xd3\x90\x5b\x58\xae\x58\xce\x1f\xfd\xf1\x0b\xd8\x0e\xb4\xce\x94\x2e\xe6\x17\xff\x3e\x55\x57\x49\x6a\x4e\x2d\x58\xe3\xe3\x99\xac\xfc\x3c\x62\x22\x4d\x36\x89\x74\xd8\x3a\x14\xee\x6a\x89\xda\xf3\x7d\x52\x98\x29\xd3\xf1\xbd\x91\x60\x96\x13\x33\xb8\x52\x4d\xe4\xd6\x2e\x1d\xe7\x98\x22\x0b\x44\xa9\x4d\xbb\xfc\x1d\x1d\xd8\x00\x92\x58\x1c\x91\x42\x9a\x04\x0b\x0f\xf6\xef\x4c\x9a\xd2\xec\x5e\xf2\x30\xe4\xa0\xbc\xa0\xb8\x23\x9b\x4c\x28\x90\x00\x50\x47\x05\x4b\xec\xf4\xdf\x16\x05\xe5\x4c\x96\x9b\x43\x83\x58\x08\x96\xfc\x6b\x99\xe9\xae\x72\x61\x4f\x2a\x0a\x88\x12\xb2\xdc\xd8\x0d\x35\x60\xc5\x9d\x91\xfa\x13\x8b\x28\xe5\xc8\xf2\x67\x0a\x02\xe4\xe3\x08\x1d\xa4\x59\x50\x17\x1e\x2f\x58\x0d\xe6\x27\x4c\x85\x7c\x8d\x7f\xbf\x61\x74\x36\x7c\x39\xa2\xf3\x3c\xd7\x8e\x3d\x0d\xe7\x1c\xf2\x7b\x8b\x18\x6d\xe8\x90\xd1\x61\xc5\xf3\x18\xad\xe5\xe1\xad\x02\x23\x83\x8d\xfe\xb5\x55\x25\x7b\x4c\xf4\x7a\x26\x6f\x95\x35\x38\x32\xa9\x5c\x4e\x8c\x11\x5c\x46\x1b\xf5\x71\x0d\x42\x00\x5a\xdd\xb6\x02\x8c\x10\x3e\x28\x86\x09\x40\xb4\x73\xa9\x62\x71\x18\xf9\xe3\xad\xf7\x55\x58\xff\x75\x2e\x30\xce\x0c\x4e\x8a\xae\x30\x5d\xa1\xf5\x40\xdb\x7c\x7d\xe2\xe1\x1c\xa2\x72\x4c\xad\xea\x71\x50\x66\x92\x90\x57\xd5\x9d\x6a\x50\x8a\xbd\x71\x06\x51\xc6\x95\xb1\x77\x99\x26\x0e\x9d\x84\xa8\x85\xe9\xb1\xd7\xd9\x8f\x7d\x8f\x60\xd8\x1d\xc0\x98\xb3\x55\xae\xca\xcc\x85\xe2\xdb\x30\x42\x9c\x06\xd2\x69\xa6\x72\xa9\x4e\xe8\x4e\x75\x9e\xc8\x7b\x5c\xf1\x2f\x35\x47\x98\x4c\x44\xc4\x15\x0a\x5c\x9b\xe1\x1f\xfa\x70\xc4\x12\x19\xa5\x25\x1c\x7c\xba\xe0\xd1\x3d\x26\x44\xe9\x32\xfa\x9a\x6f\xe6\xfb\x83\x34\x3b\x34\xa6\x32\x4d\xa9\x5a\x7f\x80\x02\xc9\x1c\x98\x80\x1e\x12\xce\x38\xbb\xbb\x9e\xb6\xd7\x7d\x9f\x34\x9d\x39\xed\xa7\x67\x75\x81\xc0\xff\xfd\x98\x0c\xc2\x5d\xd6\x28\x85\x45\x65\xa9\x3b\xe3\x52\x17\x61\x3d\x2e\xd2\xc2\x5c\x20\xe2\xeb\x16\xd3\xfe\xe0\x75\xba\xca\xca\xb9\x19\xa8\x74\x08\x40\xc0\xb4\xe2\xfb\xab\xbb\x49\xf0\xdd\xae\xa5\xf2\xfd\xd5\x1d\x0b\xea\x40\xb2\xe8\x54\x44\x85\x43\x1a\x8f\xd9\xa9\xcf\xe1\x50\xd7\xcc\x63\xf1\x90\x44\x18\x3a\x3b\x32\x5a\xd1\x4c\x02\x35\xba\xb9\xeb\x1c\x59\x3e\x4d\xf6\xfd\xd5\x1d\xb1\x70\x7a\xde\x1c\x4c\x47\x01\xd4\x18\xc3\x8e\x9d\x1a\x29\xb9\x54\xf2\x08\x29\x83\xf2\xd8\x7b\x3b\x46\x70\xb9\x8e\x78\x56\x94\xa4\x60\x3c\xbc\x1d\xdb\x39\xb9\xf6\x9e\x10\xd3\x2c\x35\x93\x46\x57\xc2\x18\x03\xc8\x9c\x66\x3a\xdd\x9c\xda\xda\xa0\x1e\x02\x0e\x80\x41\x3b\x48\xf8\x27\x2e\x72\x90\xcb\x2d\xe3\xf9\x22\x29\x72\x73\x0d\xc3\x8f\x47\xc8\x70\xb6\xb6\xd9\xb1\x70\xde\xbc\x66\x44\xc9\xee\x60\x82\x13\x59\xe8\x99\x0c\x02\x60\x5c\xb4\x31\x06\x2f\x24\x92\x01\x95\x30\x60\x6f\x2c\xb5\x69\x94\xaa\x32\xb6\xc7\x6a\xee\x92\xe7\x6d\x33\x54\xa2\x66\x12\x18\x4f\xcc\xd9\xaa\x8c\x1a\xea\xcf\xfe\x13\xf6\x51\x3e\x24\x71\xc2\x8f\x0a\xa1\x53\x7e\x54\xfc\xef\x8f\xa3\xda\x4f\xfc\xed\x97\x5f\x7e\xc4\x3c\x80\x5d\x74\x0e\x01\x6b\xd3\x81\x06\x9e\x76\x3f\x85\xe3\x3f\x34\xab\xf4\x80\x79\x3a\x4f\xee\x05\xfb\x88\xd3\xfd\x91\xc8\x8f\x9f\x36\x6d\x33\xd9\x36\x6f\xec\x29\xd3\x06\x54\xf4\xed\xf3\xc6\x76\x4c\xdb\xdb\xd5\xf8\x77\xab\x85\x99\xad\xaf\x56\xe3\xb7\x5f\xc2\x3f\x6b\x73\xb4\x6f\xf3\xba\xe8\x99\xb6\x66\xb7\x08\xa2\x96\x6d\xe9\x64\xd1\x4c\xee\x17\x46\x6c\x98\x2c\x82\x55\xdb\xb6\xf1\x79\x21\x0e\x8d\x9a\x45\x4e\xec\x01\xe8\xeb\x06\xd9\xf8\x4e\x8f\xe0\x81\x4c\xdd\x9e\x65\x1b\xe0\x9e\xdd\x94\xe1\x21\x00\x17\x1e\x0e\xe0\xf9\x81\xf7\xfb\xf5\xa7\xf6\xee\x9e\xee\xec\x6e\x66\x2a\xc4\x00\x66\x9a\x1b\xf3\x7a\xcf\x46\x56\x5e\xdd\xd5\xc6\x47\x8e\xd9\x08\x9b\x49\x70\x62\xba\xad\x0f\xd9\x45\x76\x39\xa2\xc9\x44\xbb\xb0\x41\xd7\x12\x0b\xad\x74\xf7\x6b\x5b\xef\x8a\xf6\x52\x98\xec\xd0\x79\xdd\x5a\x16\x7e\x60\x8a\x38\x10\x0a\x67\xae\xd4\xf3\x4d\x6f\x82\x75\x5f\xf1\x3b\xfa\xf8\x43\x83\x6e\xdd\xa9\x97\x1f\x20\xe2\xdb\x91\x6c\x6d\xb8\x34\xda\x9a\xad\xb5\xc3\xb1\x84\xb7\xfc\x27\x35\xe9\x2e\x7b\x52\x83\xb0\xc6\x7e\xc1\x5a\xb6\x2a\x5b\xca\x23\xfa\x56\x79\x8a\xbe\x83\x62\x0d\x66\x65\x9f\x3f\xd7\x8a\x39\x6f\x5e\xc6\x5c\xbb\x29\xcf\x57\x68\xf4\xd2\xa2\xd0\x6f\x5a\x66\xd8\xc7\xb1\x1d\x30\xc3\x56\xed\x9a\x0f\xe3\x0f\xb1\xfa\x18\x98\x54\x76\xed\x34\xd7\xca\x6a\xb2\x0d\x77\xd3\xb2\xf5\x87\x4c\xf2\x3e\xb8\x2e\x52\x39\x66\xa6\x02\x9e\xd7\x6e\x7e\xad\x03\x69\x66\x2f\xf8\xc6\xb1\xc7\x50\x69\x36\xe4\x17\x1b\xb7\x10\x90\x27\xa6\xbb\x0d\xbd\x38\x64\xfb\x36\x81\x98\x6e\xbb\x5a\x30\x93\x13\xfb\x8a\x67\xbb\xd6\x09\x5a\x59\x30\x1c\xb1\x5c\x60\x84\x0b\xd8\xcc\xb8\x1f\x75\xea\x5c\x47\x27\x86\x06\xfa\xd7\xba\x70\xa7\x45\xee\x4f\x23\xcf\x86\x1a\xf6\xa3\xa3\xe6\x7e\x6c\xc8\x3b\x25\xba\xed\x22\x15\x65\xc7\x72\x4f\xc5\x2a\xdf\xc7\x2c\xb9\xab\x52\x8b\xb8\x30\xbb\x78\x05\xbc\x66\x41\x1a\xe7\xae\xda\xfb\x5f\x93\xa8\x1b\x44\xc3\x8d\x18\x0a\x88\x4a\x48\xb7\x7e\x93\x78\xca\xf6\x5a\x65\x4d\x59\x51\x1c\x74\x16\x24\x7c\x33\xcf\x55\x77\x72\xe9\x1e\xe3\x65\x8b\xa8\x78\x0c\xd6\x98\x6c\x72\xcb\x7e\x29\x79\x8a\x47\xab\xa4\xcd\x60\x9b\x0d\xc6\x97\xaf\x7e\xcf\x26\x70\xf6\xb1\x0f\x20\x95\x01\x32\x06\xa5\x15\x8a\x25\x9b\x4c\xe4\x5a\x49\xde\x99\x65\xfd\xfe\x1b\x3d\xa7\x4c\xb1\xe6\x62\xae\xca\x66\x56\xd8\x01\x3d\x69\x29\x2d\xec\x14\x67\xf7\xe5\x42\xe4\x52\x60\x26\x79\x78\x8f\xd9\xf7\x7a\x35\x57\xf1\xb2\x58\x7f\x35\x8f\xd2\xa4\x77\xfa\x5a\x88\x57\x9d\x98\xcf\x4e\xf1\xab\x5d\x1d\xa8\x94\x5f\x69\xba\x64\xf8\x8c\xe1\xb3\x31\xfb\x8e\x47\xf7\x42\xc6\x2c\x4b\xcb\x55\x42\xb4\x37\x78\xd9\x48\xaa\x66\x85\x6a\xc7\x50\xb3\xc1\xf2\xcd\x21\x38\x93\x1b\x7e\x8f\x29\x65\x48\x85\x35\xf7\x96\x2e\xd2\x44\x67\xa8\x99\x27\xcd\xb5\xbb\x77\xb6\xdc\x69\xdc\x2c\xa6\xbe\xf6\x74\x89\xd1\x7a\x8f\x6b\x45\x18\xa7\x8a\x9d\x68\xc0\xc6\x75\xab\xb5\xc1\x4e\x66\x19\x64\xb4\x88\xca\xdc\xbc\x41\x8d\xc1\xdd\x0b\x0e\x44\x48\x8b\x54\x4a\xc6\x81\xe0\xec\x95\x66\x65\x66\x85\x08\x78\xb6\x52\xc0\x19\xe1\x14\x98\x07\x59\x12\xdd\x23\xb2\x15\x62\x37\x98\xeb\x5e\x23\xf5\x34\x13\x1e\x62\xd9\x26\x1a\x96\x48\xef\x73\x18\x6a\xa6\x91\x55\x69\xcf\x3a\xed\x19\x97\x52\xac\x85\x9c\x3f\x21\xb9\x4f\xff\x49\xab\xc4\xa0\x90\x12\xee\x3c\x84\x6e\x08\x4b\x99\x10\x99\xb7\xbf\xe1\xbb\xcc\x15\xc9\xb2\xa6\xc4\x27\x9a\x69\x5e\x24\xda\xc8\xb2\xd6\x11\xf7\xa4\x4a\x87\x8c\x3a\x1f\xc6\xe4\xd4\xc2\xe2\x54\x1b\x0b\x17\xe7\x36\x66\xef\xc1\xaf\x12\xdc\x4b\x94\xe3\x44\xea\x12\x58\xc5\x5a\x74\x92\x03\x3f\x07\x40\xd4\xf6\x20\x78\x7f\xa7\xbb\xcc\xc5\x34\x8e\xd9\xc4\xfb\xb3\x91\x15\x0a\x3d\xd5\x7b\x7a\x24\x52\x2d\x9e\xb2\xf8\x7a\xb9\x7e\x00\xf3\x05\x0b\x88\x81\x1e\xa7\xcd\xef\x9e\x25\xde\x35\xf3\x11\x68\x03\xf8\xbd\x90\xbb\xec\xfb\xfd\x5b\x88\x0e\x98\x9d\x06\x09\xe7\xd9\x51\xe8\xdc\x79\x4a\x03\xfb\x6f\x3b\x4f\xc4\x95\x2c\x8f\xcd\x90\x9b\x4b\x50\x74\x4f\xc1\x8a\xe8\xdf\x23\x2a\xaf\xc7\xb5\xd2\xe1\x3e\xb3\xf3\x87\xf7\xe8\xbc\x74\x39\xbb\x20\xd8\xd3\x0d\x30\xa2\x3c\xa5\x0a\x99\xbe\xa0\xd5\x6e\x93\xa2\x51\xc9\xcd\x37\xb3\x22\x14\x86\x01\x70\x11\xb6\xa8\x96\xdd\x2c\xb3\xf2\xb9\x72\xc2\xec\xa7\xd7\x6e\x8e\x70\xa3\x41\x3f\x7e\xa3\x2f\xa1\xbe\xe7\x20\xa3\x41\x2b\xe3\xf3\x07\x82\x3d\xd1\x05\xee\x20\xce\xd6\xfa\xa9\x20\x48\x84\x0e\xca\x4c\xc5\xcc\xaf\xf7\xae\x48\x1b\x29\x15\x42\x5c\x7f\x83\xdd\x0a\x1a\xd7\xbb\x6f\xfb\xb6\xda\x87\x00\x27\xc7\x16\x65\x92\xc6\xc8\x52\x18\x68\xa8\xca\xaa\x40\x90\x1e\x09\xf4\x91\x44\xbb\x03\xae\x65\xd1\xff\xf8\x8d\xbe\x52\xf1\x21\x0b\x6b\x38\x13\x6d\x73\x5d\xf7\x08\xa3\xd1\x21\x96\x69\xb3\x7f\x24\x32\xd5\x1d\x00\x11\xcf\x75\x35\x1f\xf0\x8e\x06\x03\xe2\x6d\x51\x2e\x6f\x20\xf9\x68\x17\x29\x53\x90\x97\xcf\x46\x59\x9b\x79\x36\xd5\xb8\x98\xbf\xae\x49\x21\x00\x95\xd7\x47\x38\xfb\xc3\xcd\xe5\xc5\xd1\x86\xe7\x7a\xcd\x81\xf4\xc2\x96\x35\xb2\xf9\xdc\xd1\x5a\x60\x81\x1d\x89\x9c\xc9\x23\xb6\x52\x23\x84\x11\x9d\xb0\x75\x51\x64\xfa\xe4\xf8\x78\x95\x14\xeb\x72\x31\x8e\xd4\xe6\xd8\x0f\xcd\x31\xcf\x92\xe3\x45\xaa\x16\xc7\xb9\x80\x40\x92\xa3\xb7\xe3\xaf\xde\xc2\xcc\x1c\x3f\xbc\x3d\x06\xf0\xc8\x78\xa5\xfe\xe5\xfc\xab\x7f\xfb\xfa\xf7\xa6\xe0\x6c\x5b\xac\x95\x3c\x21\x8c\xd2\xce\xb2\x8f\xf0\x9a\x70\x8c\x9f\xd4\x6a\xf9\xb7\xf1\x97\x61\x33\xe8\xd5\x8d\x8a\x45\xaa\x8f\x1f\xde\xce\xed\xc4\x8c\xb3\x8e\x8c\x19\xff\x08\xbd\xf8\x0c\xa1\x17\xf7\x49\xf1\x8f\xd0\x8b\x5f\x35\xf4\xa2\xbf\xca\xe5\x64\x0c\x70\x64\x7b\xf9\x68\x7e\x77\x32\xd2\x7a\x22\xf6\xc9\xa1\x96\xc3\x21\x0c\x8c\x3b\xe0\x88\x18\x98\xc0\xae\xd6\x5d\x77\x97\xe9\xb0\x38\x0e\xcd\x53\xd3\x79\xbb\x18\xc4\x03\x02\x40\xc7\x24\x02\x5b\x21\x5a\x48\x33\x9e\xb4\x05\x54\x04\x79\x7b\x0e\x18\x42\xcc\x23\xd2\x4e\x7a\xd6\x27\xdd\x16\x65\x9c\x12\xf1\xfc\x59\x12\x6f\xb5\xd6\x81\x98\xd0\xc1\xe5\x37\xf4\xee\x1e\xaa\x31\xa1\xa4\x0f\x1a\xd1\x17\x4c\x91\xf2\xdc\xb9\x51\xa8\xbb\x4f\xcc\x8b\x92\xe2\xd7\x16\xd3\xad\x1e\x6d\x3e\x94\xe7\xc8\x22\xe2\xf1\xea\xfd\x32\x88\xe0\x22\x85\xb6\xd8\x76\x75\x34\x63\xcd\xf5\xd3\x82\x03\x26\x48\x41\xec\x7c\xc1\x88\xac\x4e\xb4\xad\xd0\x9e\xc6\x96\x7d\xc9\x1c\xee\x96\xe4\x31\x2b\xf3\x4c\x69\xa1\xc7\xec\xbd\xca\x91\xd6\x8b\x38\x77\x7c\xc0\xc3\xf5\xfb\x53\xf6\xf6\x9b\x7f\xfb\x7a\x26\x5f\xb7\x28\x43\x70\x88\xaa\x7c\x45\xf1\x17\xa0\x02\x6d\xb8\x2e\x44\x7e\x9c\x2f\xa3\x63\x3c\x3a\x8e\xcd\xf7\x47\x54\xe9\x91\x5a\x1e\xb9\x14\x09\x47\xc4\x16\x3f\xde\xc4\x6f\xba\x90\x89\xed\x0a\xf7\xaf\x76\xe9\x99\x74\x28\xe6\x6d\xf3\xbb\x5f\xb0\x56\xb6\x10\x2a\x22\xa4\x85\x68\xd0\x58\x90\x8a\x51\x2d\x5d\x52\x1f\x8c\xf3\xc5\xfc\x5f\x6a\xd9\xf2\x8f\xef\x52\xb5\xd0\x6f\x1c\x01\x2c\xd7\xb6\x0e\xcf\xc8\xd8\x26\xb7\x1b\x7b\xee\x90\xdb\x37\x0d\xc5\x4b\x9a\xd5\xac\x4c\x0c\xa7\x6d\xc8\xc0\xb7\x0b\x0d\xaf\x0b\x22\x1f\x15\xcf\x55\x29\x6d\xd6\x0c\x25\x85\x5a\x02\xcc\x09\xae\x49\x16\xa5\x09\x9e\x05\xc0\xfe\x39\xee\xa9\x5c\x64\xa8\x7d\x80\x0f\xac\x7b\xb8\x0f\xcc\x1c\xb3\x6f\x9c\x5f\x22\x73\xcc\xa1\xe3\x4e\x82\xf1\x57\x1a\xf0\x43\x43\x29\x70\x2b\x0d\x41\x20\x99\xf7\xf7\xa2\x0d\x9c\x1c\xf0\x99\xab\x7d\x92\x86\x8c\xe7\xa0\xc1\x8b\xa3\x42\x1d\x01\x69\x1f\x50\xc1\x61\x2e\xa7\x2e\x08\x12\xa0\x34\x86\x1c\xf7\xe6\xfd\x1e\xed\xc4\x5b\xdb\xa7\xa0\xa1\xa4\xb0\x6a\x64\x40\x27\x48\x7a\x22\xa5\xc8\xc9\x03\xbc\x57\x33\x18\x88\xe1\x08\xa7\x72\x37\x22\xdd\x9b\x29\xc2\x3c\x3b\x2e\x1e\x91\x07\x42\x60\xcc\xe0\x6a\xb2\x56\x1b\x65\x74\x5d\x55\xea\xe0\x21\x5e\x6d\x41\x99\xe8\x54\xcc\x37\x3c\x43\x7d\xf5\xd7\xeb\x8d\xd9\x5a\xe6\x11\x9a\xa0\xc3\x97\x06\xa5\x2e\x5b\x54\x93\x35\xed\x69\xbf\xcb\xb2\xb3\x7b\xdd\x00\x42\x68\x03\x2e\xbf\x35\x7f\x10\x36\x77\x46\xf2\x37\x73\xe9\x35\x4b\xca\x5d\x23\x9d\x06\x82\x80\x36\xe4\xa2\x0e\xe1\x9b\xf6\xd4\xed\x64\x8b\x29\x37\x03\xe7\xc0\x05\x59\xf5\x99\x00\x2e\x31\xec\xc8\xc6\x1b\x1d\xb5\x06\x1c\x75\xed\x4b\xb0\xab\x95\xe6\x66\x62\xf9\xd2\x87\x35\xf5\xc6\x15\x40\xd4\xe8\xcd\x76\x7b\xba\x49\x88\x4e\xc3\x31\x46\x81\x60\x75\x8b\x2e\x90\xf3\xf0\xcd\x08\x89\xeb\x86\x8c\x1d\x54\x82\x8b\xb3\x31\x82\xc1\x5e\xe8\x1a\xc0\x61\xf6\xd7\x5d\xe6\xcc\x36\x7c\x3b\x32\xf4\xfa\xe8\x65\xd3\xca\x86\x65\xc1\x7d\xf8\xe0\x93\x23\x03\xfc\x77\x51\xc2\xf3\x8b\xcb\xdb\x10\xd9\x94\x60\x6f\x8f\xa2\xb5\x88\xee\xc1\x9a\x86\x47\x1e\x6e\x06\x0a\xc6\x07\xb8\xb5\x4f\xa9\x5a\x28\x0b\x94\xd9\xba\x2c\x33\x2e\xd3\x92\xca\x59\x9c\xe8\x2c\xe5\x5b\x80\x24\x48\x8c\x53\xf4\x70\x06\x17\xe0\x6b\x44\xc1\x3e\x67\x42\xff\x99\x36\xb3\x32\xf1\xdf\x0d\x1d\x4b\x0f\x3c\xf7\x83\xd9\x94\x07\x4c\x8b\x0d\x97\x45\x12\xcd\xe4\x46\x70\x19\x22\x58\x09\x92\x61\x06\x39\x56\x82\xf2\x25\x2c\x97\x22\x2a\x3c\xe1\x32\x5c\x42\xdc\x48\xed\xdb\x83\xc3\xfa\xee\x76\xde\xce\xae\xff\x60\xd3\x3e\x27\x1b\xc0\x47\xd3\x1a\xa2\xa3\xf1\x89\xae\x46\x48\xc1\x4b\x47\xae\xbd\xd4\xc2\x5f\x76\x4d\xb1\x85\x28\x1e\x05\xf0\x09\x11\x01\x42\x9b\x8e\x7f\x70\x1a\xa6\x43\x82\x07\x27\x8e\x7f\x90\xe8\xe5\x1b\x04\xc2\xb4\xc1\x42\xe0\xa5\x23\x3e\x94\x35\x06\xc3\x57\x44\xc9\x00\xa6\xc0\x57\x64\xd4\x7c\x05\xc7\xb4\xb9\x05\xe7\x0f\x22\x9e\xc9\x2a\xad\x24\xe9\x8c\x7e\xc3\x31\x9f\x60\xf4\x79\xa4\x8d\x1d\xe3\x5e\x8e\x9e\x33\xa0\xd2\xf2\x24\xda\x8e\x74\x60\x47\xc2\x53\xec\xf4\x4b\xde\xaa\x6c\xae\xe5\xbe\x97\x61\x9f\x83\x94\x12\x08\x52\xbe\xe1\x0a\xfa\xc7\x2d\x4a\x47\x9a\x87\x8c\xba\x0e\x2c\x4e\x46\xeb\x86\x19\xbc\xad\x8c\x99\xb4\x6c\x32\xcb\x32\x45\x96\xf4\xae\x98\x1d\xe2\xd0\xb4\x91\xaf\xbf\x5e\x04\xb4\x33\xba\xb2\x20\x67\xab\x03\xe9\x04\xc0\x7d\x94\x75\x76\xd5\x0b\xa9\x4b\x50\x29\x6c\xba\x46\xf0\x4a\xac\x44\x01\xa7\x79\x5c\xa6\x08\xd8\x04\x77\x0a\xf0\x71\xf2\x34\x65\x49\xa1\x67\xd2\xd1\x87\x62\x60\x0e\x48\x58\xeb\x6f\x89\xe9\xca\x05\x55\x40\xb1\xf0\x98\x4b\xd0\xc3\x92\x28\x29\x1a\xe1\x0e\xdb\x30\xc5\x59\x96\x09\x8e\xb1\xfc\x38\x6d\x33\x19\xde\xb9\xea\x93\x40\x81\xef\x90\x15\xff\x39\x62\xd0\x77\x18\x6e\x4d\x15\x4f\x42\xd9\x60\xef\xcc\x85\xcb\x66\x2f\xc7\xd6\x12\x7f\x10\xa1\x92\xcd\xad\xa6\xd0\xd6\x81\xe2\xef\xad\x10\xd3\x13\x95\x29\xcf\x31\x98\x69\x59\xa6\x2c\x59\x06\x89\xd8\x61\x0e\x90\x3c\xd2\x4c\x57\xa4\xe0\xac\xb6\x2e\x14\xcd\x37\x22\xe0\xad\x21\xf3\x4e\x1a\x20\x7e\x30\x23\x06\x42\x49\x4c\x59\x6f\xc6\xec\x9d\xa7\xc7\xc5\x19\x86\x3d\x11\x90\x4e\x27\x1a\xc5\x9f\x6b\x6f\x40\xb9\x00\xbd\x33\x4d\x54\xd2\xec\x48\xb7\xeb\x3a\x66\x10\x92\xd7\x0c\x83\x13\xd9\xd4\x45\xbb\x11\xf6\xad\x94\x2b\xe6\xd3\x1a\xc8\xc8\x6d\x88\x8e\x06\xda\x53\x61\x60\x23\x43\xc2\xee\x27\x34\xd4\x11\xa2\xb7\x34\x76\xb3\x23\xef\x3b\xcc\xe3\xc0\xa6\x06\x59\x14\x87\x37\x34\x58\x39\x21\x78\xac\xcf\xc8\xae\x78\x31\x14\x49\xe6\x02\xd7\x86\x37\xb4\x15\xb5\xd7\xa7\x99\x20\x3d\x06\xb6\x73\x62\xbe\x79\x62\x43\x75\xb9\x38\x42\x01\xed\xf2\x21\x81\xa8\x10\x3c\x5a\x57\x39\x24\x2c\xd3\xb3\xeb\x01\xc4\x10\xc2\x7e\x1c\x4e\x7f\x31\xf1\x6b\x0e\x12\x49\x32\xd3\xfc\x31\xbb\x94\x02\x71\x9e\x6a\x19\x1c\x2a\xd4\x00\xca\x38\x09\xc9\x76\x9c\x94\x5b\x98\x86\xc9\x7b\x4b\xad\x65\xb6\xdc\x88\x71\x5f\x3a\x48\x3d\x5c\x36\x28\x45\x3a\x74\xc9\xb6\xd4\x54\x07\xa8\x97\xfd\x08\x2a\xda\xef\xfc\x01\x5c\x7a\xb8\x04\x68\xeb\x47\xff\x69\xd9\x19\xf7\xe0\x6e\x71\x36\xd8\xa1\xba\x6e\x18\x82\x9f\xf7\x8d\xef\xd5\xba\x8a\x99\x1d\x90\x20\xf2\xee\xe2\xdd\xd9\xfb\xe9\x45\x35\xff\xe2\x1f\xef\xce\xee\xaa\xbf\x5c\xdf\x5d\x5c\x4c\x2f\xbe\x0f\x7f\xba\xb9\x3b\x3d\x3d\x3b\x7b\x57\x7d\xef\xfd\x64\x7a\x5e\x7b\xcf\xfc\x54\x7d\x69\xf2\xdd\xe5\x75\x2d\x8f\x64\x4b\x12\xc8\xdb\xe9\x87\xb3\x77\xf3\xcb\xbb\x4a\x2a\xca\x77\xff\x71\x31\xf9\x30\x3d\x9d\xb7\xb4\xe7\xfa\xec\xf4\xf2\xa7\xb3\xeb\x3d\x09\x23\x7d\x7f\x5b\x87\xf4\x39\xb0\x85\x4f\xce\x2b\x3a\x61\xcb\x3c\x11\x32\x4e\xb7\x18\x29\x62\x6f\xb6\x35\xe8\x77\x78\xf6\x26\x1b\xa1\xca\x43\x02\x3e\x6e\xd7\x82\xa9\x07\x91\x03\x0b\x18\x96\x46\x94\x21\x9e\x71\xa0\x5e\x6b\x2e\x8a\xbc\xe9\x15\xd8\x19\x55\x57\xe4\x5b\x17\xb7\xb9\xab\x39\x9e\x41\x92\x2a\x61\x99\xc8\x77\xb5\x05\x34\xa3\xbc\xcc\x8a\x64\xd1\x1d\xc2\x33\x38\xf0\xbe\xef\xdd\x1b\xf9\x8e\xdb\xc9\xe1\x2e\xda\x05\x63\x25\x92\xe5\x10\x98\x3c\x94\xf0\xd4\x74\xb9\xee\x6b\x0b\x2d\xce\xca\x45\x9a\x44\x2c\x89\xeb\xf6\x14\xe2\xc3\x00\x93\x71\x9d\x16\x3d\x13\x39\xa8\xaa\xe6\x06\x90\xe5\xe2\x88\x97\xc5\x1a\x29\x3c\x29\x70\x86\x92\xd8\xcc\xa4\x16\x51\x2e\xd0\x17\x20\x34\x18\x69\x31\x1d\x6a\x50\x13\x34\x86\x18\x6c\x62\x20\xcb\x1b\x07\x29\x6a\x3a\x7c\x04\xf8\x25\x96\x3e\xc0\x48\x8a\xef\xef\x1c\x1a\x6a\x71\x82\x09\x57\x03\x58\x18\x9c\xf0\xf8\xd0\x26\x55\x35\xfd\x36\x92\xda\x25\x15\xc5\x49\xb6\x91\x46\xed\xdd\xd8\xb7\xc6\xc2\x85\x52\x0d\xbd\xa1\xd2\xe9\xd1\x69\x2e\xe0\x10\x21\x48\x83\xb5\x5f\x00\xae\x89\x22\x93\x20\x20\xc9\x5c\xd5\x16\x62\xcd\xd3\x25\x6a\x1c\x66\x6a\xda\x59\x45\xb0\xfc\x5b\x75\x2f\xe4\x35\x4e\xd8\xaf\x22\x0e\x25\xde\x7c\x3c\xa7\x91\xb3\x08\x79\x13\xa6\x69\xa3\x5d\x55\x36\x2e\x14\x94\xa9\x02\xef\x09\xc1\x63\x0c\x40\xf2\x19\x0b\x6c\x48\xe9\x72\x99\x7c\x32\x05\xce\xa4\x68\xe5\x6c\x07\x30\x99\x65\x97\x74\x72\x19\x80\x73\x48\xd1\x77\x2f\x24\xe4\x53\x05\x72\xc0\xfd\x6b\x76\x98\xfd\xbc\x39\x17\x3b\x0c\xfa\x60\xf3\x4b\x2a\x69\x66\x43\x2f\x8f\x1d\xa7\x02\x23\xc2\x1c\x07\x07\xac\x9b\xd3\xf3\xe9\xd9\xc5\xed\xfc\xf4\xfa\xec\xdd\xd9\xc5\xed\x74\x72\x7e\xd3\x77\xfb\x3d\x47\x14\x5f\x6d\xf7\xd5\x83\xd9\x9c\x84\x38\xa6\x9d\xe7\x43\xd9\x5d\xa7\xfc\xb6\x83\x29\xd9\xdf\xfa\x24\xce\xe6\x71\xa2\x23\x73\xfc\x6d\xe7\x42\xc6\x90\xec\xe2\x49\x4b\xb5\xbd\xa8\x7a\x2f\xdc\x1b\xcc\xbd\x61\x25\x08\x9e\x76\x0f\x76\x45\xbb\xe7\x00\xc9\x04\x33\x64\x2e\xcc\xe6\x8f\x2b\x1c\x23\xe3\xfd\x19\xce\x4c\x71\x87\xf5\xad\x5a\x44\xbd\x4f\xd8\xde\x44\xeb\x12\xa8\x4c\xec\x6b\x80\x47\xed\x18\x15\x62\x20\x0e\x33\x6e\x24\x41\x16\x7a\x96\xe8\x99\xdc\x70\x19\xf3\x42\xe5\xdb\x8e\x2e\xf6\x13\x9e\xe1\xb6\xa9\x8a\xd0\xf0\xc8\x96\x42\xc4\x76\x16\xf0\x55\x2e\xeb\x4b\x09\xf3\x72\xdc\x5e\xfe\x78\x76\x71\x33\x3f\xbb\xf8\x69\x7e\x75\x7d\xf6\x7e\xfa\x27\x07\x93\xcd\xb8\x6e\x4b\x2e\x9d\xe5\xc2\x48\x17\x4b\x73\xd6\x2a\x5f\x30\x65\xb3\x2d\x87\xd2\x74\x26\xcb\x99\xb4\x92\x25\xf7\xc5\xaf\x73\x55\xae\xd6\xed\x05\xd5\x5b\x79\x35\xb9\xfd\xe1\x49\xcd\x04\x12\x4a\xcc\xeb\x8a\xbb\xad\x09\x17\x4e\x96\x24\xf7\x10\x63\x5c\x6b\x1e\x50\xa9\xc2\xab\x6d\x5e\x86\x0e\x89\xf6\xa4\xdb\x4b\x53\x68\xed\x54\xfe\x5b\x5e\xef\x5a\x40\xb7\x81\xdc\xac\x1c\x23\x00\x5f\xc7\xe4\xe0\x8d\xd2\x4e\x5a\x7e\xab\x9c\x60\x5f\x1d\xa5\x62\xb5\x12\x31\x2e\xaf\x7a\xc1\x64\x83\x23\x11\x18\xf9\x73\xbd\x6d\x14\x29\x81\xef\x01\x07\xb3\xc3\x7b\xf5\x17\xe0\x57\xee\x93\x76\x59\x71\x4a\x44\x52\xe0\xdf\x2c\xb8\xec\x70\x24\xef\x8f\x07\x6b\x2f\xfe\x32\x67\x2e\x54\x8f\x0c\x26\xd6\x65\xe0\xf7\x41\x17\xe0\xe5\x70\x7c\xab\x6b\xc7\xb5\xc8\x52\x1e\x09\x17\xe0\x82\x0c\xc0\x70\xaf\x7f\x8a\x03\x8f\xd2\x24\x4b\xb2\xb7\x04\xe9\x93\x7d\x66\xb8\xb6\x25\x00\x96\xdb\x6b\x2b\x8f\x5f\xde\xb4\xb2\xf3\xe2\x46\xbc\x9f\x60\x68\xc6\x3c\x95\x14\x17\x81\xb6\x28\x48\xfe\xda\x89\x59\x1f\xb4\x1c\x6a\x35\xff\x44\x13\x8f\x77\xe6\xaa\xa1\x9b\x5b\x66\x5d\xb7\x3c\x9c\xea\xb8\xcb\x5e\x58\x14\xf9\x4e\x32\xee\xe7\x70\x47\x5c\xe5\x6a\x93\x68\x31\x29\x8a\x3c\x59\x94\x61\x36\xe2\x81\x80\xb9\xca\xe5\xc4\x77\x38\xcb\x55\x5c\x46\x96\x3e\x0b\x7a\xeb\x61\x3f\x64\xe5\xb3\x5a\x47\xcc\x8e\xcc\xea\xa3\x9b\x9b\x88\x8f\x20\xda\x03\xf9\xdd\xda\x7c\x6c\x56\x30\x76\xd8\xfe\xae\xec\x51\xfe\xcc\x31\xa3\xdd\x83\x69\xd7\x40\xbf\x30\x70\x66\x5f\x07\x0d\xb8\x03\x35\x45\xcb\x65\xc1\xd1\x81\x5e\xd5\x51\xba\xd8\x72\xdc\x51\x33\x0c\xdc\xd5\x0f\x1b\x53\x0d\xa7\x42\xbd\x61\xcd\x35\xaa\xf3\x45\xb4\xae\x36\x1c\x7a\x53\x65\x0d\xae\x37\xd7\xa9\xc7\x87\x99\x4d\x7a\xb9\xd1\x46\x68\x68\x48\xc8\xb0\x5d\xc9\x00\xeb\xd2\x59\x77\xda\xef\x31\xe4\x62\xfe\x4b\x29\x86\x64\x75\xb6\xa1\x1a\x7f\x84\xcf\xf6\x02\x52\x12\xc4\x6e\x39\xdb\x6b\x91\x6c\x8c\x06\xc4\xf3\x68\xcd\x16\x5c\x13\x1d\x61\xc8\x96\x80\xe9\xe7\x59\x62\xbe\xe2\x51\x41\xe9\x78\x6d\xb5\x36\x25\xef\xad\x85\x42\x1a\xb5\xd6\x5b\x3d\xda\x96\xdb\xbe\x01\x18\x62\xbd\xb6\xcd\x98\xbe\x1b\xe4\x43\x08\xf5\x70\x77\x4f\xc6\x23\x16\x4e\xa7\x94\x97\x32\x5a\xb3\x2c\xe5\x48\x28\xb1\xe6\x1a\x05\x85\x45\xe8\xf0\x45\x92\x26\x05\xf0\x84\xa1\xe3\xb8\xb6\x6e\xcd\xe5\x99\xe7\xf7\x36\xdd\x02\xf7\xa4\x70\xbb\x44\xc9\x81\x48\x68\xd7\xab\xcf\x8a\x85\xf6\x82\x30\x14\xee\xfd\x36\x3b\xe1\xa0\xfd\x74\x98\xe3\x0d\x36\xbb\xef\xcb\x30\xef\x10\x95\x78\x55\xff\xbc\x36\xde\x48\x40\x79\x10\x49\xf0\xce\xb8\xac\x17\x01\x9d\xfb\xa8\xb2\xdd\xc7\x68\xb3\xc3\x2d\x4a\xf0\x70\xe0\x13\x25\x4e\x1a\xa0\xc4\xd4\xd3\x2a\xb5\xee\xfb\x65\xaa\x78\xb1\x3b\xca\x0d\xb3\x24\x75\x95\x1d\xab\x72\xd1\x95\x97\x03\x5b\xf5\xf4\x18\x3a\x2b\xfe\x9f\xcb\xe6\x1e\x9e\xa3\xbc\x10\x46\xfa\x3e\x6d\x40\xcd\xd7\x47\xf0\x79\x7b\xe1\x14\xc5\x3c\x98\x91\xc2\x2d\x03\x9f\xab\xcf\xe9\xfe\x00\x49\x6d\xd9\x4e\x75\x25\xef\xa0\x98\xc7\xc3\xe6\x2b\x91\x7b\x96\xd2\xfe\xf4\x5f\x5f\x7f\xd5\x27\x1a\xf1\x8f\x25\x37\x07\xc0\xe5\xf2\x06\x09\xc2\x0e\xe9\x74\x91\x34\xb7\x55\xbb\x18\xa8\xd7\x7a\x5b\xf5\xd2\x86\x0b\xbf\x37\xdb\x41\x5b\x6f\x6e\xcc\xd7\xfd\xc5\xee\xb4\x62\x8d\xcd\xf2\x44\x01\x51\x96\x5a\x56\x74\x8d\x16\x49\xdc\x5a\xef\x01\x23\xf9\x4b\x29\x4a\x61\x16\xd0\xa2\x8c\x57\x4d\x67\xc9\x80\x0b\x97\xef\xd2\x5a\x3d\xb2\x4d\x19\xad\x99\x2d\x9c\xc5\x22\xe5\xdb\xaa\x1a\x65\xee\x1a\x85\x02\x0a\xe5\x41\x7c\x81\x01\xf1\x7d\x54\xea\x42\x6d\x00\xa7\xee\xcb\xcd\x4b\x09\xbb\x9c\x71\xbb\xbb\xda\x0e\xb4\x0a\xa1\xe7\x13\x3d\xe4\x37\x57\x67\xa7\xd3\xf7\xd3\x9a\x7b\x7a\x72\xf3\x63\xf8\xf7\xcf\x97\xd7\x3f\xbe\x3f\xbf\xfc\x39\xfc\xed\x7c\x72\x77\x71\xfa\xc3\xfc\xea\x7c\x72\x51\x71\x62\x4f\x6e\x27\x37\x67\xb7\x7b\xfc\xd4\xcd\x5a\xbb\x27\x82\x07\x7c\xa3\x16\x39\x6f\x93\xe9\x58\x73\x15\xd5\x7a\xc2\x26\x96\x7d\xb5\xc2\x0f\x6c\xb1\x06\x00\x4e\x4a\x11\x63\x89\x90\x84\x77\xbc\xe0\xa7\xbc\xe0\xa9\x5a\x8d\xd9\x84\x51\x5c\x01\xc6\x8b\x68\xa3\x12\x12\x35\xa5\x99\x1d\x2c\xc2\xe8\x85\x91\x37\x05\xf9\x6c\xe1\x6a\x49\xa4\xb0\xa9\x08\xf3\x4a\xd9\x20\xcf\x99\x3c\x7b\x10\xb2\x28\x41\xd1\xe6\x69\xca\xa8\x5a\xfb\x42\xc0\x0a\x62\x5b\xa9\x93\x4d\x92\xf2\xdc\x27\x76\xbe\xa4\xb2\xe0\xb2\x6b\xdb\xea\x58\xe9\x9a\x94\x13\xd6\x1e\x70\x37\x65\xd0\xee\xd3\xf3\x29\x28\xba\x51\x61\xb3\x16\xda\xca\x67\x12\x49\x47\xa9\xc6\x0d\x87\x18\xa6\x42\x91\x81\x1e\xab\xa7\x97\xbb\x17\xe2\x41\x8a\x95\x75\x65\xbd\x94\x61\xc2\x35\xd2\xfe\xe3\x4c\x16\xf9\xb6\xb7\xf6\x7a\x0b\x8c\x0e\x1a\xee\x75\x04\x89\xac\x26\x7b\x46\xfb\x29\xb3\xa5\x5f\x80\x4a\x6b\xf1\xba\xe4\xde\x73\x5e\x3c\x84\x47\x75\x5c\x89\x52\x73\xf2\xfe\x56\xc7\x21\x64\x01\x83\x51\x58\xa8\x52\xc6\x9a\xc0\x9b\x9b\x44\x1e\x6f\xf8\xa7\x37\xb6\xa7\x48\x62\xe3\x52\xae\x01\x63\xa2\x48\xcd\x7d\x70\x6b\x84\xdc\xee\xe1\x9a\xc9\x1d\xe3\xb5\xff\x4e\x60\x25\x2b\x98\x0c\xbc\x7d\x07\x61\xa8\x0f\x62\xdb\x36\x7f\x8d\xb4\x99\x2c\xcc\xfd\x00\x85\x64\xb9\x30\x2f\x3a\x8c\x6b\x8a\xd0\x65\xf7\x37\xc4\xb2\x54\x52\x7b\xb7\xcb\xee\x10\x36\x72\xd0\xb6\x69\x05\xac\xf4\x57\x7c\x7a\xe7\x3d\xa5\x9a\xcc\x9c\x21\x7c\xc5\x7a\x4e\x28\x76\x87\xfc\xf2\x66\xb2\xfe\xaa\x16\x6c\x09\x81\x6c\x64\x27\xc8\x05\x78\xca\x60\x2a\x6c\xa2\x1e\xe0\xd5\x6b\x60\x62\xec\x12\x48\x85\x06\xff\x91\x34\x97\x6a\xf1\x4b\x49\x10\x80\xb7\x5f\x0e\x3b\x67\x0b\xcc\xf6\x80\xf4\xde\xf5\x3c\x08\xee\x2c\x87\x76\x95\x32\x69\x23\xdb\xbc\x2e\xa5\x39\x8a\x9f\x03\x3d\xd5\xdf\x3d\x5e\xab\x94\xfe\xdc\x1b\x6b\x66\x3d\x3b\x39\xbe\xff\x62\xcc\xcd\x3f\xd5\x08\x9b\xa9\x3a\x88\x6c\xa0\xd2\xc3\x03\x6d\xc1\xa3\xfb\x47\x9e\xc7\x68\xfe\x07\x38\xd3\x98\xfd\xa0\x1e\xc5\x83\xc8\x47\x2c\x12\x79\xc1\x89\xaf\x50\x03\x9e\x03\x36\x14\x95\x33\x93\x10\xe8\x83\xe4\x8f\x52\x97\xb9\x60\x45\xb2\x5a\x17\x22\x0f\xd1\x38\x2a\x37\xe2\xa8\x40\xaa\xda\x4c\x44\x44\xc8\xd6\x31\x00\xcb\x94\x3f\x34\x09\x18\x9f\xc2\x24\xc3\xa6\x2e\x5a\xd9\xba\xbb\x6d\xf2\xb3\x5d\xf8\x29\x1a\x30\x12\x9a\x48\xa1\x35\x62\x2b\x95\x72\xb9\x1a\x8f\xc7\x90\xe8\xe3\xcd\xa0\x85\x4e\x05\x86\x0e\x74\x87\xd2\x4f\x95\xd2\x22\xdd\x3a\x12\x31\x17\x47\x05\xc0\xdd\x4f\x85\x90\x3a\x41\xc3\x56\xcb\xf2\xbf\xa9\x3b\x97\x3e\xaf\x2f\xae\xfd\x7a\x3e\x38\x4a\xb7\xa3\x1c\xc8\xa5\x3a\xa0\x24\x7c\xbf\xfd\xe6\xf5\xa4\xa8\xf3\xf6\xb2\xa4\x92\x43\x43\xa9\x7f\x52\x49\x07\x14\xe4\x49\x64\xa3\xad\x25\x11\x11\xd2\x93\xc2\x4f\xdb\xc7\xac\x11\x11\x7c\x40\x30\xf0\x8e\xb8\xde\x81\x21\xbd\x7d\x0c\x01\x37\xf5\xe9\x1e\xbc\x2d\xf6\xa7\x77\x6b\xed\xd0\xc0\x90\x69\xcf\x6d\x30\x44\x75\xc2\xa8\xcb\x74\x0b\x37\x2e\x17\x40\x0d\xee\x81\x38\xf0\x2a\x55\x9c\x66\x10\xca\xe7\xbd\x6e\x8e\xa0\x2e\x70\xb2\xe9\x42\xe5\x7c\x25\xd8\x46\xc4\x49\xb9\x69\x15\x36\xae\xb9\x87\xc0\x47\x55\x5a\x6e\xba\xa9\x42\x0f\x55\xa0\x7d\x23\xf1\x5f\xa7\x50\x5d\x7f\x0e\x1d\x17\x19\x61\xb3\x6c\x52\x7b\xd1\x85\x44\x63\x6d\x4e\xca\x3c\xd1\xc0\xb2\xfb\x94\xc8\x59\x57\x0c\x16\x0d\x0e\xf8\x6d\x86\x46\xf6\xca\xec\x1e\x59\xcf\x28\x7d\xa2\x71\x56\xc1\x6b\xdf\x7d\x28\xd4\x41\xa9\xc3\x73\xed\xe5\xaa\x6c\x70\x4f\xf5\x02\x4a\x80\xda\x18\x64\xde\x20\xd4\x1c\x14\x48\xd0\x9e\x42\xb1\xa5\x8d\xc5\xbc\x17\x01\xf5\x61\x0c\x39\x39\x1e\x91\xf2\xe9\xc7\x6f\xb4\x05\x01\x11\x4e\xcb\x6b\x2c\x85\xaf\x04\x3d\x40\x0f\x6f\x2d\x3c\x0f\x7b\x88\x45\x00\x41\x61\xcc\x65\xd1\x5a\x80\x47\xaf\x42\x59\xf8\xc9\x4f\xbc\x4c\xdb\x5f\xa7\xf2\xe1\x55\xcc\xd9\x3a\xf9\xf9\x86\xe1\x50\x53\xfe\x84\x7c\x57\x43\x83\x42\xf6\x03\x04\x61\xb8\xe6\x4f\xd0\x04\x2b\xf3\x80\x83\x6e\xd3\x77\x98\x61\x17\x45\xb4\xf6\x9a\x07\x10\x34\x3a\x62\x49\x4a\xc8\x4d\xfd\xdc\xf8\x8c\x10\x88\xbd\x0e\x41\xac\xc9\x4a\xaa\x30\x95\x92\x92\x02\x5c\x71\x46\x00\xa9\xb0\x58\x96\x14\xfb\x91\x82\x03\x59\x09\xf7\x2d\xb5\x42\x21\x02\x8c\xfa\x59\xf1\x53\xc3\x95\x22\x41\xba\x2a\x0b\xb3\xc6\x3b\x11\xe5\x77\xae\x67\x0a\xa8\x12\x80\xcc\x64\xb5\xaa\xc6\x20\x59\x28\x5f\x92\x0b\x24\xf8\xd6\x46\x7b\x2b\x92\x07\xb3\x51\x9b\xcb\xda\x2d\x50\x90\x00\xcd\xb5\x47\x6e\x5b\x16\xb0\x84\xdf\x8b\xad\x0e\x93\x49\xd3\x8a\x62\x5d\x0b\x32\x31\xfd\xa1\xf9\xda\x3f\x15\x30\x70\xf3\xdc\xa7\x84\xec\x77\x96\x61\xa5\x1f\xcc\xc7\x3b\x30\xc2\x8d\xc2\xcd\x1a\xf4\xc1\xae\xde\xa6\x48\x62\xc2\x8f\x33\xcd\xa1\x87\x01\x02\xc8\x33\x84\x71\x86\x91\x4b\x70\xf1\x35\xf7\xdb\x99\xa4\x44\x02\xc1\x21\x67\x04\x4e\x73\xda\x28\x02\x1f\xe9\xcb\xb7\x15\xf6\x20\xa0\x56\xb5\x34\xb3\xd5\x2a\xad\x77\x19\xf2\xf2\xc1\xf2\x80\xaa\x31\x46\xd9\xda\xf0\x5a\x2b\x7c\x22\xb6\x94\x26\xb7\x13\x4f\x1a\x04\x02\xe2\x9b\xc4\x2e\x8a\x59\xc9\xf1\xf6\x13\x09\x33\x7c\x13\xd9\x0a\xe5\xb4\x40\xce\x9b\xb3\xd3\xeb\xb3\xdb\xcf\x86\x37\xb5\x60\xcf\xc1\x80\x53\xdb\xce\x77\x67\xef\x27\x77\xe7\xb7\xf3\x77\xd3\xeb\x97\x40\x9c\xd2\xa3\x27\x40\x4e\x6f\x28\x3f\xc9\xa9\x92\x85\xf8\x74\xd0\x99\x9c\x97\x72\xce\x07\x84\x3e\xb9\x0c\x45\xbb\xd4\x1d\x2c\xb4\x99\x5f\xc5\x25\x3f\x21\x6e\x5b\x42\x9d\xd8\x74\x2a\x4b\x6f\x34\x5c\x26\x69\x0a\x91\xe0\xce\xbc\x4e\x51\x86\x66\x50\x41\xfe\x58\x3a\x5f\x92\xa9\x33\xb9\xa8\xa4\xbf\x01\x93\xdf\xda\x5c\x82\x31\x06\x3c\x33\x03\x90\x27\x10\x61\xbb\x2b\x05\xcb\x2a\x91\xc2\x37\x03\x66\xcd\xb4\xaf\x93\xa6\x9e\x26\xf1\x25\x91\x75\xa4\x78\xf5\xd5\x35\xed\x8a\xab\xac\x4f\xab\x7e\xda\x87\xae\x87\xb8\x89\x13\x89\x8a\x69\x65\x37\xdf\xb4\x2f\xdd\x63\xbf\x05\x60\xdc\xcd\x4c\x72\xf0\x41\xe8\x82\xe7\x85\x9f\x48\x9a\x08\x4c\x0c\xe7\x9d\x13\xf7\x09\x22\xd0\xd4\xb2\x36\xce\x46\x14\x9a\xb1\x4e\xc0\x53\xc1\x89\xdc\x26\x4a\x4b\x5d\x88\x9c\xcc\x26\x93\x9f\x6f\x66\xf2\x3b\x73\x7c\xbd\xa1\x53\x88\xd2\x77\x61\x15\x88\xd4\x51\x95\xfa\xad\x86\x12\x4a\xb0\xd7\x68\xa3\xde\x08\x2e\x35\x83\xad\x91\xa6\x22\xf7\x2b\x03\xdb\x23\x44\x4c\x49\xb4\x81\xea\xd9\x7f\xff\x86\x11\xb8\xd5\x0c\x85\x69\xaf\x4b\x23\xb6\x51\x45\x73\x3d\x75\x11\x0d\x00\xe2\xfc\x25\x57\x4e\x4b\xe0\x53\xdf\x55\x44\x60\xfd\xd6\x45\x54\x0d\x43\xea\xb5\x96\x6e\xb1\xb8\x7f\x2c\xa5\x67\x5c\x4a\x3d\xce\xf5\xf0\x94\x60\x6b\x65\x04\xa8\xcb\x6d\xe5\xdd\xcc\x8e\xe8\x24\x05\x94\x9b\x19\xc6\xd6\x53\xa7\x96\x5d\xf6\x10\xec\x07\x14\x75\x18\x42\x7b\xd2\xc2\xa8\xe4\xd3\x18\x5a\xdf\xce\xce\xc4\xb5\x2f\xc3\x5c\x38\xb1\x58\x55\xa9\x0a\xcb\x41\xe2\xe0\xa1\x84\x75\x35\x2f\x38\xf2\x9b\x9d\x6d\x24\x42\x19\xab\xa5\xcc\x0f\x4c\xfe\x78\x1b\x62\x6a\x2b\x51\xd9\xd8\x8a\x90\xcf\xc1\x72\x38\x38\x0e\x98\x21\x8b\xef\xe9\xe9\x85\xab\x6b\xce\xf1\x89\x3e\x09\xec\x70\x71\x79\x71\x16\x42\x15\xa6\x17\xb7\x67\xdf\x9f\x5d\x57\xe2\xf9\xcf\x2f\x27\x95\x98\xfc\x9b\xdb\xeb\x5a\x28\xfe\x77\x97\x97\xe7\x67\x0d\xcc\xc3\xd9\xed\xf4\x43\xa5\xf0\x77\x77\xd7\x93\xdb\xe9\x65\xe5\xbd\xef\xa6\x17\x93\xeb\xff\x08\x7f\x39\xbb\xbe\xbe\xbc\xae\xd5\x77\x77\xba\x1b\x3d\x51\xe9\x46\xbb\xf9\xc7\x3b\x67\x03\x6a\xd5\xd6\x6d\x5c\x4d\xbf\x7c\xc0\x2e\xee\x89\x3c\xdb\xb7\x1c\x6d\xb8\x7e\x1c\xa6\xe3\xc0\x8d\x61\x9a\x3a\x68\xd5\x3d\x7f\xbe\xe8\xca\xd0\x65\xfc\x30\xb1\x67\x4e\xb5\xf9\x73\x20\x01\x77\x2a\x80\xae\x96\x9a\xe1\x96\xd2\xb3\xe3\xd0\x66\xe0\xc1\x5a\xf1\xce\x7c\x65\x32\x7e\xf1\x96\xda\x3a\xf6\xb5\xd3\x53\x79\xed\x61\x44\x7a\x2e\x36\x94\x5d\x8d\x0e\x2a\xb3\x64\x03\x49\x6c\x15\x05\xfb\x30\x84\xdd\x9b\x6e\x98\x95\x13\x2c\xc7\xae\xc4\xbe\xed\x61\x4b\xbb\xd9\xf7\x86\xb6\x9f\x2a\x69\xb6\xbd\x46\xd5\x32\xa0\xdd\x40\x99\x35\xa4\xdd\xb7\x5c\xdf\x0f\x6d\x37\x55\xd2\x6c\x37\xa8\x7d\x4f\x6a\x37\x18\xbc\x8b\x76\x1a\x9d\x01\x42\x2c\x2c\xa6\xda\x3c\x17\xe3\xef\x5e\x09\xf2\x67\xf7\x6b\xa3\xd9\x00\x2f\x7b\xbd\xcc\x78\x7f\x47\x06\xb4\xc6\x6d\x57\x5e\x63\x95\xbf\x81\xa7\xd0\xc3\x45\x2e\xf8\x7d\xac\x1e\x69\x3e\xea\xc8\x50\xd6\x4b\x9a\x57\x07\xc8\xc8\x70\x7b\x44\x14\x39\x79\xa0\x10\xa5\xe6\x8b\x07\x98\x5c\x42\xbc\xe8\xa8\x83\x05\xa9\x97\xeb\x44\x44\x40\xfd\x24\xfd\xec\xcc\x24\x6a\xf3\x6d\xe9\x9b\xcd\xac\x9a\x16\x11\x75\x08\x74\xd5\xe9\xd0\xe8\x5c\xd7\xc1\xc4\x52\x1c\x50\x99\x03\x98\x6e\x91\xc3\x9d\x09\x06\x24\x91\x60\x4c\xce\xcd\x85\x27\x17\x51\xa2\x45\x90\x31\xae\xf5\xc4\xfe\xe5\xb0\x54\x28\x05\x2f\x5a\xcd\xae\xbd\xed\xe1\x3c\x2a\x4a\x9e\x32\x08\x57\x22\x06\x46\xb4\x55\xe2\x2f\x11\x97\x18\x1a\x53\x88\x4d\x06\x51\xfd\x61\x4c\xc7\x4c\xfe\x0c\x40\x09\x9c\x82\x57\x9a\x7d\x0f\x90\x07\xfb\x32\x1d\xc2\x1b\x5e\xc0\x59\xfc\x47\xac\xc3\x3d\x1b\xcf\x64\x25\x03\x53\xf0\x55\x25\x19\xd3\x78\x26\x6d\xb6\x8e\x58\x45\x7a\x0c\x37\xbe\xb1\xca\x57\xc7\x94\x4b\xdd\x2c\x76\x75\xbf\x50\xea\xfe\x58\xc8\x63\xb0\x49\x15\xc7\xbc\x2c\xd4\x31\xc0\xa5\x70\xfe\xf5\xb1\x4d\x7a\x6c\xb3\x46\xeb\xe3\x75\xf2\x20\xe0\xff\xc6\xeb\x62\x93\xfe\x8b\xce\xd6\x9f\x8e\x56\x69\x7e\x64\xbe\x3d\x0a\xbf\x3d\xb2\xdf\x1e\xd9\x6f\x8f\xcc\x67\xf8\x7f\xd9\x16\xdd\x3b\xe2\x13\x37\x67\xd9\x68\x26\x13\xa9\x45\x5e\x80\xf6\xf3\x98\x27\x85\x4f\x75\xb5\x65\xaf\xfe\xf3\x3f\xd9\x38\xe7\x8f\x18\x11\xfb\x8e\x17\xfc\x0a\xed\x8b\x7f\xff\xfb\x2b\x70\xa8\x62\x14\x53\xc6\xf3\x5f\x4a\x51\xcc\xa4\x16\x66\x13\xb2\xff\x35\x93\xe0\x81\xdd\x6c\xe7\x05\xda\x5d\xd1\x06\x19\x6b\xf6\x2d\x96\x39\x45\x36\xd2\x58\x9b\x92\x3a\xc2\x09\x12\x9e\xb6\x64\xe9\xef\x30\xd1\xff\x92\xbe\xa3\xf7\x07\x6c\xeb\x5f\xd2\xea\xae\xb6\xc9\x96\xf4\x2f\x29\x1c\xa0\xa9\xe2\x16\xac\xc5\xdc\xe2\x85\x7b\x32\x35\xae\x6d\x8f\x34\xa0\x01\x2f\xea\xa6\x6f\xdf\x2b\x37\xc8\x88\x6e\x2d\xf7\x0d\x31\x02\xbe\x02\xef\x87\x00\xef\x79\x62\x76\xc8\x0d\x5a\x42\x41\x73\xc3\x9e\x83\x4e\x4a\xae\x73\x57\x1e\x1a\x2e\xf4\xd7\x27\xc7\xc7\x23\xb6\xd2\xf0\x9f\xc5\x2f\xf0\x1f\x40\x0f\x3d\x17\xa9\x6f\x63\x30\x1d\x10\xae\x39\xcb\xfb\x67\xe2\x39\x50\x74\x9f\x83\x47\xbe\xb6\x4c\xbf\x2b\x65\x9c\x0a\x1f\xda\x58\x71\x89\xa4\xca\xcc\xa4\x9d\xa8\x66\xe6\x21\x98\xe3\x85\x88\xb8\x11\x7c\x8d\xba\x11\x5c\xaa\x96\x85\x90\x68\x0d\xcb\x7d\xb6\x47\x8e\x96\x2b\x50\x8b\x01\x0a\xc9\x0b\x82\x9c\x0b\xf8\x11\x2a\x01\x62\xf6\x51\xfd\x11\xdb\xaa\x92\x38\xc6\x81\x39\x37\x16\x51\x0a\x89\x1c\x2c\x7b\x10\xcb\x45\x51\xe6\x92\x71\x96\x71\x19\x73\x0d\x2b\x70\x99\x83\xb7\x33\x67\xbc\xd9\xd0\x11\xc2\x71\x55\x59\x00\x27\x16\x22\x0b\xc2\x91\x40\x12\xf8\xa0\xcd\xa3\xa0\x11\x78\x26\x00\x17\x75\xe3\xc3\xf1\x4c\xda\x7c\x84\x84\x85\x43\x4b\x59\xa4\xb2\x2d\x31\x1e\xd5\x07\x3d\xb1\x96\x33\x1a\xee\x91\xc7\x9b\xd4\xdf\x1d\xb1\xa4\xea\x5a\x03\xbe\xf9\x22\x48\xf1\x6e\x93\xe4\xbf\x16\x32\x52\xb1\xc8\xf5\x1b\xb3\x0d\x13\x77\xef\x40\xfd\x21\xd1\x7e\x32\x40\x4a\x99\xc3\x8d\xac\x85\xa6\x78\x97\x60\xca\x8c\x4e\x85\xa1\xbc\x4d\xcf\xd9\xbf\x55\x7e\xeb\x28\x98\xb6\xf6\xd2\x3f\x3f\x2b\x22\x26\xc4\x75\xda\x3b\xe7\xd3\x4d\x10\xb8\x65\x43\x89\x8b\x85\xa2\x8e\x43\xca\x89\xcd\xa7\x9d\x14\x90\x21\x33\x17\xba\x98\x49\x3a\x81\x47\x6c\x29\xb8\xd1\xf3\x46\x2c\xd2\x0f\x28\x8c\xf1\xb8\x2f\x1e\x95\xc7\xe0\xd8\xf4\x36\x00\x86\xad\x14\xee\x8d\xc4\xf8\x1a\xa7\x08\x6c\x04\x18\x74\xdd\xd0\x9d\xaa\x02\x83\xd5\x2a\x10\x9f\x30\x0e\x36\x5b\x4a\x3d\xc3\x5a\x98\xac\x07\x46\x62\x8b\x8e\x62\x56\x6f\x07\x3e\x30\x82\x07\x7b\x87\x30\x90\x40\x38\x82\xc6\x4d\x58\x5a\xdc\x67\xde\x87\x1b\x52\xd6\x83\x6d\xa6\x6b\x53\xed\x18\x08\x68\xc0\xd3\xec\x16\xe6\xd3\xbd\x06\x2b\x2d\x72\x9b\xca\x05\xfb\x8a\x04\x93\xeb\x24\x8f\x8f\x32\x9e\x17\x5b\xbb\x7c\xd3\x64\x01\x19\x20\xd2\xe4\x5e\xb0\x49\x9e\xab\xc7\xe7\x1e\x85\x4e\xd1\xd2\x75\xc3\x3e\x04\xc9\x3e\xf4\x96\xdf\x4a\x2f\x5b\x37\x77\x3c\x8d\xca\xb6\xcb\xf0\xd1\x5a\x4f\x2e\x8a\x7c\x3b\x37\x0b\x71\x93\x75\x4a\x8a\x5e\x41\x13\xfd\x95\xdc\x61\x2c\xb9\x35\x13\x46\x27\x4b\x6e\x65\x56\x7f\x3b\x2c\xb9\x2d\x04\xb8\x4d\x96\xdc\xe9\xc5\xf4\x76\x3a\x39\x9f\xfe\x9f\x5a\x89\x3f\x4f\xa6\xb7\xd3\x8b\xef\xe7\xef\x2f\xaf\xe7\xd7\x67\x37\x97\x77\xd7\xa7\x67\xbb\x69\xaf\x9a\xad\xf7\x2a\xf8\x11\x0b\xeb\x39\x61\xb7\x01\x50\x03\x83\x0d\x48\xff\xa6\xfc\xb8\xb0\xaa\xcc\x66\x4e\xe4\x6a\x04\x1b\xf5\x84\x9d\xe5\xf9\x74\xc3\x57\xe2\xaa\x4c\x53\x80\x53\x61\x64\xcf\x69\x2e\xe0\xe2\x39\x62\x57\x2a\x9e\x06\xdf\x41\x38\x62\x6b\x37\xa0\x7e\x1e\xc7\xb9\xd0\x1a\xab\x1f\x51\xfd\x01\x78\xc8\x85\x3a\x12\x78\x8e\x3f\xf0\x24\x35\xf7\xb7\x13\xf6\x1d\x8f\xee\xd5\x72\x89\xe1\x33\x23\x17\x38\xc5\x7e\x29\x55\xc1\x99\xf8\x14\x01\xd5\x5b\xfb\x3a\x39\x57\xab\x5f\x01\xaa\xdc\xc3\x3d\xd5\x71\x49\x81\x54\x77\xf3\xf6\xe3\xbc\x5d\x10\x50\x2f\x3f\xe0\xa7\xef\xf1\xcb\x76\x03\x65\x91\x3e\x43\x78\xfc\xb9\x5a\xb5\x27\x1e\x02\xed\x9a\xb2\x25\x91\x23\x21\x22\x76\x11\xb5\x62\x3a\x91\xf7\x33\xf9\xf3\x5a\x48\xa6\xca\x1c\x7f\x82\x6b\xbe\x51\x33\xd3\x52\xaf\x05\xa4\xa9\x1e\xb1\x47\xc1\x36\x7c\x8b\x6a\x33\xdc\x09\x5c\xb6\x14\x58\x32\x70\x8a\x98\xaf\xd3\x44\x1a\x69\x91\x25\x36\x2e\xa1\x3e\xf5\xcf\x71\xe3\xb2\x44\x87\xfc\x70\x1e\xe2\x5d\xe7\x69\x05\x9f\x07\xa6\x32\x8f\x9b\xb4\x00\x21\x92\xdc\x90\x54\x56\xa9\xfb\x32\xf3\x94\xa8\xaf\xac\x73\x12\x86\xfb\x41\x25\x31\x8b\xcb\x2c\x4d\x22\x27\x77\x1f\x55\xde\xc9\xfb\x8c\x01\x34\xfd\x4f\x9d\x7a\x58\xd8\xae\x8e\xb5\x44\xe7\x04\x48\xba\x1d\x0c\xd0\x2f\xcc\x81\xcd\x12\x19\xa5\x25\xa4\x99\x2b\xb5\xc8\x8f\x5c\xea\x68\x17\xeb\xf7\xdb\x27\xc9\xf6\x24\x9c\x87\x87\xb5\x85\x41\xe7\xa9\x5a\x25\x11\x4f\x43\x70\xb3\x47\x45\x38\x16\x5e\xbb\xed\x29\x99\x30\xc4\x41\xd8\x06\x75\x12\x69\x65\xb9\x00\x22\xe8\x39\x88\xf2\x39\x89\xbb\x43\xda\xbd\x64\xe6\x82\x8e\xed\x0a\x39\x72\xad\x7b\xc1\x9e\x70\xbe\x6e\x9b\x89\x0d\x54\x4c\x4c\xe1\xcf\xd4\xa3\x14\x39\x68\xb0\x00\xfb\x30\x3d\x95\x0a\x74\x13\x97\x9d\xcd\xe1\x93\x6d\x76\xc2\xa5\x03\x62\x63\xe4\xec\x2a\x79\x10\xf2\xf3\x93\x9a\x07\x15\x44\x3c\x5a\x8b\xb9\xd5\xcb\x9f\x5b\x64\xb9\x03\x60\xa0\xb0\xb2\x69\x52\x42\x51\xea\xdc\x9b\x70\x75\xc2\x16\x37\x65\x17\x3a\x12\x77\x44\x64\x99\x46\xcc\x63\x11\xdd\x7f\x76\xd1\xec\x41\x56\xb6\x21\x8c\xb3\x77\x22\xba\x67\x77\xd7\x53\x8c\x06\x4e\x0a\x66\x44\x81\x5e\xfb\xb4\x4f\x9d\x77\xb7\x82\xaf\x5e\x80\xc2\xaa\x6f\xde\x2a\x9f\xaa\xc0\x65\xeb\x33\x0d\x22\x40\x14\xc4\x4b\x1a\x21\x49\xb1\x34\x00\x04\xe3\x85\xcd\x66\x04\x86\x78\xa6\x37\x90\xbc\xa8\x2c\x82\x8c\x7f\x29\x5f\x88\xb4\x83\xb8\x33\x53\xf1\xdc\xfa\x49\x0e\x05\xf3\x34\xca\xb2\x76\x0c\xf2\x3a\xda\x38\x06\x6e\x34\xd6\x5b\x7a\x91\xdd\x7f\xa3\x03\x7a\x0d\x15\xf2\x87\xc3\xbd\x9e\x6b\x08\xef\x5e\x26\x2b\xeb\x6d\x4b\x96\x94\x62\x09\x03\xfa\x8d\x1e\x0c\xf2\xd2\x94\x74\xa5\x62\x82\xe9\x39\x2e\x3c\xa3\x05\x09\xb2\x9e\x78\x5c\x45\xd8\x04\x8b\x03\x84\x7a\xcd\x8e\x10\x3c\x66\x6a\x49\xd6\xc4\x2c\x4b\x13\x60\x86\x8e\x91\x84\x1e\xd8\x33\x74\x15\x1d\x1f\x96\x66\x1b\x1b\x90\x7c\x5c\x59\x20\x5e\x57\x32\x5e\x10\x18\x18\xc1\x30\x07\x36\xb8\xf9\x03\xef\x26\x53\x7b\xf1\x8c\x69\x1d\xed\x71\xde\xe4\x2a\x25\x6c\x85\xb4\x8f\x6c\x05\x78\xac\xdb\x80\xfc\x88\xa7\x51\x49\x7e\x32\x48\x97\x6f\xb3\xe0\xef\x46\x10\x7a\xaf\x9f\x99\xe8\xaa\xd5\xbf\xae\x64\x1e\x9a\x5d\xd1\x05\x68\x3d\xd5\xa6\xb0\xdb\xbc\xb8\x4a\xd5\x02\x56\x4e\x37\x4a\x70\xc7\x89\x65\xc4\x75\x9e\xc4\x43\xf4\x1d\x3b\x26\x97\xee\xd3\x5d\x0d\xbc\xb4\xa6\x1f\x57\x93\x5d\xf7\x8c\x12\x19\xd4\x98\x1b\x87\x51\x20\x2c\x29\xab\x6a\xf5\x7a\x52\x50\x1a\x0f\x58\x56\xee\x7c\xea\xb0\x33\x54\xfb\x72\xd0\x44\x37\x99\x62\xf6\x8c\xa5\x27\x97\xd9\x3d\xc9\x07\xd0\x7d\xa0\x28\x73\x9c\x1f\xdd\x96\x45\x19\x8b\x78\xfe\x84\x3e\x9c\xd1\xb7\xfd\xfa\xe2\x46\x1a\x9b\x07\x36\x40\x79\x64\x54\x85\x98\xe7\xb1\xef\xc7\x08\xf6\x7b\xc4\x33\x30\xc3\x83\x5b\xe3\xe1\xed\xd8\xd6\x71\xed\xa3\x8b\x8c\xbc\xc4\x98\x7f\xc4\x6f\xab\x96\x1c\x38\xfb\xd6\x91\x5b\xa4\x08\xef\x36\x2b\xc7\x2f\xd7\x4a\xdc\x4d\xaf\xb5\x5b\x5f\x61\x56\x80\x1f\xb2\xb8\x5e\x42\x76\x94\x85\xf2\xde\x1e\xe8\xcf\x14\x68\x87\xc3\x88\x3e\x10\x90\xd3\xb8\x03\x29\x62\xd5\x6f\x2b\x84\x06\xe0\x8f\x07\x21\xa0\xb3\x5c\x58\xbf\xe1\x56\x14\x8e\xd7\x21\xb5\x79\x05\xc1\x2d\xe6\x7a\x5d\x25\xb6\xb1\xdc\x15\x8e\x8c\x0c\x9c\x58\xa4\xea\x47\x6a\x93\x29\x09\xb0\x24\x8c\x52\x9b\x49\x2a\xdc\x66\x87\x77\x9e\xb5\x4a\xa8\xe3\x88\x0c\x9a\x18\x38\x23\xb4\x4a\x1f\xc8\x85\x1a\x24\x31\x81\xbc\x92\xa6\x81\xa7\xe6\x6e\xa8\x72\x24\xd8\xb2\x27\x3b\x44\x02\xd4\x52\xa4\xe7\x62\x95\xe8\x42\x84\xd1\xa1\xe1\xf7\xcf\x96\xcd\xb6\x62\x3c\xd9\x35\xf4\x9d\xd9\x6c\xf7\xdd\x82\x8c\x7c\x1a\xd0\x9e\x6d\x26\xe2\xa9\xfb\x6e\xf7\x62\xa8\x05\xf0\x7b\x71\x58\x39\xef\x70\x0d\xe0\xed\x4f\x23\xd5\x97\x76\xe9\x47\xdc\x24\x11\x09\x13\xf7\x80\x46\x33\x45\xab\x92\xe7\x5c\x16\x42\xe8\x99\x24\xc7\x33\x52\xd6\x85\xac\x2c\x35\x20\xa4\xbb\xdb\x44\x4a\x17\xc8\x00\x05\x9f\x2c\x79\x92\x96\x79\xa7\xb9\x01\x57\xe5\x93\x68\x27\x76\x8d\xd2\x29\x14\xcb\xda\x26\xcd\x05\x30\x07\xbb\xc8\xb1\xa6\xd4\xdd\xc6\xd5\xf8\xde\x8e\x2e\xd8\xc3\xa5\xff\x7c\x3b\x5b\x73\x47\x4c\xf3\x37\x7a\x9e\xa9\x01\x12\xef\xc7\x6f\xf4\x95\xea\x88\x06\xd7\xbf\x34\x6c\xa2\x3b\xe0\x13\xbf\x74\x25\x64\xe1\xfa\x1e\x3c\x8f\xfb\x4c\x31\xbd\xd8\x38\xf7\xfa\x27\x3b\x65\x17\xac\xda\x35\x97\x71\x6a\x54\x5e\x5e\xd4\x79\xaf\x1d\xce\xdb\x5c\x89\x0a\x2b\x1c\xbb\x83\xfa\x20\x46\x66\x1e\x35\x02\x2c\xf7\x8d\x53\x2d\x32\x73\x27\x96\xb2\x56\x4b\x35\x5e\xb2\x2d\x4e\xc7\xeb\x30\x94\x06\xd9\x6d\xd8\x5f\x5d\x7f\x39\x0b\xdb\xfe\x99\xd4\x97\xea\x5e\x5b\x26\xab\xdf\x80\x21\xe1\x43\xf3\x48\x88\x48\xe6\xd0\x41\xed\xa2\x1b\x0e\x94\x3a\x10\x48\x66\xa4\x76\xc8\x38\x3e\x93\x94\x0e\x1e\xd1\x05\xe0\x56\x46\xbe\x35\xcd\xde\xba\xe8\xe2\xb7\xbf\xb3\x6c\x5b\x5b\xb6\x84\x45\x05\x94\x76\x2a\x8a\xca\x1c\x5c\xff\x64\x9e\x64\x02\x0f\x61\x3d\x88\x48\x06\x54\x0f\x07\xd8\x42\x3d\xb1\x4d\x4d\x72\xf6\xe8\x4a\xa7\x6e\xc1\x0c\x89\x89\xed\xdd\xa1\x4f\xf9\xca\x72\x5d\x30\x5d\x88\xac\x55\xfc\x56\xb4\xcb\x6d\x26\x26\x52\xaa\xa2\x1e\x9f\x32\x58\xbf\xe4\xae\x94\x9e\x5b\x67\xc0\x61\x34\x09\x4c\x46\x7f\xb8\xb9\xbc\x60\x19\xdf\x02\xf6\xb1\x50\x0c\x5f\x05\xc2\xd1\xba\xa0\xda\x37\x03\xd5\xce\x57\xa5\x0a\x8e\xa9\x05\x51\xb7\xfb\x27\xa8\xc6\xa6\xb2\x08\x6b\x86\x96\xa4\x91\x59\xb9\x4a\x8f\xb2\x94\xcb\x00\xde\xae\xc7\xac\x56\x7d\x88\x67\x70\x9e\x4d\x42\x8c\x41\x03\xc0\x5e\x41\x6b\x21\x2f\x5b\x01\xd0\xc0\xbb\x63\x17\xd4\x61\x10\x86\x4e\x19\xb1\x13\xd8\xf9\x01\xb3\xc0\x60\x4e\x04\x64\xcf\xb0\xb0\x0c\x87\xec\xe1\x1a\x40\xb7\x9d\x0c\xe0\x3c\x4a\xb9\xd6\x3b\x51\x3a\x2f\x42\x25\x1f\x44\x2d\xee\x17\x5f\xd5\x76\x22\x8c\x10\xb8\x4d\xf0\x5e\xea\x1e\x03\x5b\x82\x15\x5d\x3e\xe9\x5b\xa0\xef\x07\xd9\x20\x08\xfa\x40\x7c\x51\xf0\x3d\x32\x41\xde\x8b\xad\xb5\x70\x91\xa8\xe2\x1b\x31\x72\xc6\x56\x67\x4d\x0c\x40\x7f\xcd\x82\x67\x12\x50\xb1\xef\xc3\xe6\xb1\xf7\x4a\x8d\x10\x9f\x49\x95\x73\x2c\x96\x87\x08\xa7\x99\x7c\xaf\xd4\x98\xbb\x4b\x2c\xb5\x9f\xc4\x4d\xbd\x42\x42\x45\x01\xe6\xb0\x36\x9d\xfd\xf7\xe6\x0f\x89\xc4\xf4\x84\xc9\xc6\x5c\xa0\x68\x9c\x60\x45\x41\x83\x6c\x36\x7c\xf5\xa8\x59\x8c\x94\x32\x65\xa2\xd7\xe0\x76\x41\x3f\x27\xd4\x4f\x47\x0a\x02\xb2\x72\x2e\xb5\xd9\xc3\xe0\xaa\x11\x0f\x82\xec\xb5\x15\x8c\xc1\xf4\xdd\xb9\x83\x2d\xe1\xbe\xa4\xd4\x1d\x1d\xbb\x2d\xb8\x74\x1c\x72\x39\x07\xb8\xf9\x00\x42\x3b\x32\x70\x7e\xe0\xd9\xae\x60\xd8\x83\x4b\xdc\x37\x4b\x8e\x50\xab\x7e\xa3\x82\x4c\xe6\x90\xc3\xb0\x12\x11\x1b\x8e\xde\x9d\x3c\xf0\xc4\x69\xe5\xb4\xdf\x9f\x72\xa7\xb7\x81\x61\xa0\xa8\xd8\x7f\xdc\x04\xdc\x96\x0e\x32\xe8\xee\x82\x46\xb0\x43\x86\x38\x20\xe5\xc3\x2d\x3d\x66\x37\x42\xb0\x8f\x30\x52\xa6\xb2\x8f\x94\x81\x14\x50\xd0\x05\x4f\x5a\x13\xc4\xc1\xdb\x53\xb9\x54\x87\xc9\xff\x7c\xd5\x40\xd9\x1e\x34\x2a\xed\xed\x3c\x14\xc7\x0b\x96\x7e\xf9\xb2\xb4\x22\xbd\x0e\x86\xda\x5c\x5f\x79\x7b\x13\x05\x1b\xdb\x96\x1a\x95\x0c\xa6\xf8\x29\xc4\x75\xb5\x45\x62\x7a\x39\x42\x32\xf6\x7b\xa9\x1e\x25\xca\x63\xaa\x89\xbd\x36\xfb\x0f\x74\x16\xf4\x0b\xa1\x26\x58\xa2\x34\x7c\x03\xec\xf0\x13\xf7\x37\xbb\x41\x17\x38\xb6\x19\x52\x87\x69\xd0\x77\x29\xe9\x17\x1c\xe0\xaf\x27\x23\xf6\xdd\x88\x9d\x8e\xd8\x78\x3c\x7e\x33\x62\x82\x47\x6b\xdb\x22\xfc\x04\x45\x7f\xc1\x57\xa6\x6c\x4a\xfb\xb3\x0c\x2a\x80\xf4\x80\x46\x3f\xb1\x24\x88\xdc\xbf\x15\x58\xd5\x6c\x17\x30\x34\x9b\xe2\xc8\x08\x2e\x14\xad\x55\xe2\x1b\x05\xc8\x73\x11\xa9\xdc\x62\xd7\x75\xa1\x72\x8b\xc3\x7d\xe0\x39\x4f\x24\x30\x56\xf0\x66\x14\x02\xd5\x1c\x70\xd6\x8b\x4f\x7c\x03\xfd\x4f\xa4\xa3\xed\x35\xc3\x74\xeb\xda\x5f\x6c\x33\xf2\xb3\x3d\xe6\x49\x51\x18\x85\x4c\xcf\xe4\x0d\x3b\xf9\x96\x4d\xb2\x2c\x15\x6c\xc2\xfe\x8b\x7d\xc7\x25\x97\x9c\x7d\xc7\xfe\x8b\x9d\x72\x59\xf0\x54\x95\x99\x60\xa7\xec\xbf\xcc\xb0\x99\xf2\x2e\x94\xd1\x80\xb6\x23\xc6\x99\x2c\x53\x54\xf4\x5e\x5b\x8c\xeb\x1b\xd7\x2f\xee\x67\x67\x21\x8a\x47\x21\x24\xd3\x6a\x43\x47\xe1\x9f\xdc\xe9\xaf\x13\xb9\x4a\x45\x41\xeb\xa1\x8a\x46\xc6\x0a\x8e\xa0\xa7\x27\x33\xe9\xec\xd4\x7f\x32\x2d\xfe\x13\xfb\x2f\x76\x51\xa6\xa9\x69\x92\x11\x34\x66\x21\x9d\x30\x1b\x1d\x26\xe4\xf8\x31\xb9\x4f\x32\x11\x27\x1c\xe2\xc3\xcc\x5f\xc7\xb7\x30\xdb\xf3\xd2\x53\x81\x86\x7b\xda\xa5\x63\x3b\x44\xf4\xbc\x08\xd7\x84\x4b\x16\x18\x6a\x2b\x9d\x20\x94\xf0\xd3\xe1\x4a\xb0\x27\x40\xa6\xfd\x40\x77\x14\x4c\xa5\x17\x3a\x28\xdb\xeb\x77\x59\xbf\x32\xf3\xaf\x56\xfa\x8f\x5e\xd9\xbf\x76\x8d\x87\x6f\x23\x28\xa7\x38\x39\x3e\x38\x13\x2e\x32\x10\x4b\x88\xfb\x6e\x0b\x29\x3f\x6c\xd9\x78\xed\x44\xf7\xb6\xb9\x4a\xa3\x36\x5a\xf0\xd5\x88\x65\x2e\x8f\x94\xdd\x54\xce\xb1\x8d\xfb\x18\x73\x26\x90\xb2\xf9\xda\x02\x88\xcc\x5a\xa6\xf8\xc3\xe3\x58\x6d\x78\x22\xdf\x40\x1d\x96\x3a\x6f\xcf\x40\xb5\x5c\x57\xf6\x8f\xd0\x2d\xdf\x89\x66\xec\xa6\xf6\xaf\x2a\x3b\xb5\x14\x6e\x6d\xdb\xe1\xc0\x1c\x66\x3e\xc3\xe9\x67\xbc\x0e\xfd\xd4\x58\xa2\xbd\x73\x1f\x50\xbe\xb1\x0a\x7b\x0a\xe8\xf2\x9e\x41\xae\x97\x6f\xdd\x65\x2e\xfb\xa9\x9a\xe2\xb5\x32\xc4\x2a\xe9\x95\x0b\xb7\xd6\xd8\x3b\xba\x89\x61\xdc\xb3\x11\x93\x49\x7a\x6c\x44\xe5\xf1\x85\x92\x82\x71\xad\x93\x15\xb2\xde\x81\x41\x0d\x93\xc8\x5a\xa5\xec\xb6\x7a\x65\x08\x44\x10\xe8\x67\xa6\x49\x88\x98\x2e\x8c\x14\x36\x53\x90\x6e\x67\xd2\x7c\x41\x1a\x01\x44\x4f\x25\x8e\x1c\x1d\x6b\x23\xee\x71\x5b\x17\x1d\x88\x41\xe1\x2d\x0b\x6c\x17\x35\xc3\x01\x0b\x8e\x76\xe2\x01\x1e\xb7\x8b\x80\x18\x94\x4a\xb3\xac\x51\x08\xa7\x59\x88\x54\xc9\x95\x59\x15\x5d\x42\x18\xa4\xc0\x33\x35\x01\x0b\xeb\x6c\x81\x51\x56\xe8\x15\x9a\x12\xa3\xa7\x24\xb1\x37\xa9\xe9\x72\x61\xf4\x38\xe7\xed\x71\xda\x08\x75\xae\x8b\xa7\xe2\x30\xd8\xd2\x9d\x91\xc1\x2a\xb7\xc0\x39\xe7\x49\x44\xc5\xc5\x73\x38\x61\x8f\xba\x10\x1a\xf9\x20\x53\x4e\x87\x67\x7b\xc4\x54\x8e\xf4\x9d\xd6\x83\xed\xf8\xb7\x9a\xb5\x77\x6f\xe9\x9d\xd1\x1e\xed\xb6\x4f\xf2\x93\x36\xf8\x42\x7a\xec\x86\x5f\x33\xf0\xa3\x4f\x38\xc8\xfb\xc9\xf4\xbc\xf6\x5e\x33\x1c\xa4\x25\x66\xe4\x76\xfa\xe1\xec\xdd\xfc\xf2\xee\xb6\xf1\x9e\x29\x8d\x7e\xda\x13\x11\xd2\x39\x7a\xcf\x81\x89\xff\x05\x73\x98\xcd\xd5\xd2\xd2\x03\xf4\x3f\x9e\x1b\x59\xe4\xfa\x41\x2f\x8b\xe0\x76\x1d\x66\x5b\x6b\x2e\x9c\x4e\x92\x13\x39\x27\x5f\x6b\xbf\xc6\xd6\x07\xec\x52\xbe\xc7\xcf\xaf\x54\x9a\x44\xbb\x91\xdc\xf6\xb0\x34\x5a\x55\x13\x1a\xbb\x10\x10\xda\x40\x06\x5f\x6a\x14\xde\xcf\x0a\x11\x15\x1e\x4b\xd0\xec\xdc\xff\xd3\xe8\xd1\xfd\x16\x18\xb4\xc3\xba\x61\x83\xe4\xe4\x0e\x9d\x00\x27\x3b\xb0\x46\x43\xb2\x14\xd4\x72\xc1\xb2\x0b\x32\x2f\xe2\xe4\x73\xaa\x8c\x3c\x1c\x0f\x8f\x6b\x95\x92\x3d\x16\x19\xb8\x67\x32\x13\x79\xa4\x00\x75\x89\xe4\x2e\x8a\x45\xeb\x24\x8d\x7d\x46\xb2\xd7\x10\xa6\x02\x60\xf2\x37\x94\x5c\x57\x38\xf4\x8c\x2d\x7e\xc7\x99\x6f\x97\xdd\x3b\xdc\xdd\x07\x21\xcf\x9e\x13\x77\xbe\x6b\xd9\xff\x4c\xf8\x68\x1c\x0a\xe2\xcc\xab\xe1\x20\x40\xed\x0f\xdb\x33\xc8\xa5\x63\x0e\x7b\x4a\x36\x15\xf9\x6b\x73\x51\x9b\x57\x5a\x66\xf5\xa1\x04\x26\x75\xb4\xa3\x23\x08\x50\x0b\x68\xce\x46\x70\xd4\x04\x3d\xaf\x31\x4d\xea\x4c\x7a\xe4\xc7\x2b\x1d\x6a\x85\xad\xf3\x8c\xd6\x77\x8b\x6c\x1f\xb1\x57\x95\x8e\xbe\x02\xa6\x6d\xa9\xa0\x3e\xf2\xce\x57\x86\x06\x96\xeb\x88\x25\xc5\x4c\x9a\x3b\x9b\x59\x99\xb9\x48\xc5\x83\x69\x5d\xe8\x1d\x22\xbc\xa2\xb5\x9c\xd8\x6e\x43\x70\x14\xb7\x9c\x1a\xb4\x6c\x68\x13\xe6\x21\x63\x33\xba\xa5\x63\xa1\x8d\xd6\x0a\xb9\xa6\xc4\x27\xb3\x01\x12\x70\x7e\x22\xb0\x2d\x16\xd2\xb6\x0f\xf0\x6e\x98\xe7\x7f\x26\xa7\x4b\x20\x36\x00\x3a\x85\x38\x46\x1b\x84\xcd\x3e\xe4\xe8\x33\x13\xf2\x06\x29\xb2\xc8\xd8\x89\xa0\xdc\xd0\xb8\x93\xc4\x83\xc8\xb7\x05\x98\xf4\x61\x5c\xa5\xe0\xc5\x9a\x25\xc5\x08\x78\x4f\xad\xa4\x9c\x49\x1e\xc7\x14\x0f\x8e\xc5\x05\xd7\xd9\xce\x79\xa6\xe7\x0b\xf5\xb0\x4b\xad\x3e\x14\xb9\x8b\xbb\x3a\x4b\xb9\x9c\xe3\x09\xf2\x2b\x60\x77\x83\xb4\xdd\x5d\x20\x8e\x72\x31\x77\x5c\x6d\xcf\xd2\x4e\x27\xef\xaf\x2d\x74\x99\xae\x36\xe5\xc2\x56\x34\xaa\x40\xb3\x17\x9e\xd6\xc3\x59\xe9\x08\x37\x95\x33\x8b\x2d\xe9\x2f\x05\x3c\xac\x97\xd7\x30\x56\x76\xb5\xee\xc3\xf5\xda\x15\xf0\x5b\x45\x5e\xf6\x99\xf9\xda\x19\x52\x9f\xf6\xe1\xa0\xbf\x86\x86\xf8\x24\xe0\xdf\x9e\x66\xbd\x2c\xf8\xaf\xd3\x8a\xd3\x04\x01\xda\xde\x06\xfe\x7d\x0a\x1d\x40\x2b\xac\x33\xa0\xb5\xa7\x65\x0f\x6f\x81\xaa\x05\x1f\xff\x9c\x1e\x0a\x90\x53\x7d\xed\x34\x9e\x50\x04\xda\x35\x66\x53\xc9\xac\xba\x37\x62\xaf\x70\x61\xe9\x57\x64\x80\xa6\xdc\xfe\x04\x96\x89\x69\xf7\x10\x05\x43\x1d\x64\x86\x81\x70\x7e\xbb\xa1\x1f\x70\x27\x5f\xef\x8b\x8e\xcb\x77\x09\x04\xe2\x3d\x85\x6b\x05\x7d\xc8\x0b\x2c\xc0\xc6\x91\x04\xa6\x50\xea\x2e\xf8\x32\x7c\x87\xad\xb7\x93\x7d\x67\x3f\x34\x43\x94\x95\x74\x9e\xda\xe7\x4c\xe5\x33\x69\x4b\x23\x83\xb4\xc6\x04\x81\xf5\xa2\x82\xb8\x20\xd2\xf9\x83\x95\x0a\x50\x00\x9b\x13\x12\x52\x8d\x7a\x52\xf1\xba\x14\x00\x14\xd4\xc2\x21\x50\x21\x0b\x85\xaf\xcd\x28\x1e\x66\x81\x6f\xf0\x98\xaf\x13\x0f\xa7\xa9\x19\x94\xa4\xb0\x3c\xc7\x41\xcc\x9e\x2e\x81\xad\x7b\x59\x1a\x61\x14\x50\x9a\xcf\xa4\x19\x3c\xb6\x4c\x20\x76\x83\xc6\x65\x26\x3f\x28\x6d\x29\x62\xb4\x1f\x0f\x0b\x2c\xa0\x61\x7b\xe5\x52\x63\xd2\x0f\xef\xe0\xd0\x26\x8f\x0f\x92\xbd\xb9\xa3\x05\x82\x35\x89\xe7\x69\xab\xca\xdc\x77\x2a\xe2\x72\x26\xff\x6a\x86\x07\xae\x53\x5c\xda\x69\x55\x4b\xdc\xc2\x30\x83\xe0\x2a\xfb\x88\x85\xbe\xfe\xdd\x9b\x8f\x6f\x30\xb8\xaa\xd4\x90\x8d\x78\x54\x3d\x40\x5c\x76\x8b\x32\x4d\x01\x87\x60\x7b\xe0\x18\x96\x7c\x15\x3b\x71\x78\x74\xa9\x9b\xcb\xaa\x8a\xd1\x67\xa3\xf7\x33\xeb\x4f\x58\xc4\x8b\x68\x7d\x64\x75\x39\x12\x63\xf6\xf4\xa3\xe9\xc3\x28\x28\xa3\x69\xb1\xd6\x04\x0f\xe6\xc2\x99\x6f\x1c\xe5\x6c\x65\xbd\x98\x2e\x80\xf9\xff\xb6\x9e\xed\xcc\x31\x62\xe3\xe2\x44\x1c\x50\x55\xcf\x73\xaf\xdb\x5c\xa3\xfe\xc6\x49\x3e\x12\xc9\x37\x22\x66\xaf\x20\x0c\xf8\x95\x9d\xfc\x99\xcc\x16\xe3\x74\xbb\x2c\x88\xb7\xd0\x0c\xca\x18\xb2\xf2\xed\x39\xe5\xe6\x71\xf3\x9a\xb4\x67\xb0\x3b\x2f\x5a\xed\xba\x8e\x1b\x1b\x57\x53\x7f\x85\x05\x6d\x5c\x6e\x74\x6e\xaa\x98\xc0\x6a\x7a\x10\xae\xef\x47\x6c\x91\x73\x09\x09\x95\xe2\x50\xa9\xf2\xbb\x13\x2e\xcf\x48\x0a\x68\xe3\x02\x25\x4f\xb7\x10\xff\x33\x9a\x49\x64\x50\x04\xaa\xfd\x6d\x94\x26\x11\x5b\xe5\x3c\x5b\xd7\xf4\x20\xf1\x20\x64\x01\x79\xb9\xaf\x05\xd7\x87\x61\x35\xf2\x7a\x09\xac\xb7\x37\x6d\x22\xe1\xf6\xc1\x65\x8d\xf3\x1a\x9a\xd7\x71\xb4\x00\x3e\x53\xc4\xf3\x61\x7c\x57\x7b\x59\x99\x2b\x5c\x9f\x44\x3c\x07\xfe\x67\xd3\x39\x66\x6b\xdd\x07\x7e\xc0\x71\x25\x2a\x26\x8b\xe8\x3d\x14\xb0\xe1\xa8\x9d\x0e\x22\xe8\x9d\x56\xb5\x48\xee\x39\xab\xbc\xdd\x9c\xdc\x7e\x68\xa9\xb0\x61\x10\x4e\x70\x8c\x28\x6f\x2a\x90\x77\xb2\x3f\x96\x0b\x95\x5a\xf6\xd3\xe9\x3b\xa6\x72\x48\x3c\x54\x28\xfa\x29\x89\xbb\xb4\x83\x44\xc6\xe2\xd3\x41\x14\x44\xbb\x0f\x7a\xab\x36\x9b\x6a\x82\xfc\x36\xf5\xce\x82\x74\xca\x85\x39\x84\x0b\x7b\x33\x6e\xbc\xa5\xeb\x50\xe5\x49\x5a\xac\x01\x3f\x8c\x21\x3a\x7e\x50\x37\x7c\xcb\xa2\x35\x97\xab\xc0\x34\x01\x70\x4e\x91\xa9\x1c\x13\xf4\x3e\x00\xd7\xa7\xca\x2d\xc5\x03\x11\x17\x50\x9c\x90\x73\x63\x20\x3c\x5f\x59\x76\x02\xbe\x5a\xe5\x62\x05\x61\xac\x33\x59\xa1\x5e\x01\x9e\x53\x9b\x1b\x08\xeb\xd9\xc5\x5c\xf1\x3c\xf4\x4f\x5d\xb7\xc1\x22\xdf\xba\xb8\x7f\xca\x6e\xed\xf7\x73\x7d\x58\x47\x2c\x11\xe3\x11\xfb\xca\x87\x24\x88\x48\x49\x47\x1c\xd0\x11\x35\x5e\x33\xf9\xb3\x3d\x57\x87\x26\x4f\x54\x7b\xdb\xe1\x59\x23\x47\x76\xeb\xa2\xd9\xc9\xbc\x50\xf0\xa2\x1c\x70\x06\x9d\xf2\x82\xa7\x6a\x75\x6a\x3e\xbe\xc1\x6f\x77\xad\xeb\x53\x8c\x17\xb0\x1c\x7d\xe6\x7d\x73\x72\x9a\xba\x3d\x87\x7f\xdb\x58\xef\x35\x20\xa7\xaa\xdb\x80\xfc\x1c\xaa\xba\x25\x62\xda\x6f\x43\x4e\x3b\xc8\x85\x76\xf4\x69\xa8\x89\xd8\xa2\xfa\x29\x30\x49\xd7\xaf\xb1\x2d\x12\x20\xcb\x55\x5c\x46\x22\x36\x3b\x17\xee\x43\x88\x87\x72\x1c\x47\x15\x21\xd9\x76\xd0\x56\x88\xda\xe0\xd4\xfd\x5c\x36\x87\x5e\xdc\xf8\x6e\xf8\xef\x3a\xec\x0d\x56\xe3\x6b\x1b\xf4\x70\x7f\xe2\x38\xe5\x03\xcf\x29\x57\x7d\x95\xd1\x5e\xe5\xc9\x2a\x91\xbc\x50\x39\x7b\xed\x98\x0c\xde\xb8\x34\x78\xdd\x1a\xc2\x40\x31\x51\x19\x22\x14\x13\x9f\x55\xf1\x68\x5b\xa4\xe6\x2d\x5d\xf0\x4d\x16\x72\x44\x83\x15\x38\x18\x99\x14\x07\xc1\xe9\x26\x60\x3b\x4d\xb4\x8f\xda\x9d\x49\xf2\x38\xe0\xbc\xa9\x3c\x4c\x72\xd0\x79\x36\x67\x65\x31\x7f\x22\xef\x19\x7e\x3c\xcc\xf0\x44\x20\x88\x0f\x3c\xdb\xcd\x24\xc5\xc9\xe4\x80\x61\x8b\x64\x8e\xf0\x9a\x4a\x75\x7d\xee\x4e\x23\x34\x90\xd9\xba\xee\xb8\xbf\x3e\xb7\x8e\x22\x7f\x1f\xac\x5c\xb0\x60\x22\x90\x52\x17\xc3\xc0\xf0\x6a\xef\xc4\x9a\x39\xc5\x2d\xfd\xd4\x69\xaa\xca\x98\x91\x50\x23\x10\x40\x3e\xc6\xd3\x11\x38\xae\xc7\xe3\xae\xb0\xb6\x81\xe9\xcd\x9d\xfc\x81\xef\xda\x77\x20\x3c\xeb\x90\xc0\x3b\xb7\x3e\x8d\xec\x8b\x4d\x3d\x8d\x34\xcc\xbd\x13\xc7\x83\xe6\xde\x59\xc1\x81\x70\x73\x98\x81\x14\xee\xa3\x49\x9c\xc2\x7e\x0b\x1d\x08\x2d\x94\xe0\x15\xc7\xac\xbe\x3f\xb8\x3a\xcb\x42\xb1\xbb\xaa\x8c\xe7\x42\x16\x73\xa8\x71\x58\x65\x50\xc9\x15\x7c\x5e\x51\x98\x7a\x19\x82\xff\x7c\xab\xd0\xbe\x6f\xd9\xb5\xfe\xc2\x6e\xc8\xa6\x65\xe4\x55\x02\x10\x62\x7d\xcf\x5e\x27\x80\x78\x0a\x7c\xa1\x6e\xe2\x3a\xa6\x8b\x3a\xf4\x84\xd1\x0b\x3a\x54\x11\xed\xbd\x3a\xe4\x5b\x0f\xae\x6a\x28\x85\xcc\x7b\xc4\x09\x60\x44\xad\xfd\x2d\xc8\xb8\x71\x51\xf9\x1b\xd8\x91\xcd\xfc\xa5\xec\x6f\x22\x57\x3e\xfe\x0b\x8d\x55\x61\xc1\x3b\xf5\xf5\xa7\x27\x0b\x47\x7d\x1c\xd3\x54\x87\x79\x5a\xe1\x17\x22\x30\x43\x8b\xc2\x62\x6b\xaf\x23\x1d\x2e\xa4\x4c\x44\xf3\x8e\xa4\x3c\xbd\x9a\x12\x5c\x3c\xc3\x24\x3b\x49\xed\x30\xb3\x1b\xf4\x18\xec\x15\x14\x58\xb5\xe1\x19\xa1\x0b\x09\x48\x5e\x77\xde\x8c\xa1\x13\x7f\xfe\xd3\x5f\xc6\x49\x47\x88\x37\x34\x7d\x28\x58\xcb\x35\xfe\x7d\x9e\x08\x19\x83\x33\x96\xc7\xcd\x7c\x71\xb2\x62\x9d\xaf\x88\x67\xb3\x0c\x9f\x25\x1e\xbc\xfd\xa8\xd5\x73\x5c\x44\x9f\xc1\xa3\xef\x85\xac\xdb\xbe\x15\x7f\x5f\x97\x2a\xa1\xe7\xf1\x56\xf2\x4d\x12\x7d\xd6\x36\x6e\x13\x91\xc6\xd0\x44\xaa\x7d\x9f\x57\x2a\x16\xd1\xfd\x50\x9d\xe0\xc9\xd9\x2e\x44\x74\xcf\x7e\xb8\xfd\x70\x8e\xc9\x8d\x13\x3d\x93\x17\xbc\x48\x1e\xc4\x5d\x9e\x3a\x77\x00\x81\xb4\xf3\xd4\xee\x91\x2a\xfb\x7a\xc0\xf4\x65\xa9\xda\xad\xe2\x10\x26\xc7\xd8\x6c\x8f\x16\x65\x74\x2f\x8a\xe3\x9c\xcb\x58\x6d\xb0\x1b\xc7\xba\x5c\x2e\x93\x4f\xe3\x82\xe7\x1d\x99\x32\xd0\x8e\xf0\x2b\xea\xb9\x3e\xff\x59\xe1\x75\x5e\x54\x75\x1f\x21\xcc\x9b\xb2\xea\x57\x94\x5b\x8c\x4a\xe4\x1b\x01\x54\xa7\xac\x9a\x65\x06\x4a\xc1\xc8\x69\x48\xc6\xaa\x35\xc5\x4f\x28\x4a\xf5\xfe\x31\x50\xee\x3f\x06\xad\xf2\x2e\xec\xb0\x51\x3e\xc1\xe9\x86\xdf\xe3\xfd\x70\x95\x0b\xad\x47\x4c\x2b\x68\xf1\x4c\xda\x48\x04\x1b\x2d\x07\xb8\x17\x20\x4b\x4e\xb7\x2c\x52\x99\x83\xcc\x63\xbf\xd6\xea\x11\xec\xf4\x61\x9c\x30\xa4\xf0\x2e\x65\x91\xa4\x8c\x2f\x0b\x32\xe2\x43\x66\x08\x9b\x09\x4e\x8f\x67\x12\x5c\xb1\x11\x74\x1f\x20\x12\xce\xfd\xe2\x3a\xa1\xd9\x92\x47\x49\x9a\x14\xc4\x57\x07\x21\x66\xdc\xf4\xd7\x9c\x07\x66\x2c\x73\xbe\xe5\xa9\xbf\x58\xf1\xb4\xf4\xa1\xd1\x47\x5a\xec\xe0\x43\x4d\xf4\x1c\x0d\x04\x2f\xb7\xc1\x3d\x0a\x30\x09\x9d\x0f\xc8\x1d\x3f\x31\x95\x5f\xd4\x4e\xd1\x7f\x0a\xff\x5b\xb9\x87\xef\xd2\x0a\x0e\xb8\x90\x1f\x72\x38\x36\xaf\xdc\x2e\x7d\xba\xd7\x33\x92\xd8\xa2\x93\x2b\xaa\xb8\x0f\x3e\x76\xc7\x23\xf8\x4c\x3a\x2e\xfd\x63\x9b\xf4\xae\x59\xc3\x80\xd1\x6b\x57\x12\x3f\x93\x39\xa3\x8b\xd0\xbf\x4f\xf3\xad\x35\xfe\x4a\xa9\xf4\x50\x8b\x3c\x51\x72\x24\x4a\xce\x21\x0f\xf4\x21\xd7\x49\x5c\x00\xce\xb0\x35\x7d\xe7\x7c\xee\x8e\x21\xbf\x9a\x3d\x8e\xe0\x60\xd4\x04\x10\x64\xd0\x88\x1d\x38\x75\x9d\xb5\x80\x2e\x06\xe2\xed\xa1\x0c\x44\x6b\x59\xd5\xbe\xe9\x22\x08\xd8\x59\xb8\x6f\x23\xb0\x08\xd7\x5a\x38\xc8\x58\x87\x59\x9b\x6b\x55\x39\xc3\x5d\xc8\x36\xee\xc6\x31\xa8\xdb\x8e\xe7\x86\x4b\xb2\xfc\x91\x16\x3f\x93\x81\xc6\x8e\x8c\x78\x36\xa0\xc1\x8d\x5a\x9b\x3d\xaf\xb2\x0c\x0f\xb6\xe7\x1d\x92\x52\x62\xa7\xe4\x7c\x17\x26\x87\x04\x2c\x48\xa4\x36\x8b\x44\x5a\x4e\x0a\x32\x72\xc3\x55\x63\x62\x19\x7b\x9d\x43\xc2\x5e\x19\x30\x65\x50\x6d\xec\x9d\x9a\x13\x92\x1f\x87\x22\x6b\xdf\x75\x3c\xbc\xdf\x3d\x6f\xf6\x8b\x0e\x4f\x63\xbd\x07\xe6\x00\x49\x1f\xf9\x56\x43\x02\x75\x61\xa4\xe2\x12\x0d\xbb\xd5\xf6\x8f\x02\xf5\xc3\xb2\x41\xcf\x24\x8c\x10\xb2\x85\x59\x41\x6a\x24\x2b\x2c\xc0\xd4\xa6\x8a\xf7\x4c\x6f\xaf\x74\xfb\xe0\xfc\x3a\xbe\x9a\x7c\xa7\xaf\x06\x9d\xd0\xff\x3d\xdc\x33\x3b\x8c\xc0\x07\xda\xa2\x83\x63\x12\x35\x46\x82\x09\x41\xd8\x98\x73\x51\x8f\xd8\x86\x27\x92\xb6\x01\xa6\xe3\x8c\xc5\xa2\x5c\xad\x3a\x4d\xa4\xbf\x7d\x5f\x4b\x75\x9f\xfc\x8f\xb7\x85\xef\xe4\x2a\x7c\x0e\x6b\xf1\xd4\xd6\x84\xe6\x6b\x73\xef\xfb\x3c\x06\xe2\x5f\xd1\x1a\xdf\xea\x12\x6b\x2c\xa2\xe7\xb1\xc6\x4f\xfb\x58\xe3\x2d\xb6\x0b\x02\xfc\xe8\x3a\x6d\xf1\x37\xff\x30\xd3\x7f\x1e\x33\x7d\xaf\x45\x81\xa4\x3e\xf3\xa4\xaa\xa0\xef\x68\xe1\x13\x79\x2f\x1d\x15\x34\xb4\x0a\xb9\xe5\x8c\x74\x8f\x35\x5b\xf0\xe8\x05\x88\x30\xe1\x74\x3c\xdc\x1e\xb8\x07\xfc\x72\xa3\x36\x82\x41\x55\x1a\x13\x39\x31\x8a\x62\x1c\x01\x5a\xd5\x74\xd0\x23\x46\x08\x8f\x02\xc7\x29\x22\x57\x62\xaf\x54\xbf\x96\xe2\x91\x99\xd3\x6a\x14\xc2\xf7\x82\xe9\x81\x0c\x7f\x6f\x8c\x76\x58\xc1\xfa\x3b\xc2\x8e\x5c\xac\x78\x1e\x43\x84\x09\x6d\xc9\x94\x47\xf7\xe6\xdf\xd0\x3e\xaa\x91\x20\x86\x96\x2b\x00\x61\xaf\xbe\xb4\x44\x46\x48\x85\x68\x39\xdd\x5d\xfb\xf0\x73\xcd\x78\x94\x2b\x8d\x46\x23\x97\x18\x1b\xe2\xab\x41\x81\x7d\x48\xe2\x92\xa7\x58\x63\xa7\xa5\x7d\x28\x7c\xad\x0e\x38\x0a\x72\xd8\x35\xd1\x6c\x34\x1d\xc8\x50\x05\xc3\x38\x9e\xc9\x77\xce\x61\x72\xc2\xee\xb4\x20\x94\x99\xb6\x59\x00\x76\xb6\xf4\xc5\xd4\x87\x06\x26\xb0\x53\x87\xd8\x31\x00\x16\x64\x1d\x0c\x84\xee\x1e\x89\x3d\x74\xaa\x87\x4c\xca\x60\x5a\xe8\x69\x90\x48\xdf\x0f\x0b\xde\x13\x72\xc1\xe3\x6d\xc8\xc5\x98\x48\x06\x5e\x3a\xc6\xe3\x4d\x22\xcd\x26\xb0\xc9\x5a\xdd\x49\x63\xf3\x36\x20\xe4\x18\x72\x9a\xa5\x69\x4d\x08\x6a\x26\x85\x51\x2e\x79\x9e\xa4\x5b\xb8\x4f\x64\xb9\x38\x0a\xea\x09\xe6\x87\x22\x9e\x20\x03\x05\x91\xc8\x94\x5a\x2c\xcb\x14\x6f\x1d\x70\x2f\x77\x1d\x20\x89\x74\x37\x1d\x19\x85\xa3\xa0\x4c\x42\x41\xc5\x98\x9f\xf3\x39\xa2\x47\x1a\xde\xca\x61\x1e\x37\xcf\x15\x9a\x03\xc8\x7d\xad\x1e\x6d\xa8\xdb\x23\xf7\x58\xe6\xae\xd3\xf5\xd9\xbc\x2c\xbb\xf5\x50\x7b\x03\xb4\x72\x2a\x20\xfc\x73\xae\x35\x7a\x26\x62\x27\x9b\x12\x09\xdd\xa1\x14\xd7\xde\x72\x5d\x6a\x8c\x98\x33\x73\x09\xe7\x97\x35\x74\x54\x0d\xd7\xcc\xf5\x2e\xd1\x4a\xb2\x59\xf9\xe5\x97\x5f\x0b\xf6\x25\x84\x10\xd2\x7d\x04\xfd\x63\xc0\x16\x8a\xa5\x83\xc8\x76\x15\x08\xa4\x12\x6d\xcc\x08\x6b\x83\xa8\xda\x78\x7d\x00\x79\xf2\x68\xcd\x74\xb9\x40\x04\x23\x27\x17\x0b\x97\x8e\x75\xfc\x5c\x01\x18\x11\x4f\x76\xdb\xfa\xff\x47\x1c\x0a\x98\xf4\x65\x26\x33\x85\xc4\xf8\x00\xfd\x5c\x08\xb6\xe1\xf9\x3d\xe4\xf0\x45\xf3\x3c\x24\x02\x78\x9d\x88\x71\xd5\xbd\xf0\xa6\xd2\x1e\x72\xe8\x20\xe1\x35\xcb\x4b\x29\x6d\x52\x32\x66\x14\x53\x6f\xeb\x1f\xcd\xe4\xa2\x0c\xef\x9e\x15\x67\x81\x5f\x5a\xe0\x30\x00\x61\xab\x80\xa9\x84\x1a\xc5\xb5\x6f\xd7\x98\xf5\xf0\x1a\xcc\xe4\x33\xbb\x0d\xf6\x19\xfc\xae\x48\x07\xb3\xc6\xbc\x20\x5e\x01\xba\x1b\xe6\xcd\x86\xe9\xc0\x65\x0f\x4a\xce\x15\x24\xcf\x1e\xb1\x1f\x92\x07\x31\x62\x37\x19\xcf\xef\x47\xec\x1d\xba\xff\xfe\xa0\x16\x6d\x36\xbc\x06\xa1\xc4\xc1\x76\xbc\xa7\x99\xb1\x76\xd1\xbc\xb4\x6b\xff\x3f\x37\x88\x01\x58\x97\xef\xfb\xbf\x27\x22\xaf\x83\xeb\xe3\x7f\xba\x25\x62\x8f\x9b\xfa\x1f\xe0\xb5\xff\x91\xb7\xe2\xdd\x34\x1f\xff\x14\xfe\xd7\xca\x2f\xab\x71\x81\xee\x49\x52\xae\x15\x95\xf6\xdb\x0a\x6c\x4e\xe2\xfa\xa1\xdc\x8c\x6f\xee\xb7\x15\x28\x7c\x3c\x76\xa1\xed\x03\x40\xf7\xf4\xa9\x1d\xaf\xd3\x54\xe9\x32\xdf\xbd\xf9\xaf\xab\xad\xb6\xb5\xb7\x10\xbd\xc2\x62\xdb\x2c\x04\xb0\x16\xf4\x85\x9f\xe0\x6b\xf3\xbf\xaa\xc5\x1c\xb0\x56\x87\xed\xf0\xb6\xe2\x1c\x7d\xb4\x8a\x2a\x4d\xf5\x27\xe4\x4d\x26\x80\xef\xca\xab\xa2\xde\x21\x50\x5b\x61\xce\x34\x32\x93\x96\x71\x1f\x23\x66\xf3\x5c\x00\x35\x78\x2e\x20\xd1\x23\x23\x86\xc3\x74\x1b\x68\x44\xc1\xcd\xc7\x83\x62\xc2\x28\x37\x08\x56\xa5\xfb\xd6\x42\x08\xe9\x46\x7b\x88\x2a\x01\x34\xd8\xb5\xd1\x27\xb4\xdb\xa3\xb0\x89\x17\x3a\x92\xd2\x36\xbe\x0b\xee\x82\xa0\x72\xaf\x44\x11\x48\xf3\x9a\x6a\x51\xd9\x9a\x15\x0f\xd5\x6f\x0a\xf1\xdf\xea\x83\xae\x91\x73\x55\x0c\x28\xbd\x7c\x7a\xcf\x61\x2f\xbf\xe2\xc5\x1a\x2f\xb4\x1b\x55\x08\x94\x99\xc8\x12\x84\xeb\x05\xad\xce\x8b\x54\x2d\x20\xc3\x62\xb1\x83\x41\x32\xa2\xad\xdd\x6b\xe8\x9a\x13\xd6\x47\x32\x18\x69\x02\x91\xb6\xb9\xd0\x40\xb8\xd2\xf4\x52\xf5\xc5\x27\x0f\xbb\x74\x37\x9b\x6b\x84\xfe\xbb\xc6\x65\xbb\x99\x92\xc3\x6c\x6b\x00\xab\x9e\x3d\x21\x82\xa6\x91\xe0\x84\xa8\xaa\xc9\x0d\x8c\x6c\xb5\xb5\xfe\xda\x44\xfe\x33\x39\xc1\x27\xc1\x21\xc0\x7d\x8e\x2d\x87\x07\xa5\x9c\xcd\x6e\xff\x61\xf8\x2a\x9b\x84\x08\x44\xb2\x10\x8c\xbc\x2d\x13\x2e\x03\x23\x88\x6a\x94\x45\x92\x0b\x26\x01\x85\x30\x93\xba\x5c\x1c\x79\x62\x12\x73\x8b\x7b\x00\x32\x1d\x2d\x32\x0e\x57\x19\xe0\x2b\x3a\x6a\x39\x86\xd1\x32\xe9\x73\xe5\x58\xfa\x40\x9e\x92\xf0\x87\x58\x49\x8c\x8c\x77\x7d\x77\xe5\x98\xcb\x1a\xdc\xa2\x2d\x5c\x09\x0f\xbb\x5d\xf2\x02\xb2\x79\x41\x04\xe6\x35\xa2\x28\x7e\xed\x03\x3c\xf4\x86\xf6\x3d\xba\xc1\x9f\x36\x93\xff\x6a\xcf\x86\x6e\x50\xf1\x80\x95\x6e\x46\xc6\x1c\x51\x9d\x60\xe7\x4a\xdb\xec\x15\x32\x50\x02\xbb\x1b\xd5\x58\xf2\x6d\xa5\x72\x8b\x6b\x09\x53\xba\x28\x0a\x97\x85\xa7\x0f\x89\x0e\xc8\xde\xa1\xb6\x1b\x21\xd8\x49\x2e\x96\x27\x1f\x73\xb1\x9c\xdb\x99\x1e\x43\x87\xc6\xa6\x47\x4d\xca\xf7\x9e\x8b\x43\x67\x4a\xb6\x93\x1f\xee\xa1\x46\xad\x75\x09\xcb\x09\xfa\x94\x2c\x99\xcf\x6e\x6b\xfa\x03\x0c\x10\x22\xae\x73\xd1\x37\x5a\xf6\xd9\x8f\xb9\x2e\x24\x58\x0f\xa8\x55\x47\x12\xd4\xff\xf9\xc7\x5b\x65\xcc\xfa\x1c\x6f\xb7\x55\xc8\x8c\x15\xf6\x5c\xba\x03\xaf\x1b\x17\xfa\x79\xd1\xe9\x30\x81\x3a\xe3\x8f\x92\x78\x6c\x06\x99\x9e\xfa\x1d\x6b\x35\x00\x51\x70\xac\x35\x30\x70\x7e\x97\x49\x6b\xe9\x4b\x5c\x1e\xcd\x11\xf3\x37\x68\x9e\xa6\x61\x46\x0d\xef\x69\x9b\x49\x1f\x97\x6a\xb4\xd6\x34\xb5\x26\xbc\x8a\xbe\xe1\x12\x1e\xeb\x82\x17\x62\x64\x49\x57\x88\xae\x90\xfc\x61\x47\x0b\x0e\xa9\xad\x5d\x0e\xb5\x7d\xbb\xf9\xb9\x2e\x91\xbf\xb1\xb8\xe8\x3d\x9e\x67\xac\x76\x7e\x2f\x1a\x70\xe6\xbd\x6d\x6d\xf7\x74\x04\x94\x12\xb0\x99\xad\x94\x8d\x78\x9e\x5b\x94\x3f\xd5\xca\x2c\xdd\x79\x78\x2b\xe9\x68\xe7\x5a\x44\xf7\x99\x4a\xe4\x60\x59\x54\xa1\xb8\x80\xc5\x5e\x30\x5f\x9a\xbb\x1d\xf6\x3a\x1c\x2b\xfa\x24\x76\x44\x03\xbc\xc2\x42\x43\x3d\x19\x1b\x67\x2e\x53\x76\xf7\xb2\x7b\x6e\xfb\x85\xf0\x67\xc3\x0b\xd8\x62\x5b\xfc\x43\xb5\x53\x85\xb7\x18\x76\x2a\x4c\xa0\xbc\x11\xfd\xd5\x73\xb0\x39\xab\x50\x18\xb6\x0e\x29\x98\x20\xff\x61\x19\xfa\x87\x65\xe8\xbf\xb9\x65\xe8\x73\x9a\x85\x00\x1b\xf3\x92\x36\xa1\x1d\x0e\xf2\x03\xb6\xa3\xab\x75\x70\x8c\x63\xab\x76\x3c\x0a\x92\x7e\x07\x91\x8e\x4d\xa0\xbf\x25\xc2\x30\xe3\xb3\xe0\xd1\xbd\x90\x9d\x3e\x7a\x4b\x5f\xd4\x99\x7f\xf5\x79\x11\x2c\x6d\xec\x4b\xc1\xd7\xbb\xa1\x2c\x1e\xea\x44\xa4\xc1\x6d\x84\x20\x66\x9f\x80\xee\x69\x3a\x7e\x04\xa0\x31\x95\x3b\x62\x6b\x4d\x51\x78\xe8\x8c\x44\x9a\x24\x04\x4b\xd5\xa8\xa0\xfb\x62\xe2\x6c\xc5\xf3\x4c\xa9\xb4\x15\x1a\xf7\xac\x03\xd8\x08\x94\xe9\x3b\x78\x53\x54\x46\x75\x08\x18\xb3\xa3\xe8\x83\x2e\x7c\x88\x06\xc6\x63\x40\x26\x0e\x58\x4d\x71\x09\xb1\x94\x7e\x38\x82\xf4\x8a\xdc\x19\x5c\x08\x23\xb6\x10\x11\x87\xc4\xaf\x16\xbc\x17\x71\x17\x7d\x12\x92\x22\x35\xc2\x41\x74\xb3\x9e\x0e\xaf\x25\x94\x3b\x4f\xda\xd2\x6e\x0c\xdd\x5c\x35\x0d\xc1\x42\xcb\xb1\xe5\x16\x49\x62\x69\x17\xf7\x25\x34\xb6\x1c\xd3\x73\xc8\xbe\xd8\xef\x84\x6b\x95\x3b\x53\x2a\xe8\x14\xca\xe9\x2f\x48\x7f\x80\x70\x9c\x4d\x4f\xe4\xce\x4c\x4e\x5c\x9e\x5b\x8f\xfd\x72\xc8\x3d\x74\x97\x22\x66\xb1\x31\x35\xc8\xe5\xe8\x6f\x2e\x23\xa6\xcb\x68\x0d\x6c\x95\x55\x39\x15\xca\xad\xe6\x8e\x1d\xcd\xa4\xb9\x10\x81\xa9\x65\xc3\x21\x2e\xfe\xd1\x28\xab\x3a\xf9\x9b\x70\xf0\x2c\x22\xef\x0a\x11\x59\x78\x71\x52\xb2\x15\xbd\x66\x89\x43\x11\x60\xe1\x31\x25\x65\x16\xf3\x42\x8c\x67\x1e\x6d\x93\xa0\xa5\xd3\xa2\x3c\x48\x65\xd6\x61\xc7\x42\x1c\x63\x4d\xd2\xa6\xc9\x52\x44\xdb\xa8\x91\x85\x68\x37\x4d\xc4\x3f\xae\x6d\xbf\xad\x6b\x1b\xb2\xec\x62\xcc\xe0\x90\xa1\xa5\xa6\x5e\xfb\xcf\x0f\x1b\x5c\xc1\x82\x96\xe8\x01\xe3\xfc\x19\xaf\x9d\x2d\x3a\xf0\x30\x7d\xbe\xf7\x3d\x68\xf7\x71\xe6\x2f\xb6\xfe\xb0\x0e\x28\x10\x1a\x6a\x61\xe8\x5c\x2c\xc2\xa5\x63\x14\xda\xde\x6e\xfd\x6e\x96\x99\xdf\x14\x38\xa9\xcf\xc5\xd5\x68\xdc\x0e\xae\x74\x61\x35\x6d\x29\xf0\xbc\xdb\xa1\x71\x07\xac\xee\xbc\x78\xa5\xdd\xa8\x57\x25\xa0\xc5\xfe\x4f\xe4\xf6\xa0\x00\xcc\x6d\x26\xe6\x65\x9e\x1e\x04\x37\xbe\xbb\x3e\x3f\x76\xda\x06\x68\xce\x9d\x79\x8f\x8a\x5a\x6a\x68\x9b\x93\x58\xc4\x04\x07\x8d\x54\xca\x16\xe5\x72\x09\xf9\x4b\x08\x18\x6a\x85\x11\x64\xa6\x2f\x75\x61\xcf\x13\x64\x9a\xe1\xba\x98\x49\x25\x05\x9b\x7d\x71\x3c\xfb\xc2\x1c\x65\x39\x8f\x0a\x91\x23\xc9\x40\xca\x75\xc1\xb4\x58\x81\xaa\x45\x95\xde\x5d\x9f\x43\x54\x62\xb1\xc6\xe2\xdc\x95\x15\xe3\x3d\x91\xf3\x19\x72\xfd\x00\x41\xb5\x0c\x32\x6e\x41\xdb\x5f\x73\xcd\x12\x39\x93\x1f\x4d\x11\xc7\x2b\xa5\x56\xa9\x18\xdb\x09\x19\xbf\x23\xd3\xe3\xc7\x37\xd8\x02\xf8\x3c\x84\xf5\x9b\x03\x91\x4b\x25\x93\x88\xa7\x10\x90\x33\x93\xa0\x35\x8f\x4c\x67\xc0\x34\x3a\xfb\x62\x3c\xfb\x82\x81\xfb\xb4\x60\x3c\x8a\x44\x56\x88\x18\x53\x9b\x4e\x25\xcb\x00\xbf\x18\x89\x11\x2b\x04\xdf\x68\x4b\xe9\xcc\x32\x73\xc7\x84\xab\x21\x4b\x24\x21\x9d\x16\x89\xe4\xf9\x16\xc1\x4c\x98\xac\x9c\x82\x3f\xb6\x33\x29\x3e\x01\xfd\x67\x02\x0c\xa0\xa5\x76\xb4\x34\x94\x98\xc0\x74\x79\x22\xb7\x63\xf6\x03\x32\x34\x20\x05\xea\xdd\xf5\xb9\xa5\x37\xa2\x18\xd0\x99\xd4\xd1\x5a\x6c\x04\xfb\xb8\x2e\x8a\xec\xe3\x08\xff\xab\x3f\x82\xc7\x51\x2a\x86\x4f\x47\xcc\x4c\x91\x51\x54\x2d\x5e\x3e\xdd\x42\x0e\xd9\x32\xa3\x84\xf3\x33\x09\x5c\xec\x79\x88\xee\x35\xa3\x0d\x35\x06\x57\xf0\x0a\x2e\xdc\x48\x71\x48\xee\x78\x62\x06\xe7\x7f\xb1\xe9\xd2\x57\x69\x06\xd0\xe6\x16\x73\xad\x02\x85\x44\x43\xc8\xd6\xd8\x7c\x30\x91\xec\x87\xdb\xdb\x2b\xf6\xfd\xd9\xad\x55\x76\xee\xae\xcf\x71\x5d\x00\x9d\x0a\xe3\xec\xcf\xf5\x29\xbe\xdd\x66\xe2\x2f\x7f\xfe\xcb\x4c\x32\x9b\xa3\x5c\xda\x91\xc6\x1d\x3d\x42\x4a\x58\xc0\x3b\x81\x63\x16\xa8\x9c\xa1\x3e\x4c\xb9\x43\xcd\xcf\x51\x3b\x7f\x24\x6b\x01\x9c\x51\xa9\x52\xf7\x65\xe6\xcc\xdc\xa1\x1e\x66\x2a\xbc\xbb\x3e\x87\xd2\x81\x4e\xa9\x58\x43\xfe\x34\xe1\xac\x2f\x30\xf1\xdc\x36\xc6\xfc\xfb\x41\x25\x31\xe3\x72\x6b\xbe\xc5\xa2\x61\x59\xe6\x62\xa9\x72\x31\xb2\x6f\x9a\x02\x78\x91\x2c\x92\x34\x29\xb6\x20\xa5\x6c\x5e\xfb\xcc\x72\xe4\x9b\x02\xcc\x6d\x86\x00\xde\x66\x81\x61\x1a\xdb\xd7\x77\x3a\x44\x80\xc3\xa4\xb9\xdc\x88\x78\xd1\x31\xdf\x2e\x72\xc1\xef\xcd\xea\xa6\x12\xc6\x6f\x28\x67\xac\x38\xc1\x33\x66\x59\xca\x08\x97\x86\x69\x03\xad\x7e\xba\x39\xa5\x5b\xc6\x1f\x78\x82\x39\x65\xad\xbb\x7c\xb9\x4c\xa2\x84\xa7\x24\x39\x16\xe5\x12\xd2\xc6\x70\x4d\x29\x8b\x10\x7c\x68\x0a\x81\x5b\x86\x4d\xd8\x8f\x0b\x6a\x21\x56\x09\x02\x8e\x1f\x93\x62\x8d\x71\x05\x63\x9c\x67\x9e\x25\x7a\x1c\xa9\x0d\xec\xb7\x1b\x58\x4a\x9a\x2e\xbd\x80\x03\xaf\xad\x73\xf6\xda\x42\xed\x36\x59\xb1\xa5\xb5\xf7\x86\x6d\x92\xd5\xba\x80\x44\x2e\x50\x3b\x40\x22\x92\x4d\x96\xc2\xa5\x8f\x3c\x8c\x16\xef\xab\xc5\x86\xcb\x22\x89\xba\x7c\x4a\xad\x29\xc1\xfb\x61\x3c\x17\xdb\x62\xb7\x1d\xef\x03\xf1\xec\x73\xa4\xd0\x0f\x24\x32\xab\x0b\x64\x92\x81\x90\x5e\x26\x20\xf0\xaf\xa7\x9c\xdd\x77\x85\xfa\x38\x91\xdb\x8f\x9e\x84\x94\xcb\x20\xf7\xd5\x8e\xda\xed\xfe\xe7\xa9\xa2\x59\x63\x7c\x26\x01\xd5\x69\x04\x06\x25\xa3\xdd\x79\xc6\xb8\x23\xc5\xcc\xec\x95\x5d\x34\x69\xb2\x80\xba\x49\x56\x68\xa6\xcb\x0c\xe2\x09\x0a\xc5\x32\x1e\xdd\x1f\x97\xd2\xfc\xc7\x08\x43\xdc\xee\x3a\x24\x27\x9a\x49\xb5\x64\x65\x81\x1b\xc7\x2e\x61\x30\x8a\x04\xa6\x00\x7f\x41\xdb\x88\x62\xad\x62\x17\x17\x66\xca\x84\xf1\x33\x2d\x3a\x23\x7a\xe9\xb7\x27\xec\xca\x54\x68\x16\x31\xd5\xcd\x5d\xf7\x13\xc9\x4e\xff\xf5\x5f\xe1\x7d\x33\xb8\xef\x95\x62\x4b\xa5\xd8\xb7\x6c\x3c\x1e\xff\x3b\xfe\x66\x0a\xe5\x72\x4b\x7f\x71\xb9\x1d\x9b\xe2\xde\xe7\x6a\xf3\x7a\xa9\xd4\x1b\xfa\x1d\x92\x36\x9b\x7f\x24\x4b\xf6\xda\xbc\x74\x07\x55\xdd\xaa\xd7\xb3\xf2\xcb\x2f\xbf\xfa\xbd\x79\xf5\x0d\xfb\x4f\x7c\x27\x78\xfd\xef\x61\x53\xbf\xda\xd3\xd4\x3f\xf0\x07\xde\xa7\xad\xec\x5b\x38\x6b\x4c\x01\x3b\xdb\x98\xe8\xd7\xef\x95\x1a\xc3\xed\x3f\x6c\x1d\x16\x6b\xde\xc0\x56\x04\x6f\xfd\x7b\xd0\x6c\x66\xdb\xfd\xf5\x9e\x76\x23\xaa\xde\xb5\x1c\x8b\x7f\xaf\xd4\xeb\xf1\xd8\xc8\x2d\x1a\x57\x6c\xf5\xeb\x37\xd5\x81\x86\x0e\x34\xdb\x6f\x1e\x4f\xb1\xf9\xef\xce\x6e\x4e\xaf\xa7\x57\xb7\x97\xd7\x6f\x4e\x6c\x0f\xfc\x0c\x04\xdf\x33\x9b\x5a\xdb\x35\xfc\x7f\xef\x69\xf8\xf7\xca\xb6\x19\x1a\x7d\xf2\x2d\xc3\xd9\xcc\x16\xe3\xf7\x4a\xfd\xe7\x78\x3c\xfe\x3b\x3d\xe6\x72\x3b\x32\x07\x93\x79\x27\x43\x51\xfe\x81\xe7\x7a\xcd\x53\xd3\xa7\xa0\x0d\xae\x13\xad\x25\xda\xe2\x92\x65\xad\xb0\x3b\xb9\xf1\xc5\x41\x65\x30\xb1\xf0\xd6\x3f\x7f\xcb\x64\x92\xfa\xe9\x0b\xea\x80\x79\xba\x05\x6a\x89\xe8\xde\x6d\x17\x97\x23\x74\xb1\x65\x59\x7d\xe3\x62\xdc\xd9\xd6\x66\x28\x30\xe2\x7e\x26\x5f\xb5\x48\xf4\x63\xa3\xda\x8d\xe1\x81\x39\xa0\x5e\xd9\xec\xf1\xf6\x58\x70\x99\xb5\x70\x64\xc1\x11\x8d\xbb\x55\x52\x8c\x5a\x9b\x7e\xe8\x0e\xbc\x80\xac\x0a\xd4\xce\x57\xc7\xaf\x28\x50\xc8\x57\x51\x25\x92\x9f\x7d\xb1\x54\x6a\xbc\xe0\x39\xb4\xee\xd3\xf1\x76\xfc\xb7\xd9\x17\xd8\x1f\x54\x3e\x50\x31\x82\xc2\x67\x5f\xc0\x53\x58\x0e\x33\xf9\x87\x9b\xcb\x8b\x99\xfc\xf6\xdb\x6f\xbf\xc5\xd1\x32\x7f\xb7\xf8\x5e\xcc\x71\x05\xe2\x16\xf5\x94\x52\xdb\x94\x92\x62\x55\xa6\x3c\x9f\xc9\x76\x77\x4d\x2c\xbc\xd0\x1c\x79\xe7\x0d\xad\xb3\x91\xcd\x6e\x01\x49\xca\xac\x8c\x43\xdb\xe4\xc7\xff\xdf\x34\xf9\x23\xa9\x88\x4e\xc8\x87\x43\x30\xb6\x8b\xf9\xc4\x2e\x55\x33\xd8\x66\xfd\x7a\x3d\x6b\x99\xa4\x82\x36\xae\x5d\xdc\x57\x22\xd7\x4a\xfa\x35\x43\x17\x02\xe0\x36\x03\x07\x00\xfb\x96\xbd\xfd\xf7\xda\x53\x33\x0f\xf6\xe1\x57\x15\x49\xc0\x98\x2f\x6a\xf6\x05\xb4\x7a\xf6\xc5\x09\x9b\x7d\xd1\xb6\x6e\xaa\x0d\x1b\x63\x53\x66\x5f\x8c\x7c\x01\xd0\x8c\x0b\xbe\xc1\x42\xca\x2f\xbf\xfc\x3a\xc2\x26\x60\xe8\x5a\xf0\xa6\x69\x52\xf7\x8b\x41\x13\xa7\x35\xd7\x99\x1d\x08\x1b\x02\xf9\x28\xd2\xf4\xe8\x5e\xaa\x47\xcc\x33\x0e\x7e\x22\x8a\x52\x66\xb8\x3c\xaa\x93\x4b\xb9\xc9\x6a\x33\x6e\x83\x36\x5d\x35\x2e\xbd\x1d\x4c\xe8\x4c\x7e\x84\xa5\x63\x67\x94\xe8\x88\x80\x0e\xd4\xd5\x04\x97\x1a\x5a\x09\x36\xc6\x82\x16\xc2\x4c\x42\x31\x6e\xce\xd9\x6b\x00\x7e\x51\x57\x1a\x9a\xb5\xbd\x3c\xfd\xe5\xcf\x7f\x79\x73\x72\xc8\x3c\x55\x8b\xab\x4c\x15\xf4\x07\xcb\x78\x3b\xfe\xea\xed\x57\x7a\xf6\x05\x8d\x7a\xfb\x15\xfb\x3c\xd1\xc5\x4f\x35\x0d\xec\x09\xc9\xce\x8d\xe2\xf0\x52\xce\x0b\xdb\x54\x6c\x66\x5f\xa7\xc5\x75\xd5\xad\xa0\x96\xd6\xac\x03\x97\x33\x9b\x06\xde\xb4\x7b\x90\x7a\xe7\xc6\x0b\x2f\x5b\xec\x31\xe7\x59\x26\x72\x6b\x2b\x6f\xb8\x33\x20\xa7\x3a\xd4\x62\x45\x7f\x9b\x30\x33\xcb\xa6\x56\x34\xbc\x06\x43\x37\x6e\x9f\xb9\x8b\x32\x4d\x3b\x67\x6e\x7f\xb2\xe4\x8b\xbb\xf3\xf3\xf9\x4f\x93\xf3\xbb\x33\xdb\xfd\xd6\xe4\xc3\xc1\x6b\x9d\x63\xe2\x5a\x42\x63\x82\xb8\xaa\x02\xb0\x54\xe5\x46\xe4\x96\x29\xcc\xf7\x1a\x71\x24\x65\x9a\x56\xd3\x62\xcf\xe4\x47\x2a\x07\xc4\x40\x29\x13\xab\xa6\xec\x1c\xb8\x6a\xfd\xf0\xda\x47\x53\xf8\x47\xfc\xf6\x88\xf9\x4e\x9c\xb0\x0b\x57\x6b\xc7\xb8\x12\xe1\xc4\x01\xdb\x01\xe3\x6d\xbb\xb6\xc3\x73\x27\xfe\x7f\xda\xf6\xb8\x93\x90\xf4\xcb\x48\x5e\xcc\xd7\xff\x2c\xbb\x03\xc7\xee\x63\x15\x0a\xee\xcc\xa5\x31\x7a\x0d\xa1\xdc\x11\xa6\x6b\xd7\x05\x71\x16\xe3\x98\xcd\x24\x0a\x62\xd3\xa6\x42\x75\xb7\x89\x4d\xc9\x83\x94\x72\xb9\x2a\xf9\x4a\xe8\x11\xb3\x95\xcf\xa4\xbd\x9d\xda\xbb\x8e\x03\xe6\x00\x23\x6b\x6d\x09\xd5\x42\x80\x13\x39\x93\xd4\x27\x38\x61\xa9\x78\x0c\x47\xfd\xc3\x8d\xeb\x0e\xc5\x7d\x63\x41\x94\x71\x5e\xce\x24\x4e\x2e\xda\xc6\x2c\xd8\x10\xd4\x8e\xe6\xd9\xc4\x01\x1e\x8c\xf7\xba\x98\x15\x6a\x05\xb0\xc7\x99\x74\x2c\x58\x08\xce\xb0\xf7\x35\x9f\x1b\x14\x9b\xb4\x5f\x9e\xd8\xc9\xb0\x7b\x82\xda\xd6\xbe\xea\x0f\x3e\x03\xcc\x86\x9b\xb7\xde\xe5\x77\x2f\x5b\x2f\xc6\x7a\x02\x72\x78\x20\x38\xba\xa8\x11\x81\xfa\xac\xbd\x35\xb6\x5f\xf8\x4e\x67\xf4\xa8\x2a\x17\xe9\x80\x26\xe1\xfb\x3b\x1b\x85\x22\x79\x77\xa3\x7a\x58\xa4\xaf\x6b\x5b\xcb\x2c\xd3\x5d\xd5\x2e\x94\xea\x98\x97\x67\xc4\xec\x56\x1a\x45\x1f\xec\x1b\x8c\x32\x2a\x9e\xb2\x5e\x7a\xf0\x01\xd5\x87\xc8\x4a\x9f\x5d\x0d\x4a\x13\xfd\xa4\xe6\x78\xfd\xa9\x77\x8b\x9c\x86\x40\x87\xdd\x20\x09\x4b\xe7\x5c\x45\xc0\x76\x88\x49\x7b\x4d\xc1\xf0\x16\x91\xa0\x78\x31\x9b\x67\x04\x9b\xc8\xac\xff\x91\x5b\x44\x23\x3f\x73\x23\x68\x64\x54\xe6\xda\x88\x4b\x92\x77\x24\xb5\x55\xce\xf8\x4c\x5a\x36\x18\x2b\x8e\x27\xd6\x1e\x9c\xbb\x5f\x91\x63\x29\xc3\x94\x75\xe0\x14\x2a\xc0\x4a\x4e\xd2\x70\x26\x1f\x78\x9e\x70\x09\x98\xe6\x85\x86\x7c\xc3\x70\xa5\xdb\x32\x7a\xe0\x08\x38\x74\x68\x64\xde\x23\xf3\x6a\x6a\x40\xe5\x9c\xff\x27\xf3\xbf\xbf\xff\xd3\xff\x0d\x00\x00\xff\xff\x48\x2f\x2f\x28\xd9\xeb\x06\x00") +var _adminSwaggerJson = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xec\xfd\xfd\x73\xe3\xb8\x95\x2f\x8c\xff\xbe\x7f\x05\xaa\xf7\x5b\xd5\x99\xc4\x76\x4f\x32\xb9\xfb\x4d\x79\xeb\xd6\xf3\x68\x6c\x75\x8f\xee\xb8\x6d\x8f\x2d\xcf\x64\xea\x7a\x4b\x03\x91\x90\x84\x98\x02\x34\x00\x68\xb7\x92\xca\xff\xfe\x14\x0e\x00\x12\xa4\x48\x89\x7a\x35\xe5\xc6\x6c\xd5\xc6\x2d\x92\x78\x39\x00\x0e\xce\xeb\xe7\xfc\xeb\x3f\x10\x7a\x27\x5f\xf0\x78\x4c\xc4\xbb\x73\xf4\xee\x2f\x67\xdf\xbe\x3b\xd1\xbf\x51\x36\xe2\xef\xce\x91\x7e\x8e\xd0\x3b\x45\x55\x42\xf4\xf3\x51\x32\x57\x84\xc6\xc9\x07\x49\xc4\x33\x8d\xc8\x07\x1c\x4f\x29\x3b\x9b\x09\xae\x38\x7c\x88\xd0\xbb\x67\x22\x24\xe5\x4c\xbf\x6e\xff\x44\x8c\x2b\x24\x89\x7a\xf7\x1f\x08\xfd\x1b\x9a\x97\xd1\x84\x4c\x89\x7c\x77\x8e\xfe\xaf\xf9\x68\xa2\xd4\xcc\x35\xa0\xff\x96\xfa\xdd\xff\x81\x77\x23\xce\x64\x5a\x78\x19\xcf\x66\x09\x8d\xb0\xa2\x9c\x7d\xf8\x87\xe4\x2c\x7f\x77\x26\x78\x9c\x46\x0d\xdf\xc5\x6a\x22\xf3\x39\x7e\xc0\x33\xfa\xe1\xf9\xcf\x1f\x70\xa4\xe8\x33\x19\x24\x38\x65\xd1\x64\x30\x4b\x30\x93\x1f\xb8\x18\x7f\xf8\x17\x8d\xcf\xb8\x18\xff\x1b\xfe\x98\x09\xfe\x0f\x12\x29\xf3\x8f\x98\x4f\x31\x65\xe6\x6f\x86\xa7\xe4\xdf\x59\xa3\x08\xbd\x1b\x13\xe5\xfd\x53\x4f\x3d\x9d\x4e\xb1\x98\x6b\xf2\x7c\x24\x2a\x9a\x20\x35\x21\xc8\x74\x8a\x1c\xbd\xf8\x08\x61\x74\x2e\xc8\xe8\xfc\x37\x41\x46\x03\x47\xf5\x33\x43\xed\x2b\x18\xda\x6d\x82\xd9\x6f\x67\x96\x66\xd0\x32\x9f\x11\x01\x13\xed\xc5\xba\xf5\x4f\x44\x75\xa0\xd9\xfc\xfd\xbf\xf8\xaf\x0b\x22\x67\x9c\x49\x22\x0b\xe3\x43\xe8\xdd\x5f\xbe\xfd\xb6\xf4\x13\x42\xef\x62\x22\x23\x41\x67\xca\xae\x6c\x07\xc9\x34\x8a\x88\x94\xa3\x34\x41\xae\x25\x7f\x34\x66\xae\x7a\x99\xf1\x42\x63\x08\xbd\xfb\xff\x09\x32\xd2\xed\xfc\xe7\x87\x98\x8c\x28\xa3\xba\x5d\x69\x76\x53\x3e\xdc\x77\x85\xaf\xfe\xfd\x1f\x55\x7f\xff\xdb\x9b\xd1\x0c\x0b\x3c\x25\x8a\x88\x7c\xfd\xcd\x7f\xa5\xb9\xe8\x45\xd2\x9d\x9b\x15\x2d\x0f\xba\x34\xd3\x1b\xf8\x0b\x27\x27\x88\x8b\x31\x7a\x22\x73\x04\x5b\x8a\xc4\x48\x71\x58\x3b\x41\x24\x4f\x45\xb4\x38\x7b\x0a\xdf\xeb\x6d\x56\x7e\x22\xc8\xef\x29\x15\x44\x2f\x93\x12\x29\x29\x3d\x55\xf3\x19\x0c\x4f\x2a\x41\xd9\xd8\x27\xc2\xbf\x4f\x1a\x4d\xca\xee\xce\x15\x13\xbb\xc6\x53\xa2\x77\x9a\x9e\x83\xfd\xa2\x30\x1f\x34\x24\x09\x67\x63\x89\x14\x6f\xcf\xd4\xcc\x59\x5b\x63\x66\xe6\x83\xda\x89\x3d\xb2\x8e\x7b\x25\xc2\x0c\x0d\x09\xd2\xec\x86\xc6\x44\x90\x18\x61\x89\x30\x92\xe9\x50\x12\x85\x5e\xa8\x9a\x50\xa6\xff\x3d\x23\x11\x1d\xd1\xc8\xd1\xac\x3d\xb4\x81\x3f\x97\x53\xe6\x41\x12\xa1\x07\xfe\x4c\x63\x12\xa3\x67\x9c\xa4\x04\x8d\xb8\x28\xee\xe3\x47\xd6\x9f\x68\x3a\x4c\x87\x94\x01\x3f\xd1\xb4\x74\x3b\xe4\x4f\x8e\x5c\x7f\x42\xba\x3f\x94\x32\xfa\x7b\x4a\x92\x39\xa2\x31\x61\x8a\x8e\x28\x91\xe5\xd6\xfe\xc4\xed\x11\x42\xa7\x48\xd3\x99\x08\x05\xf4\xe6\x4c\x91\x2f\x4a\xa2\x53\x94\xd0\x27\x82\xde\x5f\x51\xa9\x50\xe7\xb6\xf7\xfe\x04\xbd\x37\x4c\x00\x01\xfb\x7d\x7f\x00\x0a\x67\x7f\xff\x8f\xc7\x4f\x14\x1e\x97\x39\xc9\xbb\x8e\x66\x51\xf7\xe6\xf6\xcb\x5b\xf8\x9f\xff\xf0\xdb\xb1\xeb\xb5\xfa\x4a\x31\xf7\x49\x7e\x99\xd8\x9b\xa4\xe9\xfd\x01\x04\x2b\x5e\x1d\x52\xaf\xd5\xb6\x37\x87\x6e\xb7\x7c\x75\xc8\x23\xbb\x3b\xf4\x1c\xf6\x7d\x7f\xbc\xbd\xcb\x63\x9b\x9b\x03\x2b\x38\xd2\x98\x32\xc3\x01\x32\x86\x20\xa4\x66\x02\x6e\xd8\x2d\x99\xe9\x36\x17\x89\x37\x33\xef\x2e\x71\x57\x84\x47\x95\x16\xce\x3b\xa1\x53\xba\x6a\x7d\x7b\x2c\xd6\x22\xb3\xe5\xe4\x2c\x9d\x0e\x89\xd0\x64\x70\x9b\x15\x66\x3b\xd4\x9b\x57\xa5\x82\x91\xb8\xc1\x34\x7f\x4f\x89\x98\x2f\x99\xe7\x08\x27\xb2\x6e\xa2\x94\x29\xa2\xf5\x93\xd2\xe3\x11\x17\x53\xac\xec\x0b\xff\xf5\xd7\x75\x09\xa1\xf8\x13\x59\xb5\xfe\x3d\xb3\x9a\x11\x96\xb0\x0d\xa6\x69\xa2\xe8\x2c\x21\x68\x86\xc7\x44\x5a\x8a\xa4\x89\x92\x27\xf0\x9a\xd6\x89\x88\x38\xcd\xae\x57\xe8\xc1\x89\x15\xa9\x34\x87\x7e\x94\xc9\xfc\x8c\x7c\x51\xd0\xd2\x23\x03\xc1\x02\x48\xe4\x5f\x97\x7b\x20\xe5\x66\x7b\x46\x72\xa1\x06\xc3\xf9\xd9\x13\x59\xe8\xb7\x76\xe7\x60\x86\xb0\x52\x82\x0e\x53\x45\xf4\xbc\x75\x1b\x8e\xe3\x01\xc3\x37\xd2\x47\x13\xd6\xf0\x7a\x13\x8e\xa9\x20\x11\xcc\x6d\x9d\x03\x93\x7d\xa5\xe7\xad\xf9\xfd\xdc\xcc\x5e\xb3\x7f\x2d\x6c\x55\x50\x20\x5b\xf2\x47\xf6\xc8\xd0\x29\xba\xec\xde\x5f\x74\xaf\x2f\x7b\xd7\x9f\xce\xd1\xf7\x73\x14\x93\x11\x4e\x13\x75\x82\x46\x94\x24\xb1\x44\x58\x10\x68\x92\xc4\x5a\xa0\xd2\x83\x21\x2c\xa6\x6c\x8c\xb8\x88\x89\xd8\x1f\x19\x4b\x4f\x09\x4b\xa7\xa5\x9b\x12\x7e\xcf\x47\x5f\xfa\x42\xcb\x4f\xd9\xa3\xc2\x93\xff\x59\x20\x30\xcc\x58\xf7\xed\xb5\xf6\xaa\x12\xdb\x11\xeb\xfd\xc7\x25\xba\x1d\x40\xed\x0f\x1a\x72\xd0\x90\xab\x29\x13\x34\xe4\xad\x28\xbc\x7f\x95\x68\xc7\xd2\xc0\xe1\xaf\x91\xe3\x50\xf7\x8f\xeb\xca\x38\x84\xb6\x1f\x74\xe3\xa0\x1b\x07\xdd\x38\xe8\xc6\x45\x52\x05\xdd\x38\xe8\xc6\xad\xd3\x8d\x1b\x2c\x63\x10\xd4\x7c\x41\x2d\x9a\xd0\x24\x16\x84\x7d\x50\x58\x3e\x0d\xc8\x17\x12\xa5\x46\xce\x00\x37\x4d\xf1\xc7\x81\x56\x24\x78\x4c\x8a\xbf\x14\xfe\x61\xfc\x3a\x6b\x7f\x96\x4b\x86\x6b\x7f\x9a\xd9\x22\xd6\xfe\x12\x2c\x17\xcd\xbe\x83\x5f\x68\x5c\xf9\x36\xfc\xb2\x62\x0e\xee\x9d\x25\x83\x75\xaf\xd4\x8e\xca\xbd\x60\x05\xe0\xca\x77\x04\x51\x62\x3e\xc0\x4a\x91\xe9\x4c\xad\x69\x95\xc1\x28\xd1\x62\xf6\x32\xb1\xfa\x9a\xc7\xa4\xeb\xfa\xfb\x0d\x19\xe9\x9e\xc4\x68\x38\xb7\xc7\x62\x44\x04\x61\x11\xa9\x6f\xa1\x8f\xe5\x53\xde\xc2\x2a\xd9\xbc\xd0\x9f\xfc\xc8\x85\xfe\xfc\x28\xdc\x71\x85\x91\x1f\x42\x46\xdf\xe4\xa4\xbe\x31\x17\xde\xa6\x5c\xe7\xcd\xd9\xc2\x36\xe4\xa1\xc1\x72\xb6\x3d\x25\x9b\xda\xd9\xb8\x40\x72\x2e\x15\x99\xae\xb4\xb8\x1d\x0f\x21\xec\x25\xd9\xd6\x01\x97\xee\xe9\xaf\xe0\xd4\x17\xa5\x8e\x70\xbc\xd7\x20\xd9\xae\xec\xe5\x6d\x9f\xa7\x0b\x59\x5e\x3e\xd5\x7b\xb7\x7c\x9e\xbb\xee\x28\xa6\x59\x90\x87\x77\x3d\xc8\x3d\x59\xa0\x6a\xd7\xca\x51\x7b\x00\x03\x58\x61\x7b\x28\x7a\x5c\xb2\xf3\xa7\x3f\xf5\x8d\x76\xc6\x42\xab\x26\x54\x7a\xf6\x4b\x14\x71\x61\xc4\xe1\xd8\x9e\x77\x63\x7e\xe8\xf4\x3b\xf7\xdd\xfe\x39\xea\xa0\x18\x2b\xac\x0f\xb8\x20\x33\x41\x24\x61\x0a\x4c\x3b\xfa\x7b\x35\x47\x53\x1e\x93\xc4\x18\x21\x3e\x6a\xe9\x1f\x5d\x62\x85\x2f\xb0\xc2\x09\x1f\x9f\xa1\x0e\xfc\x53\x7f\x4c\x25\xc2\x89\xe4\x08\xbb\x6d\x45\x62\xd7\x04\x66\xb1\x63\x2d\x18\x45\x7c\x3a\xa3\x49\xe6\x6d\xca\xec\x6d\x94\xc5\xf4\x99\xc6\x29\x4e\x10\x1f\x6a\xae\x22\xcf\x1e\x59\xf7\x99\x30\x95\xe2\x24\x99\x23\x9c\x24\xc8\x76\xeb\x5e\x40\x72\xc2\xd3\x24\xd6\xed\xba\x51\x4a\x3a\xa5\x09\x16\x5a\xa6\x35\xa3\xbd\xb1\x6d\xa1\xfe\x84\x64\x63\x85\x71\x69\x6a\x4e\xf1\x13\x91\x88\x2a\x34\xe3\x52\xd2\x61\x92\x9f\xf9\x87\x1e\x82\x71\x5f\x5c\xf5\xc0\xc4\x13\x29\xc4\x0d\x0f\x75\x9d\x5b\x93\x9e\xeb\x71\x8a\x19\x23\xd0\x31\x57\x13\x22\x6c\xf7\xf6\xe5\xd7\xb6\xd6\x3c\x5c\xdf\xdf\x76\x2f\x7a\x1f\x7b\xdd\xcb\x45\x73\x4d\xbf\x73\xff\xe3\xe2\xaf\xbf\xdc\xdc\xfd\xf8\xf1\xea\xe6\x97\xc5\x27\x57\x9d\x87\xeb\x8b\x1f\x06\xb7\x57\x9d\xeb\xc5\x87\x76\x5b\x35\xb6\xfc\xf8\x23\xdb\xd9\xd9\x3a\x3a\xa3\x50\x30\xea\xaf\xb1\xec\xbb\x36\xea\x9f\xbc\x5d\xab\xfe\x88\x26\x60\x74\x68\x6c\xd1\xcf\xac\x46\xf6\x4b\x34\xc3\x52\x1a\x39\xd0\x8c\xe0\xec\x91\x7d\xe6\x42\xb3\xeb\x11\xd7\x1c\x51\xcb\x8a\x4a\xa4\x91\xa2\x6c\x9c\x7d\x74\x8e\x1e\xd3\x6f\xbf\xfd\x2e\xba\xa2\xec\x09\xfe\x22\x6d\x24\x4e\x70\x79\x04\x97\x47\xeb\x5c\x1e\xff\x51\xf1\xe9\xfe\xdd\x03\xc1\xc6\x1f\x6c\xfc\xfb\xb3\xf1\x07\x13\xbf\x37\x86\x60\xdf\xde\x96\x10\xc1\x00\x16\xec\xdb\xdb\x13\x22\xd8\xb7\x5b\x3a\xe3\x70\xbc\x83\x7d\x3b\xd8\xb7\x83\x7d\x3b\xd8\xb7\x83\x7d\x3b\xd8\xb7\xbf\x1a\xfb\x76\x0b\x43\x9e\x82\x91\x3f\x18\xf9\x83\x91\x3f\x18\xf9\x83\x91\x3f\x18\xf9\x8f\xc7\xc8\xaf\xa5\xdd\x0f\xe5\xd0\xff\x3d\x81\xfe\x69\xe1\x92\xcd\x52\x05\xa2\x24\x4f\x95\xfe\x53\xf7\x0f\x7b\x65\x09\x04\x40\x33\x83\xf2\x27\xa2\xb2\x17\xb5\x68\x7b\x14\xb1\xe2\xbf\x70\xf1\x34\x4a\xf8\x4b\x36\xf2\x4f\x44\xe9\xc1\xdf\xd9\x5e\x02\x18\x60\x00\x03\x44\x01\xea\x60\xd7\x50\x07\xad\x32\x51\x1f\x94\xbf\x1f\x35\x4b\x0f\x1c\x3d\x30\xbf\xc0\xfc\xea\x68\x73\x94\xcc\xaf\xd9\xd4\x8e\xce\x7a\xb3\x7f\x9e\x5e\xb4\x77\xe5\x82\x7b\x45\x4a\x6e\x7d\x34\x4e\x6d\xb0\x4d\x4d\x2c\x8d\x17\x2a\x73\x90\x6b\xa2\x18\x90\xb2\xe2\xaa\x28\xbc\x7c\x34\x1a\x40\x61\xd4\x87\xbf\x2b\xde\x78\x5e\xe8\xd7\x12\x16\x13\xa2\x5e\x36\x24\xd4\x1b\xbe\x34\x0f\x17\xb3\x72\xf8\xdb\xee\xab\xbd\xd3\xc2\x95\x66\xff\x0b\x0c\x3f\x30\xfc\xc0\xf0\x5f\x89\xe1\x6f\x40\xf7\xa0\xc2\x2d\x5c\x6a\x95\xb0\x4b\x4d\x71\x96\xd6\x49\xba\x58\x23\xcb\xa2\x71\x5a\xc5\x8a\x3c\x8a\xca\xc4\x89\xaa\x4c\x89\xc5\xd4\x88\xca\x5c\x88\xed\x92\x1f\x36\xbd\xab\x9b\xa7\x33\x7c\x22\xaa\xf0\xf2\xd1\xe8\x9f\x85\x51\x1f\xfe\xb2\x7e\xf5\xb0\x9d\xd7\xe2\xd3\x5f\x5f\xea\x46\xc8\xd5\xd8\x23\xe9\xde\xba\x58\xd3\xde\x6c\x8c\xaf\x20\xfd\x22\xe4\x5b\xac\x45\xa3\xb7\x95\x60\xf1\x56\x33\x2a\x8e\x33\x85\x22\xe4\x4c\x84\x9c\x89\x9d\xaa\xbc\xa5\xa7\x5f\x55\xce\xc4\x31\x27\x49\x1c\xde\x3c\x11\x4c\x0e\xed\x37\x39\x04\x8b\x83\xfd\x2f\x68\xdf\x6b\xcf\x3c\x88\xf6\x41\xfb\x6e\x32\xf3\xa0\x7d\x07\xed\xbb\x85\x47\x34\x68\xdf\x41\xfb\x0e\xda\x77\xd0\xbe\x83\xf6\x8d\x82\xf6\xed\x35\xf4\x5a\xa9\xb5\x6d\x70\x6e\x1e\x95\xcd\x21\x9f\xfa\xc0\x1d\xf0\xc5\x74\xd4\x02\xdb\x6d\x92\xa1\x0a\x7f\x39\xa5\x7e\x5d\xc4\xc4\x5a\x2d\xfd\x32\x1f\x6c\x17\x98\xe4\x6f\x96\x55\xac\x50\xd8\x17\xbe\x3b\x8a\x18\x81\x85\x51\x87\xb4\xd4\x4d\x45\x9e\x57\x12\x1a\xf6\x44\x81\x23\xb9\xbe\xd6\x5f\xa8\x37\xac\x4b\x06\x1d\x72\xfb\x14\xba\xa3\xd1\x1d\x8f\x47\x67\x3c\xbc\x6c\xf1\x16\xc5\x89\x20\x4d\x78\x63\x08\x17\x6f\xb8\x78\xc3\xc5\x1b\x2e\xde\x70\xf1\x86\x0c\x7c\xfb\xfe\x5e\xe5\x89\x92\x30\xd1\x08\x4a\x6b\xe7\x35\x1c\x2a\x44\x09\xef\x06\x5e\x55\x91\xa1\xfc\x35\x25\xf2\xaf\x47\x29\x53\x1c\xa2\x28\x43\x10\x2a\x8e\x44\xa8\x78\x93\xb6\xa4\x20\x29\x05\x49\xa9\x9a\x32\x8d\x24\xa5\x47\xd6\x9f\x68\x3a\x4c\x87\x94\x65\xde\x3c\xb7\x43\xfe\xe4\xc8\xf5\x27\xa4\xfb\x43\x29\xa3\xbf\xa7\x24\x99\xe7\x3c\x49\x96\x5b\xcb\x90\x3d\xd1\x29\xd2\x74\x26\x42\x01\xbd\x39\x53\xe4\x8b\x92\xe8\x14\x25\xf4\x89\xa0\xf7\x9a\x31\xa3\xce\x6d\xef\xfd\x09\x7a\x7f\x05\x05\x86\xd0\x2c\xc1\x4c\xbe\x6f\x8d\xe3\x26\xc0\x2a\xef\x0b\x56\x39\xa0\x2a\x07\x54\xe5\xa6\x04\x0a\xa8\xca\x01\x55\xf9\x78\x51\x95\x77\xa6\x1f\x6e\x88\xcb\xf9\x2a\x9a\xe2\x71\xfa\xb2\x83\xa6\x88\x82\xa6\x18\x34\xc5\xa0\x29\x06\x4d\xf1\x48\x34\xc5\x76\x50\x38\xa8\x89\x41\x4d\x0c\x6a\xe2\x0e\x89\x13\xd4\xc4\xa0\x26\x06\x35\x71\x41\x4d\x3c\x5e\xcf\xe1\x77\x41\x1f\x0c\xfa\xa0\xff\xfb\xf1\xe9\x83\x41\x75\x0a\xaa\x53\x35\x65\x8e\x53\x75\x6a\x8d\xec\x73\x8c\x21\x45\x41\x29\x6c\x4e\x88\xa0\x14\x36\x26\x55\x50\x0a\x97\x10\x27\x28\x85\x41\x29\x0c\x4a\x61\x63\xa5\xf0\x98\xdc\x85\x41\x3b\x0c\xda\xa1\xff\x7b\xd0\x0e\x83\x76\x18\xb4\xc3\xe0\x58\x0b\xaa\x61\x50\x0d\x83\x6a\x18\x54\xc3\x55\xc4\x09\xaa\x61\x50\x0d\xbf\x2e\xd5\x90\x3c\x13\xa6\x24\x14\x43\xf4\x15\xa5\x77\x33\x2e\xeb\x15\x3c\x9f\x3b\x54\x28\x77\xd0\x66\xb1\x28\x21\xa0\xb6\xfd\x86\x26\x58\x22\x1e\x45\xa9\x28\x9d\x81\xb2\x7a\x77\x21\x08\x56\x04\x5a\xd0\x1f\x1e\x83\x5a\xb7\x38\xdd\x43\x01\x10\x0f\x79\xbc\xb0\xdb\xcd\x41\xa8\x7a\xb2\x5c\xcc\xda\xd9\xd4\x7f\x4f\x49\x33\xad\x76\x8f\x9b\x1a\xa2\xa1\xcd\x66\x5c\xac\x76\xf6\x62\x8b\xea\xef\x78\xd7\x2f\xd4\xea\xdf\x68\xe7\x67\xad\xe8\x8f\x8f\x22\x06\xba\x7a\xde\x87\x3a\x02\xd5\x8b\xfc\xc6\xc2\x6e\x5f\xfd\x9c\xd7\xad\x71\xcb\xce\x7a\x65\x71\xc3\x76\x5f\x70\x47\x71\xc4\x5f\xef\x86\xab\x5d\xd7\x70\xc2\xbf\xba\x9b\x7c\x86\x05\x61\x6a\xd0\xa4\xa0\xa9\xc2\xf2\x69\xc7\x67\xbe\x50\x68\x62\xa3\x33\x0f\x2d\x1c\xcd\x99\x5f\x9c\xef\x61\xcf\x7c\xe3\xd5\x0e\x9c\x60\xb7\x9c\xa0\x6a\xe1\xdb\xc0\x09\xda\x7d\xa6\xc3\x91\x86\xff\xc2\xa6\x5e\x6f\x53\x1f\x8f\x2e\x7a\x0c\x1b\xfc\x75\x55\xd1\x57\xdf\xe4\xed\xd4\xd2\xb2\x9a\x6f\x8d\xb7\x78\x5f\xd0\xf1\x98\x08\x63\x69\x8e\xf4\x56\xb4\xee\xcc\x25\xa0\xa7\x79\x95\xb3\x95\xdb\x3a\x7b\xf5\x18\xb6\x74\x36\x58\x33\xf6\xaf\x66\x2f\x2f\xcc\xbb\x25\x9b\xb8\x08\xb4\x20\x48\xc4\x9f\x89\x68\xbc\xb3\xef\x08\x6c\x67\x60\xde\x33\x41\x9e\x29\x4f\x65\x32\x3f\x15\x29\x43\xee\x26\x40\x59\x5f\x26\xca\xe6\x85\x26\x09\xe2\x2c\x99\x23\xa9\xb0\x50\xee\x31\x1b\xa3\x91\xe0\x53\x38\x22\x09\x96\x0a\x3d\x31\xfe\xc2\xd0\x08\xd3\x24\x15\x04\xcd\x38\x65\xea\xec\x91\xf5\x18\xba\x33\x63\x84\xaa\x28\x27\x28\x95\xfa\x60\x45\x98\x31\xae\x50\x34\xc1\x6c\x4c\x10\x66\x73\x5b\x5e\x30\xdf\x26\x88\x0b\x94\xce\x62\xac\x08\x74\x51\x82\x94\xcc\xc6\x08\x61\x07\x54\x22\x2a\x11\xf9\xa2\x04\x99\x92\x64\xae\xfb\xd0\x07\x41\x71\x64\xe9\x63\x86\x6a\x8b\x95\x11\x21\xb8\x90\x50\x4f\x65\x38\xff\x27\x66\x8a\x32\x82\xc0\x13\x22\x4d\x48\xc1\x29\xba\xe2\x12\xfc\xb2\x3f\xfe\x4d\xa2\x28\x49\xa5\x22\xe2\x04\x0d\xd3\xb1\x44\x94\xa1\x59\x82\xd5\x88\x8b\xa9\x1e\x21\x65\x52\xe1\x21\x4d\xa8\x9a\x9f\xa0\x29\x8e\x26\xa6\x2d\xa0\x81\x3c\x79\x64\x31\x7f\x61\x52\x09\x82\xb3\xde\xdd\x43\xf4\x07\xff\x99\xd9\x0d\xf2\x9b\x13\x28\xaa\x46\xa7\xb3\x64\xee\x0f\x3f\x5f\x7e\xb3\x26\xba\x11\x12\xa3\x21\x89\x70\x2a\x6d\x64\x94\x12\x73\x44\xbe\x4c\x70\x2a\x61\xed\xf4\xf4\x6c\x45\x9a\x88\x4f\x67\x09\x51\x04\xd1\x11\x52\x42\x6b\x1e\x78\x8c\xa9\x26\xdd\x3d\x21\x0d\x38\x9a\x5d\x40\x7b\x04\x7e\x03\xff\xda\x94\x0b\x82\x62\xa2\x30\x4d\x96\x46\xcb\xd9\x6f\xb3\xb6\x8e\x42\xf5\x7c\x25\x9e\x17\xd4\xc9\xbd\x32\xf2\xe2\x36\x6e\x1f\x27\x4f\x20\x78\x69\x07\x42\x0a\xb3\x51\x55\x11\x4e\xb6\x94\x57\xee\xec\xa0\xc2\xf1\x0d\xc7\xb7\x3c\x92\xc3\x1f\x5f\xb3\x17\x5b\x7a\x7e\x0f\x96\xd8\xdc\xac\x9c\xf2\x15\x95\x2a\x7b\xf3\x38\xb0\xac\xb2\xe1\x1e\x22\x2a\xfd\x4d\x1e\xd6\x10\xc4\x1d\x82\xb8\x6b\x29\x73\x9c\x41\xdc\xad\x09\x57\x0c\x01\xcf\x7b\x0a\x78\xa6\x32\x44\x3c\x87\x88\xe7\xa6\x04\x0a\x11\xcf\x21\xe2\xf9\x78\x23\x9e\xd7\xd4\x1d\x36\xcc\x7f\xad\x73\xcd\xad\xa3\x3f\x7c\x22\xea\x48\x95\xfe\xa0\x39\x04\xcd\x21\x68\x0e\x3b\xd7\x1c\xb8\x70\x1e\x8c\x16\x54\x2d\xdb\x15\x97\x76\x5f\xbf\x8b\x49\x42\x14\xa9\xb7\xb5\x12\x31\xd5\x0a\x91\x91\x40\x28\xd3\xa2\xea\x58\x10\x29\xb7\x65\xb3\x59\xc3\x47\xca\x6c\xb3\xf1\x07\x23\x6b\xe0\xbe\x35\x53\x0b\xdc\xf7\x8d\x71\xdf\xa3\x72\x1b\x78\x1c\xea\x50\x7e\x83\xec\x56\x99\xa5\xf5\x92\xfa\x83\x89\x6d\xc8\x83\x2d\xcc\x0e\xd7\xea\x96\xe2\xd9\xe1\xb6\xfb\x7c\xcb\x5b\xc6\xf4\x75\xa4\x57\x8c\x19\x7c\xb8\x5f\xc2\xfd\x52\x33\xb5\x70\xbf\x84\xfb\xe5\xf5\xee\x17\xc7\x9e\x5a\xe5\x94\xe6\x62\x5c\x30\x19\x2d\xbb\x88\x0e\x15\xec\x7a\x5c\xb7\xce\x61\x43\x47\xde\xde\x95\xd3\x9e\x03\xda\xb6\xf8\xdd\x10\xb2\x1b\x42\x76\x8f\x2a\x64\x37\xf0\xed\x23\xe0\x72\xad\x0b\x6e\x3d\x8e\x78\xd6\xb0\xb7\x8f\x62\x6f\xb7\x2d\xf2\xb3\xd5\xc1\x9e\x47\xb5\xa7\x0f\x14\xeb\x19\xcc\x1f\xc1\xfc\x51\x4d\x99\x10\x16\x19\xe0\x6d\x17\xa7\x15\xa2\x3d\x43\xb4\x67\x88\xf6\xdc\x25\x71\x42\xb4\x67\x88\xf6\xfc\x6a\xa3\x3d\x5b\x1e\xe0\x79\x54\x1a\x43\xd0\x16\x82\xb6\x10\x9c\xa5\x6b\x4e\xed\xe8\x64\xf4\x5d\x71\x66\xf7\x75\x8b\x22\x3c\x8f\x8a\xdb\xbe\x46\x80\x67\x60\xbf\x81\xfd\x56\x53\xe6\x28\xd9\x6f\x7b\x0c\xe9\x21\x16\x72\x21\x16\xf2\xa8\x98\xf1\xc1\x43\x21\x03\x27\x0e\x9c\xb8\x9a\x32\x81\x13\x1f\x7f\xd4\xa0\xf1\xaa\x0e\x66\x09\x66\x03\x1a\x7b\xa1\x83\x1f\xfe\x95\x1b\x2b\xf6\xe5\xd9\xd4\x47\x2b\x36\x55\x48\xb3\xaa\x9f\xe2\x37\xfd\x49\x92\x3b\x3a\x10\x1f\xea\x61\xac\xac\xc7\x6a\x7c\x23\xb7\x09\x66\xbd\xf8\x38\xc0\x6e\x2a\xa7\x7f\x08\x67\xe8\xdb\x0b\x35\xdc\xe6\x92\xc2\x0a\x9c\x6e\x98\x32\x63\x76\xcd\xab\xc9\x16\x8c\xca\xed\x98\xe8\x36\x57\x96\x37\x31\xef\xd6\x72\x97\x91\x47\x94\xf6\x4d\x3b\xf8\xe2\x42\xa9\xc9\xe0\x6d\x6a\x38\xe1\xe0\x6d\x6a\xaf\xb7\xa9\xc1\x32\xee\xc5\x85\x7c\xe0\xe3\x79\x50\x99\xf5\xa8\x25\xd5\x20\xa8\xa2\x20\xd6\x05\xb1\xae\x7e\xd6\x41\xac\x0b\x62\x5d\x10\xeb\x82\x58\x17\xc4\xba\xd7\x17\xeb\x1a\x4c\xf3\xab\x8d\x32\x58\x25\xaa\x36\x2f\x3d\x64\x72\x7c\x20\x15\x30\x9d\x25\x1c\xc7\xcb\x22\xbd\x72\x61\xf2\x37\x94\x0b\x6e\x4b\x24\x50\xd3\x7a\xfe\xd9\x31\x08\xa0\xf9\x68\xbf\xb2\xfc\xa7\xc5\x89\xb7\xc5\x5b\x50\x84\xaf\x6c\xe9\xde\x3e\x0a\x37\xc0\x6b\x6d\xee\x37\x09\x73\x13\x4e\x6c\xc3\x13\x7b\xb8\xfc\xc5\xea\x53\xbc\x86\x95\x44\xfe\xf5\xb8\x8e\x71\xa8\x5d\x11\x30\xaa\x2a\xa6\x16\xa2\x4d\x42\x92\x66\xc8\x66\x7c\x73\xa6\xb6\x90\xcd\x18\xb2\x19\x83\x21\x72\xf9\xb4\x83\x21\xf2\x4d\x64\x33\xae\xaf\x4c\x6c\x98\xdc\x78\x18\xb5\xe2\xc8\xac\x03\x41\xad\x08\x6a\x45\xc5\xd4\x82\x5a\xf1\x15\xaa\x15\xed\xa0\x70\xd0\x29\x82\x4e\x11\x74\x8a\xa0\x53\x04\x9d\x62\xe7\x64\x0c\x3a\x45\x03\x9d\x02\xfe\xb2\x10\xc3\x6b\x2b\x18\x6b\x2a\x16\x2b\x70\x54\x8e\xd6\xe7\x18\x34\x8a\xa0\x51\x04\x8d\xe2\xe0\x1a\x45\x6b\x26\x64\xd9\xe7\x8a\x39\xdd\xbb\x05\x29\x01\xba\xb7\x6f\x3e\x6e\x44\x03\x68\x69\x85\x30\x51\x54\xd0\xb2\x5d\xa7\x3f\xf5\x75\x14\x13\x4d\x0e\x62\x79\x1e\x6c\x8d\x22\x2e\x0c\x53\x8e\xed\x2e\x37\xf2\x44\xa7\xdf\xb9\xef\xf6\xcf\x51\x07\xc5\x58\x61\xbd\xad\x05\x99\x09\x22\x09\x53\x20\xab\x11\x88\xa3\x07\x58\xfd\xc4\x48\x15\x1f\xf5\xfd\x83\x2e\xb1\xc2\x17\x58\xe1\x84\x8f\xcf\x50\x07\xfe\xa9\x3f\xa6\x12\xe1\x44\x72\x84\x1d\xe9\x49\xec\x9a\xc0\x2c\x76\x07\x0a\x03\x5a\x3c\x4d\x32\xe5\x34\x53\x2f\x28\x8b\xe9\x33\x8d\x53\x9c\x64\xe9\x09\x8f\xac\xfb\x4c\x98\x4a\x71\x92\xcc\x11\x4e\x12\x64\xbb\x75\x2f\x38\x00\xfa\x21\xc9\x46\x29\xe9\x94\x26\x58\x68\x76\x6c\x46\x7b\x63\xdb\x42\x5a\x31\x76\x63\x85\x71\x69\x6a\x4e\xf1\x13\x91\x88\x2a\x34\xe3\x52\xd2\x61\x92\x1f\x80\x87\x1e\x82\x71\x5f\x5c\xf5\x40\x66\x8b\x14\xe2\x86\x73\xb8\xce\xad\x02\xe3\x7a\x9c\x62\xc6\x08\x74\xcc\xd5\x84\x08\xdb\xbd\x7d\xf9\xb5\xc5\xaf\x87\xeb\xfb\xdb\xee\x45\xef\x63\xaf\x7b\xb9\x28\x7f\xf5\x3b\xf7\x3f\x2e\xfe\xfa\xcb\xcd\xdd\x8f\x1f\xaf\x6e\x7e\x59\x7c\x72\xd5\x79\xb8\xbe\xf8\x61\x70\x7b\xd5\xb9\x5e\x7c\x68\xb7\x55\x63\x51\xce\x1f\xd9\x3e\x64\x39\xf7\x75\x03\x4c\x0f\x7b\xb8\x14\x56\xa9\x34\x35\x65\x04\x19\x53\xa9\x80\xfd\x37\x91\xc2\x56\x43\x79\x1c\xad\xf4\x15\x0a\x9b\x05\x59\x2c\xc8\x62\x41\x16\x3b\x36\x59\xec\x70\x26\x81\x23\x0a\x53\xfc\xee\xb8\xee\x9e\x50\x76\x21\x30\xe7\xf6\x33\xe7\xd6\xb9\xde\x5a\x63\x3a\x3f\x46\x48\xd7\xe0\x54\x6c\x4e\x88\xe0\x54\x6c\x4e\xab\xe0\x54\x5c\x42\x9c\xe0\x54\x0c\x4e\xc5\xaf\xd8\xa9\x78\x94\xb1\x89\x41\x95\x70\xef\x05\x55\x22\xa8\x12\x6f\x54\x95\x68\x0d\x85\x83\x1e\x11\xf4\x88\xa0\x47\x04\x3d\x62\x39\x71\x82\x1e\x11\xf4\x88\xa0\x47\x1c\x5b\x3c\xe2\x71\x69\x12\x41\x8b\x08\x5a\x44\xbb\xb5\x88\xd6\x4c\xe8\x78\xbc\xc5\xcd\xe6\x13\x22\xf7\x42\xe4\x5e\x88\xdc\xab\x8d\xdc\x7b\xa3\x9a\xfc\xae\xe4\x37\xf7\x75\xdb\x02\x12\x8f\x4b\xfc\x0a\xd5\xc5\xb2\xa7\x41\x18\x0b\xc2\xd8\x57\x2a\x8c\xb5\x08\x44\xb1\x15\x45\xd2\xa6\x58\x45\x13\x3c\x4c\xc8\x20\xb3\x65\xc9\xa6\xea\xfd\x15\x95\x4a\xa2\x28\x95\x8a\x4f\xeb\x2f\x97\xcf\xae\x87\x4e\xd6\xc1\x05\x67\x23\x3a\x4e\xcd\xdd\xf2\x1b\x6c\x7d\xef\x44\xe7\x02\xee\x7c\x46\x56\xf9\x15\x2b\x5a\x3f\x8a\x6b\xa9\x7a\xe8\x87\xba\x9d\xd6\xd1\x47\x72\xdb\xa5\x55\x26\xb4\x08\x39\xb8\xeb\xde\xdf\x3c\xdc\x5d\x74\xcf\x51\x07\x44\x2c\x70\x27\x98\xad\x40\xff\xa9\x27\x85\x14\x96\x4f\xf9\x5a\x0a\xb3\xcd\x25\xc8\xd9\xe0\xbf\xd0\x22\x3f\x3a\x45\x17\x57\x0f\xf7\xfd\xee\x5d\x4d\x83\x76\xa3\x40\xa9\x54\x32\x9d\x25\x58\x91\x18\x3d\xa5\x43\x22\x18\xd1\xd2\x4e\x94\xa4\x5a\xb8\xc9\xbd\x1a\xa6\xd1\xee\xdf\xbb\x17\x0f\xfd\xde\xcd\xf5\xe0\xa7\x87\xee\x43\xf7\x1c\xb9\x1d\xa7\x9b\xd5\xe3\xd2\xa3\x88\xe7\x0c\x4f\xb5\x62\xa5\x7f\xc8\x8b\xb3\xfe\x9e\x92\x94\x20\x2c\x25\x1d\xb3\x29\x61\xaa\xdc\xa2\x1b\xf0\x55\xe7\xfb\xee\x55\xb1\xe5\x09\x41\x3f\xfe\x2d\x1f\x54\x82\x87\x24\xb1\x6e\x16\xf0\x1c\xe8\x8d\x9e\x77\x64\xfd\x2f\xa9\xa1\xea\x4f\x0f\x9d\xab\x5e\xff\xd7\xc1\xcd\xc7\xc1\x7d\xf7\xee\xe7\xde\x45\x77\x60\x85\xe5\x8b\x8e\xee\xb7\xd0\x93\x95\xa9\xd1\xef\x29\x4e\xb4\xd2\xc5\x47\xe0\xb7\xa0\x11\x41\x2f\x13\xc2\x50\xca\x60\xc7\x19\x4d\x4e\xab\x77\x59\xa7\xfa\x94\x99\x19\xdd\x5e\x3d\x7c\xea\x5d\x0f\x6e\x7e\xee\xde\xdd\xf5\x2e\xbb\xe7\xe8\x9e\x24\xa0\xeb\x38\xa2\xc3\x2a\xce\x92\x74\x4c\x19\xa2\xd3\x59\x42\x34\x35\x8c\x2e\x37\x24\x13\xfc\x4c\xb9\xb0\x47\x77\x4c\x9f\x09\x33\x74\x84\x33\x0b\xed\x3b\x9d\x62\xe0\x91\xee\xe6\xfa\x63\xef\xd3\x39\xea\xc4\x71\x36\x07\x09\x6d\x14\x76\xce\x0b\x17\x4f\xa3\x84\xbf\x9c\x16\x87\xad\x99\x03\x74\x6f\x36\x11\x7f\x26\x42\xd0\x98\x94\xf6\x51\xe7\xfe\xbe\xf7\xe9\xfa\x73\xf7\xba\x0f\x14\x53\x82\x27\x12\x4d\xf8\x0b\x58\xe8\x61\x86\x60\xb8\x7f\xc6\x34\x81\xce\xdc\x62\x71\x86\x5e\x26\x14\xbc\x3a\x54\xfa\x04\x33\x6a\xa7\x48\xd9\xab\x1b\x9d\x0b\x07\x6f\x51\x1b\x2b\x9f\xa4\xc5\x37\x4a\xc7\x62\xd9\x0b\x85\x5d\xbe\xf8\xe2\xaa\xdd\xba\xf8\x45\x69\xbb\xd5\xeb\xa0\x0b\xfb\xa5\x7e\xa6\xf9\x5a\x37\x56\x41\x8b\x34\x5c\x53\x78\x58\x57\x03\x35\x3e\x30\x5f\x09\x05\x97\x9a\x13\xf3\x1d\x4f\x3c\x2e\x6d\xb4\xb1\x18\x91\x17\x5c\x3d\x72\x81\xe2\x38\x12\xef\x5e\x57\xa2\x38\xec\xd1\x38\xb4\xd6\x10\xe4\xa5\x20\x2f\x05\x79\x29\xc8\x4b\x41\x5e\xca\xfe\xdb\xb3\x3c\x41\x94\xa0\x91\xfc\x90\xed\xab\xfd\x82\xb2\x12\xa9\x37\xac\xa2\x53\x82\x6c\xcf\xf6\xa4\xd6\x0a\x21\x59\xa9\xfb\xa5\x16\xf3\x4f\x44\x65\x2f\x7e\x36\x0d\x1f\x85\x30\xf1\x8b\xe5\x28\xd9\xe0\x3f\x11\x65\xc7\x1f\x12\xfa\x43\x42\x7f\xcd\xd4\x82\x57\x60\x7b\xaf\x00\x17\x48\xce\xa5\x22\xd3\x23\xf1\x0f\xc4\x64\xb6\xd8\x61\x69\x62\xf0\x8e\x89\xef\x5a\x08\x47\x36\x9e\x73\x1b\x3b\x90\x90\x67\x92\x80\x20\xab\x04\x7e\x26\x42\x5a\xf1\x6c\x28\x08\x7e\xd2\x32\x6d\xcc\x5f\x7c\xe1\x2c\x26\x0a\xd3\x64\x1f\xfa\x73\x93\x70\xe5\xef\xfe\xf2\xaa\xf7\xe1\xf1\x5e\x81\xe1\x06\x0c\x2e\xe4\x70\x59\x7c\x85\x97\xc5\x31\x06\xf1\x84\x3b\xb0\x2d\x77\xa0\x26\x77\x3c\x70\xe1\x7c\x1f\xfe\x55\x30\xca\xfd\x7b\x5f\xfa\xe1\x1d\xe4\x47\xc9\x65\x17\xa1\xe6\x52\x71\x17\x96\xef\x37\x1b\x04\xb8\xe2\x46\xf4\xbe\x38\x0a\x7d\xd0\x1b\x6f\x9b\x5c\xd3\x77\xbe\x61\xdf\xdd\x13\x53\xa2\x70\x8c\x15\xd6\x47\x68\x4c\xd4\x19\xba\x61\xf0\xac\x8f\xe5\xd3\x09\x72\xf7\xba\xe6\x9d\x79\x28\x86\x9f\xa2\xb3\x27\x56\xd9\xd0\x66\xf5\xda\xc1\x9b\x6d\xe4\xe8\x41\x4b\xdf\xe5\xd4\x82\xe0\x15\xd2\xb1\xdb\x8c\x9b\xe7\xbe\x6e\x14\xed\xbc\xbb\x5b\xd9\xb4\x78\xc4\x17\xf3\x61\xe3\x9b\x77\x7a\x4d\xa7\x30\xf4\x70\x07\x9b\xff\xc2\x1d\x1c\xee\xe0\x70\x07\x2f\xa3\x4c\xb8\x83\x8f\x38\x00\xbe\xe2\xca\x7a\xd5\x08\xf8\x26\x66\x05\x63\x53\xc8\x0d\x0a\x6b\x02\xf5\xe6\x36\x84\x95\xf0\x5a\x15\x52\xcb\x4a\x6c\xad\xfc\x1b\x7a\x24\xc1\x6a\xde\x2c\x0f\x81\xaf\xb5\x53\x61\xc5\x20\xbc\x04\xab\xc2\x1e\x25\x9a\xb7\x27\xce\x6c\x23\xcb\x60\x05\x97\x0c\xa6\xcc\xdc\x49\x79\xb6\xb7\x3c\xc0\x0e\x5b\x73\xa2\xdb\x48\x36\xde\xc4\x3c\xe1\xc6\xc9\x2c\x1e\x51\x5a\xb2\xa8\x7b\x81\xe1\x7a\x1d\x3b\xfe\xeb\x43\x6f\x91\xb7\x8b\xbc\x15\xc0\xa5\xd6\x3a\x24\x01\x5c\xea\x10\xe0\x52\x0d\x96\x71\x2f\x88\x71\x07\x3e\x9e\xaf\xa9\x3a\x1c\x8f\x13\xf2\xc8\x74\x86\xa3\xd2\x17\x82\x0f\x72\xff\xf6\xcf\x60\x2a\x0c\xa6\xc2\x6a\xca\x04\x53\xe1\xd7\x15\xae\xb5\xab\xfb\xde\x7d\xfd\xda\x5e\xc8\x23\xbb\x99\x83\x13\x32\x5c\xc2\xe1\x12\x0e\x97\x70\xb8\x84\x77\x45\xe1\xe0\xaf\x5b\x53\xe9\x3e\x0a\x2f\xdd\x91\x5d\xeb\xc1\x49\x17\x6e\x7c\xf7\x71\x70\x69\xad\x9a\x67\x70\x69\x05\x97\x56\x70\x69\x05\x97\x56\x70\x69\x65\xbf\x07\x97\xd6\x41\x77\xeb\x57\x6b\xb6\xab\xd6\x18\x78\x4c\x06\x15\x48\x32\xd9\x4f\x03\x3f\x6d\xb0\xf0\x6b\xc1\x7d\x57\x78\xe2\xfb\xf2\x0a\x0f\xf2\x72\x39\xd0\x2f\x8d\xd7\xce\xc3\x5f\x66\x4e\xe4\x31\x69\x9c\x77\x5f\x78\xb9\xed\x71\x81\x6e\xa2\x46\xf5\xf0\x47\x7e\x80\x3c\xfb\xf2\x4e\x78\x63\xd1\x67\x35\xbb\xfa\x2d\x5a\xea\x2a\xce\x68\x30\xdb\xad\x24\xd4\x1b\x06\x1c\xb0\x4c\xf8\x00\xe3\x79\xd3\x77\xd8\x07\x8b\xfe\x38\x70\x60\x83\x3b\xbc\xd4\x2e\x4d\xd3\x9a\xe9\x3b\x1b\x8d\xf3\x20\xad\xba\xe2\x2a\x3e\x6d\xfb\x45\x07\x73\x5e\x32\xe5\x70\xdd\x85\xeb\x2e\x5c\x77\xe1\xba\x0b\xd7\x5d\xf9\xba\x73\x57\xcf\xa0\xe2\xd2\xab\x7e\x96\x5f\x7d\xd5\xcf\xb3\x0b\xb0\xfa\xf1\xfa\x28\x6a\x8d\x9c\x47\x8d\x75\x38\x70\x1b\xf9\x6f\x1f\x49\x76\x97\x3f\xe4\x43\xb8\x8e\x6a\x37\xc6\x1b\xbb\xd8\x96\x6e\xf2\x37\x77\xbd\x2d\x3b\xb2\xe1\x92\x6b\x48\xae\xb7\x7a\xd5\xed\xc5\x51\xb5\x5f\xc7\xc3\x01\x9d\x56\xaf\x65\x77\xde\x8b\x79\xfd\x33\x17\x04\x51\x36\xe2\x88\x43\xf8\x8e\x54\x22\x8d\x14\x65\xe3\xec\xa3\x73\xf4\x98\x7e\xfb\xed\x77\xd1\x15\x65\x4f\xf0\x17\x69\xa3\x51\x3e\xf8\xcc\x82\xcf\xec\x18\x7d\x66\x26\xd6\x6e\x30\xc3\x82\x30\x55\xa1\x5b\x94\xaf\x13\x78\xdd\xaf\x49\xed\xa4\x0e\x68\x00\x69\xd1\x1e\xd9\x0b\x39\xbb\xaa\xde\x58\x5e\x58\x49\x7b\x79\xc3\x6e\xa5\x96\xeb\x23\xed\xf1\x2a\xbd\x55\x29\x3d\x18\xa1\x82\x11\xaa\x3c\xcf\xc3\x19\xa1\x36\xa0\x7b\x88\x8d\x38\xfc\x55\x75\x3c\xde\xa3\x96\xdf\x67\x6d\x73\x1e\x85\x5b\x2d\xdc\x6a\xe1\x56\x6b\x01\xdd\xc3\xad\xb6\xf4\x56\xfb\x4a\xdc\x43\xc7\x70\x7b\xb5\xc4\x3b\xf4\x56\x6f\xae\xe0\x35\xd9\x01\xb9\xde\xea\x2d\xf6\x5a\x8e\xd2\xc3\xdb\x9e\x83\x7f\x28\xf8\x87\x82\x7f\x28\xf8\x87\x82\x7f\xc8\xff\x3d\xf8\x87\x96\xd1\xfd\x60\xea\x89\x15\x81\x06\xd9\xd1\x95\x1f\xfe\x95\xff\x9d\xa9\x25\xbe\x6a\xb1\x0c\x86\xe8\x42\x10\x38\x15\x5c\x58\x10\x1b\x69\xeb\xc3\xd7\x2b\x19\x9f\xb1\x8a\x26\x78\x98\x90\x4e\xd6\xad\x2b\x97\x0f\x0a\xc6\x6f\x08\xab\x82\xd0\x0b\x65\xe9\x96\xe8\x22\x06\x1a\xe2\xd6\xbc\x9d\x37\x7a\x0c\x0a\xc9\xc2\xa0\x0f\x0b\x62\xb4\xb8\xf0\xcd\x0e\x90\x5b\x19\x1a\x03\xbb\xf3\x8a\xf1\x6b\xb9\x9b\x8f\xf2\x8b\x41\xa2\x17\x9a\x24\x5a\x92\xb1\x52\x5b\x4b\xa4\xd1\x57\x87\x36\xa9\x5d\xf9\x57\x05\x38\xa9\xe2\x0e\x55\x2c\xa1\x89\xdd\x7c\x7b\x3e\x60\x8a\x00\xbb\xcd\x86\x59\x6c\xf5\xbe\x15\xa6\xf5\xb7\xc1\x09\x3e\x11\x75\x28\x36\xb0\xe9\xd9\x5f\x7a\xee\x05\x19\x11\x41\x58\x44\x5a\x08\xab\xb1\x0e\xde\xcb\x2f\x66\x92\x16\xec\x65\xea\x36\xad\x3f\x55\xc5\xad\x9e\x56\x10\x75\xad\xa0\xd7\xef\xdc\xff\x38\xb8\xeb\xde\xdf\x3c\xdc\x5d\x74\xcf\x51\x07\xd8\x20\x7c\x63\x0e\x08\xfd\x27\x34\xa7\xb0\x7c\xca\xcd\x1d\xc2\xb0\x01\x09\x9b\x1e\xf4\x49\x4d\x45\x74\x8a\x2e\xae\x1e\xee\xfb\xdd\xbb\x9a\x06\xed\xf1\xd1\x72\xa2\x22\xd3\x59\x82\xb5\xfc\xf8\x94\x0e\x89\x60\x04\xae\xe6\x24\x95\x8a\x88\x5c\xcb\x34\x8d\x76\xff\xde\xbd\x78\xe8\xf7\x6e\xae\x07\x3f\x3d\x74\x1f\xba\xe7\xc8\x9d\x43\xdd\xac\x1e\x17\x1c\x3d\xe3\xf9\x31\x3f\xe4\x35\x60\x7f\x4f\x49\x4a\x10\x96\x92\x8e\xd9\x94\x30\x55\x6e\xd1\x0d\xf8\xaa\xf3\x7d\xf7\xaa\xd8\xf2\x84\xa0\x1f\xff\x96\x0f\x2a\xc1\x43\x92\x58\xb5\x17\xf0\x29\xf4\xf1\xcf\x3b\xb2\xfa\x70\x6a\xa8\xfa\xd3\x43\xe7\xaa\xd7\xff\x75\x70\xf3\x71\x70\xdf\xbd\xfb\xb9\x77\xd1\x1d\x58\x04\x9b\x8b\x8e\xee\xb7\xd0\x93\x15\x35\xd1\xef\x29\x4e\xa8\x9a\xeb\x75\x94\x86\x2f\x9a\xfa\xb6\x29\x33\xc5\x71\x41\xd6\xc6\x7e\x85\x5b\x39\x23\x91\x99\xd1\xed\xd5\xc3\xa7\xde\xf5\xe0\xe6\xe7\xee\xdd\x5d\xef\xb2\x7b\x8e\xee\x49\x42\x22\x25\x33\xa2\xc3\x2a\xce\x92\x74\x4c\x19\xa2\xd3\x59\x42\x34\x35\x0c\x98\xd8\x90\x4c\xf0\x33\xe5\xc2\x32\xb4\x31\x7d\x26\xcc\xd0\x51\x6f\x2b\xd3\xbe\x83\xd3\x19\x78\xa4\xbb\xb9\xfe\xd8\xfb\x74\x8e\x3a\x71\x9c\xcd\xc1\x54\x46\x2f\xec\x1c\x67\x58\x39\x2d\x0e\x9b\x8e\xb4\xc2\xa2\x19\x0c\x2c\x1f\x7f\x26\x42\xd0\x98\x94\xf6\x51\xe7\xfe\xbe\xf7\xe9\xfa\x73\xf7\xba\x0f\x14\x53\x82\x27\x12\x4d\xf8\x0b\x28\x74\x30\x43\xd0\xf3\x9e\x31\x4d\xa0\x33\xb7\x58\x9c\xf9\xa7\xdf\xeb\x79\xc2\xd3\x24\xd6\xcb\xf4\xea\x3a\x4a\xe1\xe0\x2d\xaa\x29\xe5\x93\xb4\xf8\x46\xe9\x58\x2c\x7b\xa1\xb0\xcb\x17\x5f\x5c\xb5\x5b\x17\xbf\x28\x6d\xb7\xc5\x17\x6a\xf7\x4b\xfd\x4c\xf3\xb5\x6e\xac\x99\x15\x69\xb8\x26\x97\xdd\x81\x6d\x6f\xb9\xc1\xb6\xa5\x7a\x97\xfb\xfa\x5d\x4c\x12\xa2\x48\xad\xa0\x74\x09\x8f\x5f\x4b\x50\x32\xbd\xbf\x0d\x59\xc9\xcc\x25\x88\x4b\x5f\x8d\xb2\xe4\x16\xbc\x15\xca\x92\x39\x6b\xbe\xce\x04\x29\x83\x9e\x72\x6d\xf2\x04\xdf\xa6\x99\xe5\x28\xd2\x02\xdb\x63\x67\xd9\xfb\xa5\x78\x68\xe6\x10\x6c\x48\x8b\x23\x09\x36\xa4\xed\xd8\x62\xe1\xc7\x0a\x28\xdd\x83\xb3\xca\x4d\x04\xac\x02\xbf\xbc\x84\xd7\x8f\x93\x6b\x96\xc7\x7e\xcc\xbc\xd3\x6b\xad\x1d\x4c\xe4\xeb\x65\x9f\x0b\x47\xbc\xd9\xc4\x6d\xe8\xcf\xf1\xce\xbb\x2d\xd7\x46\xdd\xb1\x6e\xf3\xe5\x51\x2c\x86\x7c\xbc\x1e\x89\x63\x62\xff\xaf\xe2\x92\x78\x73\x62\xf2\x57\x67\x34\x08\x3e\x96\xe0\x63\x09\x3e\x96\xe0\x63\x09\x3e\x16\xb4\xa9\x8f\x65\x57\x92\xd6\x51\x3b\x24\x8e\x53\x54\x3a\xac\x47\x22\x48\x4b\xc7\x2e\x2d\xb5\x45\x29\x3c\x2e\x17\x4b\x51\x1d\x5c\xbb\xea\x56\x5b\xf4\xc2\xb7\x64\x1c\x3c\x2e\x1d\xb1\x75\xe6\xc0\xaf\x8e\xf1\x6d\x66\xfa\x3b\xda\xe9\x06\xad\x38\x68\xc5\x41\x2b\x0e\x5a\x71\xd0\x8a\x51\xd0\x8a\xd7\xd6\x8a\xdf\x92\xa0\x78\x74\x1a\x72\x90\x15\x5f\x7b\xc2\x5f\x99\xac\xd8\x16\x9b\x40\xdd\xc9\x6d\xa9\x65\xe0\xeb\x0c\x28\x3a\xe2\x9b\x20\xe4\xbc\x22\x6f\xe9\x42\xc0\xcd\xd7\xc2\x47\x5b\x1e\x70\xf3\xf6\xec\xaa\x47\xcc\x23\x43\x36\x70\x10\x2b\x77\x34\xdd\x60\x82\x0c\x26\xc8\x60\x82\x0c\x26\xc8\x60\x82\x44\xed\x4e\x7e\x5e\x69\x70\x0a\xf9\xcf\xfb\x32\xac\x1e\xb1\xa4\x18\x72\xa1\x83\xb0\xb8\xbb\xe9\xb6\x55\x77\x6e\x93\x0d\x52\xae\x5f\x51\x62\x25\x12\xb7\x9d\xf6\x6f\x4b\x18\xd8\x15\x95\x4e\xd1\x3d\x26\x7e\x25\xf7\xcd\x92\xb6\x81\xdf\x75\x2b\xfa\x66\xd1\x77\x17\xe8\x00\x04\x88\xb0\x34\x0a\x5e\x9a\x28\x3a\xd3\xa2\x3c\x1e\x13\x69\xf1\x88\xb5\xe0\x7d\xe2\x78\x97\x78\x26\xe2\x34\x03\x9c\x86\x2e\x1c\x0a\x37\xa8\x2d\x8a\xa3\x11\xa0\xcd\x03\x5d\xc9\x17\x05\x4d\x3d\x32\xc0\xe1\x06\x1a\x9d\xb5\x11\xff\x36\x80\x03\x2f\x21\x4e\x00\x07\x5e\x8b\x9b\x04\x70\xe0\x96\x80\x03\xaf\xab\x82\x99\x53\xe9\x6b\x61\x70\xc8\x9d\xd4\xea\xcc\x52\x47\xaa\x8c\xcd\xb8\xac\x97\x4c\xee\xc8\x98\x4a\x60\x49\x4b\xaa\x5d\x39\x99\x04\x0a\x2b\xc0\x56\xff\xa8\x5f\x40\x31\x99\x25\x7c\x0e\xf6\xaf\x25\xe2\x8a\xeb\xe2\x76\x41\x63\x68\xbb\xc4\xe2\x46\x7e\x28\x9d\xaa\x2d\x32\x77\x3e\xef\x56\x48\xd9\x79\xc8\xff\xeb\xcb\xdb\xc7\x14\x78\xb5\x77\x81\xfb\xb0\x7c\xf6\x98\xaa\xbd\x07\x75\x22\xa8\x13\x4d\x76\x4d\x50\x27\x56\x11\x28\xa8\x13\x41\x9d\xd8\xa7\x3a\x71\x60\x09\xe6\xc3\xbf\x4a\x45\xd4\x97\x05\x20\x3e\xd8\xa8\x43\x70\xce\x52\x09\x47\x7e\xa5\x20\xf3\xc8\xaa\x1f\x38\x27\xe5\x90\x38\x1e\x33\x4c\x95\x57\x76\x4d\xea\x4b\x6b\x46\x84\x9a\x7b\x6f\x92\xe9\x4c\xcd\xff\xfb\x91\x51\x95\x85\x78\xd1\x31\xe3\xc2\xec\x18\xfd\xf1\x04\xb3\x38\xd1\x97\xba\xcc\xda\x89\x30\x63\x5c\x01\x2b\x87\x09\xc4\xe8\x99\x62\xc3\xf8\x3b\xb7\xbd\xc6\x81\x8e\xc7\x24\x6a\x1d\x36\x92\x71\xff\xb5\xd3\x0e\x5f\x59\x74\xc5\x84\x3e\x25\x7c\x88\x93\x64\x8e\xd2\xa2\x47\x49\x37\xd0\x92\x39\xb4\x45\x7b\x6b\x87\xba\xe6\x60\x05\xca\x6a\x5b\xcb\xac\x11\xc7\xc4\x64\x0e\x6d\x8e\xf0\x96\xf0\x8d\xb1\x9b\xb6\x1c\xd5\xb6\x19\x5a\x82\x70\xd2\x54\x38\x39\x22\xb6\x71\x58\xd9\x24\xdc\xe4\xc7\x7f\x93\x2b\x2c\x9f\xfc\x4a\xe6\x70\xa1\xbb\x62\xf4\x85\x2a\xbe\xe5\x92\xbe\xff\x6e\xf2\x5e\x9e\x5e\xb0\xfa\xdd\xac\x1a\xfa\xea\x57\xa1\x32\x7a\xcd\x8b\xb6\x8c\xbe\x79\x0c\xd3\x2b\x8f\xc3\xfd\xe8\x77\xe8\x7e\xcb\x5b\x76\xbf\x3c\x13\x21\x29\xb7\xaf\x09\xa2\xc4\x7c\x80\x95\xd2\x0c\x69\x03\x1b\x75\x2d\xd7\xec\x63\xf9\xd4\xac\x2a\xfb\x27\xa2\x0a\x2f\xb7\x5d\xac\x71\x13\x85\x79\x16\x46\xbe\x7f\xfe\xd4\x60\x1b\xbf\x31\x71\xa7\xf1\x91\x5c\x31\xef\xe3\x2b\x4c\xdf\x94\xc1\xac\x31\xf1\xaf\xa5\x48\x7d\x33\x86\xbb\x2a\x1e\xf2\x18\x0b\xd6\x2f\xbb\x41\x5a\x33\xc2\xd2\x25\xf6\x16\x4f\x6e\xf1\x4a\x0e\x47\x74\x19\x8d\x9a\x9e\xc5\xa3\x39\x81\x25\x49\x6b\xc5\xdc\xee\xdd\x02\xd9\xd7\xdd\x4e\x68\xdf\xbc\x0a\xc2\xe2\xae\x47\xb5\x1f\x07\xb0\xb7\x1a\xeb\xa4\x10\xf6\x5c\xb1\x75\xe3\xce\xca\xce\x90\x4b\x2a\xcc\x8e\xa6\x9a\x40\x41\x2a\x2a\xfd\xfa\xec\x11\x17\x46\xda\x8c\xed\x99\x35\x0e\xad\x4e\xbf\x73\xdf\xed\x9f\xa3\x0e\x8a\xb1\xc2\xfa\x90\x0a\x32\x13\x44\x12\xa6\x8c\x29\x82\x29\xaa\xe6\x68\xca\x63\x92\x18\x3b\x80\x31\x0e\x5e\x62\x85\x2f\xb0\xc2\x09\x1f\x9f\xa1\x0e\xfc\x53\x7f\x4c\x25\xc2\x89\xe4\x08\xbb\x8d\x43\x62\xd7\x04\x66\xb1\x63\x0f\x18\x45\x7c\x3a\xa3\x89\xc9\x6b\xf3\xfd\xdb\x94\xc5\xf4\x99\xc6\x29\x4e\x10\x1f\x82\x0d\xe5\xec\x91\x75\x9f\x09\x53\x29\xe8\xb8\x38\x49\x90\xed\xd6\xbd\xe0\x19\x30\xdc\x28\x25\x9d\xd2\x04\x0b\x2d\x3d\x9a\xd1\xde\xd8\xb6\x50\x7f\x42\xb2\xb1\xc2\xb8\x34\x35\xa7\xf8\x89\x48\x44\x15\x9a\x71\x29\xe9\x30\xc9\x8f\xf1\x43\x0f\xc1\xb8\x2f\xae\x7a\xe0\x34\x8c\x14\xe2\x86\x0f\xba\xce\xad\x07\xdd\xf5\x38\xc5\x8c\x11\xe8\x98\xab\x09\x11\xb6\x7b\xfb\xf2\x6b\xfb\xff\x1e\xae\x6d\xba\x58\xf7\x72\xd1\x01\xd8\xef\xdc\xff\xb8\xf8\xab\xcb\x0f\x5b\x7c\x72\xd5\x79\xb8\xbe\xf8\x61\x70\x7b\xd5\xa9\xc8\x3b\xb3\xdb\xaa\xb1\x2f\xd1\x1f\xd9\xe6\x87\x69\xff\x8a\x46\x4b\x43\x13\x9b\x1b\x1d\x1a\x59\x1c\x1a\x9b\x1b\x9a\xda\x1a\x9a\x19\x1a\xea\xad\x0c\x7b\x08\x53\x6b\x6e\x0a\xb8\xa2\xb2\x68\x0b\x38\x8e\x98\xb5\xc2\x90\xf5\x1c\xf6\x6d\x08\xf8\xea\xac\x00\x5f\xa9\x09\x20\xe8\xff\x7b\xa1\xdb\x5b\x55\xfe\x5b\xae\xf9\x6f\x13\x94\x9a\xe1\x5f\x84\xa8\xd4\xc5\xa8\x54\x12\x82\x52\x43\x50\x6a\x53\x02\x85\xa0\xd4\x10\x94\x7a\xb4\x41\xa9\x65\x45\x2b\x78\x6c\xdb\xe0\xb1\x6d\xb9\x8e\xd6\x66\x87\xed\x5b\xd5\x5c\x82\xf3\x32\x38\x2f\x83\xf3\xf2\x48\x4f\x6e\x70\x5e\x36\xa7\x51\x70\x5e\x06\xe7\x65\x70\x5e\x06\xe7\x65\x70\x5e\x06\xe7\xe5\x6b\x9a\x46\xda\x10\x1b\x7a\xcc\x2e\xdb\xe0\x89\x5d\xe1\x89\x6d\xb9\x92\xdf\x4a\x47\xec\x5b\xd5\x11\x82\x6a\x1f\xfc\x92\x5b\x4d\xbb\x55\x4a\xfd\x5b\xbb\x37\x83\x2b\xb6\x39\x21\x82\x2b\xb6\x31\xa9\x82\x2b\x76\x09\x71\x82\x2b\x36\xb8\x62\xbf\x42\x57\x2c\x8d\xb7\x2e\xb9\xd5\x44\x6f\xd1\xb2\x62\xdc\x05\xf3\x50\x66\xdc\x12\xbf\x81\xf4\x88\xe5\x53\x66\x01\x6a\xa0\xcf\xf4\xe2\xa3\x50\x64\x2a\x27\x7c\x08\x85\x66\x1b\x8d\x05\x2b\xcd\xc1\x15\x40\x15\xe8\x27\xb9\x51\xb1\x85\x35\x02\xb6\xd1\x51\xbc\x89\x79\x6a\x8a\xd3\x3e\x3c\xa2\xb4\x6f\xda\x41\xf0\x0b\x82\x5f\x90\x6d\x1a\x4e\x38\xc8\x36\xed\x95\x6d\x5e\x4b\x61\x69\xdf\xf1\x3c\x3a\xfb\xc4\xde\xc5\x52\xd9\x18\xb4\xcd\x94\xc9\x06\xd7\x5d\x3a\x4b\x38\x8e\x57\x05\xc8\xfd\x86\x72\x59\x6d\x89\xb8\x69\xda\xd5\x1f\xb4\x5c\xda\x5c\x88\x8d\x33\x23\xff\x1a\x50\xe3\x6b\xa7\xfe\xaa\x78\x66\xb0\x7f\x33\xd4\xa2\xb5\x10\x08\xf7\xbf\x99\xdb\x9e\x8d\xf7\xca\xbb\xf9\x4d\xa6\xde\x85\x23\xba\xfa\x88\xc2\x1f\x85\x00\xef\x7d\x59\x42\xca\xc7\xb6\x91\xd1\x43\xfe\xb5\xe5\xe7\x36\x5b\xdf\x43\x98\x38\xde\xe4\x29\x7d\xc3\xce\xe6\xe0\x50\x5e\x1e\xf5\xb3\xa3\x00\xd4\x47\xd6\x9f\x68\x3a\x4c\x87\x94\x65\xf1\x76\x6e\x87\xfc\xc9\x91\xeb\x4f\x80\x75\x69\xf1\x2f\x93\x79\x6e\x0a\x93\xe5\xd6\x32\x45\x09\x9d\x6a\x2d\x35\x22\x42\x01\xbd\x39\x53\xe4\x8b\x92\xe8\x14\x25\xf4\x89\xa0\xf7\xfa\xc8\xa3\xce\x6d\xef\xfd\x09\x7a\x7f\x85\x53\x16\x4d\xd0\x2c\xc1\x4c\xbe\x6f\x8d\x82\x15\x6c\x66\xa1\x9a\x4a\xf0\x96\xee\x92\x38\xc1\xa2\x18\x2c\x8a\xad\xb3\x28\xb6\x45\x67\x30\x49\xa5\x78\x4a\xda\xa2\x3d\xb4\x5d\xeb\x0f\xda\x43\xd0\x1e\x82\xf6\x10\xb4\x87\x82\xf6\xd0\x0e\x0a\x07\xd5\x21\xa8\x0e\x41\x75\x08\xaa\x43\x50\x1d\x76\x4e\xc6\xa0\x3a\x2c\x53\x1d\xe0\x2f\x87\x1b\xb3\xae\x1e\xd1\x58\x7f\x68\x00\x12\x73\x34\xca\x43\x50\x1c\x82\xe2\x10\x14\x87\x83\x2b\x0e\xad\x99\xd0\xdb\xc3\xbb\x08\x88\x11\x01\x31\x22\x20\x46\xd4\x20\x46\x1c\x4a\x64\x33\xf2\xda\x91\xa5\xc8\x1c\x85\xd0\xf6\x6a\x39\x32\x6f\x4f\x8c\x0b\x59\x3f\x21\xeb\x27\x98\x21\x43\xd6\x4f\x30\xb4\x05\x43\x5b\xab\x0d\x6d\xaf\x65\x3d\x3f\xf0\xf1\x3c\x80\x70\xda\xf2\x88\xe5\xef\x8e\x41\x02\x3d\x60\xcc\x41\xb0\xb2\x05\x2b\x5b\x35\x65\x8e\xd3\x3d\xdf\x9a\x5b\x3f\x00\x3c\x05\x89\x3f\x04\x1e\x84\xc0\x83\x95\xc4\x09\xfa\x50\xd0\x87\x5a\xa7\x0f\xbd\xa2\xa2\xd0\xba\x30\xe5\xa0\x31\x04\x8d\x21\x68\x0c\x6f\x56\x63\x68\x0d\x85\x83\xba\x10\xd4\x85\xa0\x2e\x04\x75\x61\x39\x71\x82\xba\x10\xd4\x85\xa0\x2e\xb4\x3a\x34\xf9\x58\x14\x86\xa0\x2c\x04\x65\xa1\xdd\xca\x42\x6b\x26\x14\x82\x78\x43\x10\x6f\x08\xe2\xfd\x6a\x82\x78\xdf\xa8\xc2\xbe\x57\x31\xcd\xb1\xc8\x65\x82\xd7\xa2\xbc\xf4\xf3\x02\x63\x6d\xad\xc8\x94\x8f\x76\x53\xdc\xc7\x5d\x91\xfa\x85\x8b\xa7\x51\xc2\x5f\x06\x99\x56\x67\x83\xc2\xf3\x7f\xdb\x7c\x3e\xef\x87\x5c\x78\xf6\x7e\xcc\x84\x68\xef\x37\xd7\x7a\x11\x20\x34\x5d\x85\x0f\x2a\x11\x17\x28\x9d\xc5\xf0\x67\x94\x4a\xc5\xa7\xf5\x52\xf5\x67\xac\xa2\x09\x1e\x26\xa4\x93\xf5\x7b\xc1\xd9\x88\x8e\x53\xb3\x3f\x7e\x03\x56\x88\x9d\x64\x73\xe2\x24\x23\xcd\x14\xdd\xf8\x96\x49\xe2\x0f\x30\x8e\x5f\xec\x9b\x79\x27\x47\x11\x80\xbe\x38\x6c\x33\x9d\x43\xc1\x8d\x16\x77\xd1\xb6\x2c\xce\x6b\xad\x1d\xe2\xcf\xe2\x99\x58\x25\xaa\x82\x15\x3a\xd3\x4c\x68\x0c\x9b\xf3\x65\x42\xc1\xb2\x06\x96\x38\xb0\x3e\xe5\x0d\xa3\x17\x9a\x24\x20\x71\x18\x5a\xb4\x6f\xe6\x8d\xb4\x17\x3b\x71\x7b\xf6\xde\xc4\xbc\x1d\xf3\x58\x31\x73\x77\x04\x8d\x1b\xe2\x48\xa7\xfd\x9a\x08\xbb\x2b\x18\xd9\xab\xe2\xec\xd6\x5e\x9f\x35\x39\x55\x1f\xfe\x55\x79\x25\x36\xa9\x9d\xfa\xda\xf7\xe0\x27\xa2\xde\xcc\x25\xf8\x89\xa8\x43\xdd\x80\x6f\xf1\xda\xdb\xf4\xae\x5b\xca\xf8\x04\x19\x11\x41\x58\x44\x8e\x35\x27\x6b\xe1\x8a\x3b\xda\xe9\x6e\x74\xb3\x1d\xed\x6c\xd7\x31\x60\xfd\x62\x26\x69\xcd\x55\x53\xc7\x72\xfd\xa9\x2a\x6e\xdd\xcb\x05\x17\x98\x35\x56\xf5\x3b\xf7\x3f\x0e\xee\xba\xf7\x37\x0f\x77\x17\xdd\x73\xd4\x81\x83\x0e\xdf\x18\xf6\x4e\xff\x09\xcd\x41\x3e\x6c\x66\x0c\x13\xe6\x8e\x93\xc0\xaa\xc1\x0d\xae\xa9\x88\x4e\xd1\xc5\xd5\xc3\x7d\xbf\x7b\x57\xd3\xa0\x65\xfe\x94\x8d\x91\x22\xd3\x59\x82\x15\x89\xd1\x53\x3a\x24\x82\x11\x50\xac\x92\x54\x2a\x22\x72\xe7\xb8\x69\xb4\xfb\xf7\xee\xc5\x43\xbf\x77\x73\x3d\xf8\xe9\xa1\xfb\xd0\x3d\x47\xee\x16\xd1\xcd\xea\x71\xe9\x51\xc4\x73\x86\xa7\x34\x32\x3f\x64\xa5\x68\xd1\xef\x29\x49\x09\xc2\x52\xd2\x31\x9b\x12\xa6\xca\x2d\xba\x01\x5f\x75\xbe\xef\x5e\x15\x5b\x9e\x10\xf4\xe3\xdf\xf2\x41\x25\x78\x48\x12\xeb\xad\x07\x07\xb4\xbe\xbc\xf2\x8e\xac\x1b\x3f\x35\x54\xfd\xe9\xa1\x73\xd5\xeb\xff\x3a\xb8\xf9\x38\xb8\xef\xde\xfd\xdc\xbb\xe8\x0e\xac\x31\xe6\xa2\xa3\xfb\x2d\xf4\x64\x6d\x36\xe8\xf7\x14\x27\x54\xcd\xf5\x3a\x4a\x73\xe9\xa3\x97\x09\x61\x28\x65\x70\x81\x18\x4b\x21\x66\x5e\xa7\x72\x46\x22\x33\xa3\xdb\xab\x87\x4f\xbd\xeb\xc1\xcd\xcf\xdd\xbb\xbb\xde\x65\xf7\x1c\xdd\x93\x04\x6c\x69\x8e\xe8\xb0\x8a\xb3\x24\x1d\x6b\x4e\x30\x9d\x25\x44\x53\xc3\xd8\x0a\x87\x64\x82\x9f\x29\x17\xf6\x3a\x1e\xd3\x67\xc2\x0c\x1d\xf5\xb6\x32\xed\x3b\x9b\xd5\xc0\x23\xdd\xcd\xf5\xc7\xde\xa7\x73\xd4\x89\xe3\x6c\x0e\x12\xda\x28\xec\x1c\x77\x74\x4f\x8b\xc3\xa6\x23\x1a\x41\xf7\x66\x13\xf1\x67\x22\x04\x8d\x49\x69\x1f\x75\xee\xef\x7b\x9f\xae\x3f\x77\xaf\xfb\x40\x31\x25\x78\x22\xd1\x84\xbf\x80\xa3\x17\x66\x08\xfe\xdf\x67\x4c\x13\xe8\xcc\x2d\x16\x67\xfe\xe9\xf7\x7a\x36\x66\x4d\x91\xb2\x57\xf7\x5d\x16\x0e\xde\xa2\xb5\xaf\x7c\x92\x16\xdf\x28\x1d\x8b\x65\x2f\x14\x76\xf9\xe2\x8b\xab\x76\xeb\xe2\x17\xa5\xed\x56\x6f\xe3\x5c\xd8\x2f\xf5\x33\xcd\xd7\xba\xb1\x89\xb3\x48\xc3\x7d\xc8\xd8\xee\xeb\x77\x31\x49\x88\x22\xb5\x32\xf1\x25\x3c\x7e\x7d\x99\xd8\x8c\xe3\xcd\x88\xc5\x66\x3a\x41\x32\x0e\x92\x71\xe3\x09\x07\xc9\xb8\x6a\xc2\x6f\x44\x32\x6e\xa1\xd5\xc7\xb1\xa8\xd6\x59\x7d\x82\x7f\xa4\xb4\x52\xc7\x79\x05\xbe\x9a\x7b\x24\xf8\x0f\xd6\xbb\x42\x8e\x7f\xde\xc1\x7f\x10\xfc\x07\x95\x37\xc9\x9b\xf7\x1a\x1c\xe7\xd5\x70\x40\xa7\x41\x50\x23\x96\xcc\x37\xa8\x11\x47\x36\xdb\x60\x60\x0f\x06\xf6\x60\x60\x0f\x06\xf6\x60\x60\x47\x9b\x1a\xd8\x1b\x70\xd9\x43\x98\x53\x5b\x1a\x44\xfc\x56\xdc\x06\xc7\x29\x17\x1f\xd6\x6b\x10\x44\xe3\x25\xf3\x0d\xa2\xf1\x91\xcd\xb6\x85\x76\x91\x76\x59\xd8\x69\x5c\x65\x10\x39\x20\x34\xbd\x1b\x49\x53\x78\x7a\x47\xd0\x5e\x7c\x14\xec\xfc\xd5\x10\xea\x03\x9e\x7b\xc0\x73\x0f\x70\x2d\x01\xcf\x1d\x05\x40\x92\x00\x48\xd2\x66\x40\x92\x06\xcb\xf8\x16\xf0\xdc\x0f\x63\x61\x78\x43\x49\xca\x4e\x30\x94\x85\xd8\x0d\x2e\x57\x05\x6f\x80\x95\x20\x9d\x25\x1c\xc7\xcb\xc0\x62\x9c\x1c\xe9\x03\xc6\x2c\x11\x3d\x4d\xdb\xbf\x2c\x2a\x4f\xad\x95\x3c\xdd\x58\xcd\xc8\x0f\x65\x3e\x68\x8d\xc2\xe5\xa6\xdd\x0a\x35\xab\x58\xbb\xb5\x85\x1b\xfa\xa8\x02\x6a\x0f\xbb\xa3\xdf\x64\xd1\xd6\x70\x4c\x57\x1f\xd3\xc3\xd5\x47\xa9\x3a\xba\x8d\x0d\x21\xf2\xaf\xc7\x74\x76\x0f\x84\x7c\xfc\xf6\x4e\x6c\x40\x68\x0b\x08\x6d\xb5\x94\x39\x4e\x38\xe7\xd6\x28\x5e\xc1\x96\x16\xa0\x8f\x03\xf4\xf1\x2e\x89\x13\x2c\x8d\xc1\xd2\xd8\x3a\x4b\x63\x9b\x74\x88\x3d\x96\x4e\xd9\x4e\x9b\x38\x2a\x4b\x40\xd0\x26\x82\x36\x51\x31\xb5\xa0\x4d\x7c\x85\xda\x44\x3b\x28\x1c\x54\x89\xa0\x4a\x04\x55\x22\xa8\x12\x41\x95\xd8\x39\x19\x83\x2a\xf1\x3a\x65\x55\xaa\xf4\x89\x86\x29\xa9\x47\xa5\x4c\x04\x45\x22\x28\x12\x41\x91\x08\x85\x63\x96\xcf\x29\x14\x8e\x09\x85\x63\x42\xe1\x98\x37\x50\x38\xe6\x90\x22\x5c\x0d\x5a\xf9\x71\xa4\xd9\x1c\x85\x10\xf7\x6a\x79\x36\x6f\x4f\xa4\x0b\x99\x43\x21\x73\x28\x98\x28\x43\xe6\x50\x30\xc2\x05\x23\x5c\xab\x8d\x70\xaf\x65\x59\x3f\xf0\xf1\x3c\x90\xa0\x7a\x24\xd1\xce\xdf\x1d\x83\x34\x7a\xe0\xf8\x84\x60\x81\x0b\x16\xb8\x6a\xca\x1c\xa7\x2b\xbf\x35\x52\xc0\x31\x56\x8e\x0d\x1a\x40\x73\x42\x84\x20\x85\xe6\xb4\x0a\x41\x0a\x4b\x88\x13\xf4\xa3\xa0\x1f\xb5\x4e\x3f\x7a\x65\xc5\xa1\xb5\x21\xce\x41\x83\x30\xef\x05\x0d\x22\x68\x10\x6f\x54\x83\x68\x0d\x85\x83\xfa\x10\xd4\x87\xa0\x3e\x04\xf5\x61\x39\x71\x82\xfa\x10\xd4\x87\xa0\x3e\x1c\x4d\x58\xf3\x31\x29\x10\x41\x79\x08\xca\x43\xbb\x95\x87\xd6\x4c\x28\x04\x00\x87\x00\xe0\x10\x00\xfc\xd5\x04\x00\xbf\x51\x05\x7e\xb7\x62\xdb\x7f\x58\x42\xbd\xf3\x04\x8c\x4c\x12\x79\xf7\x7d\xc2\x87\xfd\xf9\x8c\xe8\xff\xbd\xa4\x53\xc2\x24\x50\x82\xaa\xb9\x2f\xa6\xd5\x6c\xa8\xc5\xad\xf4\xee\xbe\x77\xfd\xe9\xca\x2f\x0f\xf4\xee\xf3\xc3\x55\xbf\x77\xdb\xb9\xcb\x96\x3b\x9b\x95\xbf\xc4\xf6\xbb\x82\xa4\x69\x4f\xf2\x1d\xd1\x2a\x35\x30\x83\x7b\x85\x55\x2a\x37\x1b\xd9\x5d\xf7\xbe\x7b\xf7\x33\x94\x37\x1a\x5c\xf6\xee\x3b\xdf\x5f\x15\xf6\x79\xe1\x79\xe7\xe2\xa7\x87\xde\x5d\xfd\xf3\xee\xdf\x7b\xf7\xfd\xfb\xba\xa7\x77\xdd\xab\x6e\xe7\xbe\xfe\xeb\x8f\x9d\xde\xd5\xc3\x5d\x77\x29\x3d\x96\x8e\x76\xb9\x6e\x25\x81\x48\x50\xe2\x03\x45\x96\x19\x8a\x9c\x86\x28\x93\x8a\x1d\x97\xaf\xea\xeb\x1c\x3d\x58\x53\x05\xb5\x8d\x9b\x7b\xc3\x6b\xc8\xe8\x58\x31\x95\x78\x98\x90\x78\xa1\x25\x47\xc3\xba\x96\x70\x61\x50\x2f\x58\x7a\x92\xb4\x66\xe5\x91\x39\x3e\x08\x8a\xae\x29\xc2\xe2\x8a\x3e\xcc\x3a\xd4\xf6\xc0\x34\x4b\xa6\xcf\xa4\xd0\x53\x94\x0a\x41\x98\x4a\xe6\x88\x7c\xa1\x52\xc9\x85\x46\xdd\xf2\xd5\x35\x6b\x19\x42\xd6\xe0\x04\x4b\x34\x24\x84\x15\xc7\x2f\x48\x42\xb0\xac\x18\xb3\x5d\xfd\x66\x64\xc9\xd6\xca\x1a\x99\xcc\x1d\x3b\xc2\x34\x49\x05\x29\x9d\x16\x3e\x9d\x61\x41\x25\x67\xdd\x2f\xfa\x8a\xd6\x07\xf9\x06\x3e\xe7\x62\xb3\x13\xd3\xfd\xc9\xdf\xc1\xd7\xc5\x7f\x7e\xea\x17\xff\x55\x38\xf3\x57\xfd\xe2\xbf\x96\xef\x75\xaf\xe1\xf2\xce\x3e\x45\x9f\xfa\xe7\xe8\x13\x40\x8c\x0a\xd4\x9f\x60\xb3\x63\xaf\xfa\xe7\xe8\x8a\x48\x09\xbf\xe4\x1f\x2b\xaa\x12\x98\xdb\xf7\x94\x61\x31\x47\x6e\xfa\xa6\x72\x1f\x8e\x26\x88\x64\xa4\x29\x13\x8f\xfd\x23\x65\x60\x91\xc8\xa9\x77\xc5\xc7\x34\xc2\xc9\x76\x44\xec\x5c\x17\xf8\xc0\xcd\xdd\x52\x52\xf8\x6f\x2f\xd2\xa2\x73\x7d\x09\x55\xf1\xdc\x50\x2b\x66\x7e\x4d\xa4\xde\x24\x11\x67\xb1\xf5\xa9\x69\xa1\x66\xee\xe9\x2a\xff\xe0\x50\x59\x30\x95\x94\x8d\x75\x8b\xe8\x03\xba\xb9\x7b\x64\x37\x22\x36\xf6\x5d\xa2\x85\x7c\xb3\xe7\xa8\x44\x8c\x2b\x44\xa7\x33\x2e\x14\x66\x4a\xeb\x37\x20\xdd\x58\x8a\x18\x0e\x70\xc1\xa7\xd3\x54\x61\x7d\xd0\x16\x88\xca\x8c\x95\xe7\x9e\xa8\x5e\x0c\x8e\xb0\x0a\x1a\x1a\xf1\x27\x9f\xcb\x4c\xe8\xf6\xb5\xe8\x55\x34\x0d\xd0\x78\x41\x43\x77\x4d\x60\x21\x70\xf1\x02\x7e\x47\x15\x99\x96\xdf\x6f\x78\xed\xfe\xbb\xd2\xee\x71\x61\xb2\x22\x88\xe8\x88\x68\x42\x15\x89\x94\x3e\x82\x1b\xed\x89\x87\xeb\x1f\xaf\x6f\x7e\xf1\x05\xa3\x77\x9d\xcf\x97\xff\x55\x80\x81\xed\xdc\x7d\x5e\xf8\x61\xf0\xf3\x7f\x2d\xfc\xf2\xff\x5f\xba\x9f\xca\x3d\x2d\x98\x2f\xbc\xb9\x9c\x82\xa6\x00\xa6\x6e\x37\x55\x44\xa7\x78\x4c\x90\x4c\x67\x7a\x07\xc8\xb3\xe2\xfa\x6a\x49\xf9\x8a\xe3\x98\xb2\xb1\x29\xfe\x76\x45\x15\x11\x38\xf9\x8c\x67\x1f\x9d\x59\x7e\x03\xea\xfc\x9f\xfb\x42\x01\xc2\x77\xbf\x76\x3e\xfb\x25\x0c\xdf\xdd\xde\xdd\xf4\x6f\x96\xce\xba\xd0\xc2\xe2\x31\xd2\x8f\xcf\xe1\xff\xa3\x0f\x48\xb7\x9e\x09\xf4\x53\xa2\xb0\x56\x74\xd0\x1f\x4c\xbd\xac\x2c\x13\x86\xb2\x04\x4e\xcd\x4c\xd0\x29\x85\x2b\xc5\x18\x26\xbf\x31\x3a\x43\xa6\x14\x65\xe7\xc6\x7c\x00\x46\x00\x77\x29\xb3\x18\x8b\x18\xfd\x43\x96\xeb\x61\x82\x3d\xdc\xfc\x40\x62\x74\x8a\x26\x4a\xcd\xe4\xf9\x87\x0f\x2f\x2f\x2f\x67\xfa\x6d\x2d\xc0\x7e\xd0\x7f\x9c\x12\x76\x36\x51\xd3\xc4\xd4\xff\xd4\x54\x38\x47\xb7\x82\xeb\x2b\x04\xec\x0e\x44\x50\x9c\xd0\x7f\x92\x18\x0d\x0d\xff\xe3\x23\xf4\x5b\xc4\x05\x39\xcb\x17\xc6\xda\xca\xec\x3d\x62\xed\x69\x1f\xf4\x4b\x15\xcc\xa4\xbc\x9e\x28\x26\x11\x8d\xad\x98\x41\x58\xc4\xc1\xa0\x6a\x5c\x30\xba\x3d\x57\x64\x4c\x2b\x6a\xb3\x54\xe5\xe4\xf4\x74\x30\x1c\x13\xaf\x7c\xa7\x95\xaf\xb3\x0d\xa7\xf5\xb9\x9e\xd1\xc6\x53\x49\x04\xdc\xad\x18\x6e\x55\xf7\xea\x4c\x4f\x38\xe2\x09\x1a\xa6\xa3\x11\x11\x7e\xf8\xc0\x89\x56\xd2\xa8\x44\x82\x44\x7c\x3a\x05\x89\x41\x7f\x95\x4a\xb3\xab\x81\x62\x76\xb4\x67\x8f\x0c\xd6\x5f\x6b\x6f\xb0\x03\x62\x0e\xac\x8e\x11\x12\x23\xcc\xe6\xa6\x9b\x61\x3a\xf2\xdb\x37\x75\x75\x71\x8c\xa8\x7a\x64\x9d\x24\x41\x82\x4c\xb9\x22\x5e\xf9\x34\x70\x75\x16\x09\x0e\x2c\x52\x90\x59\x82\x23\x12\x9b\xfd\x90\xf0\x08\x27\x68\x44\x13\x22\xe7\x52\x91\xa9\xdf\xc0\x1f\xc0\x04\xa5\x69\x46\x25\x8a\xf9\x0b\x4b\x38\xb6\xf3\x28\x7f\xf6\x4d\xf1\x34\x76\x5d\xcd\xd3\xae\x10\x5c\xc0\xff\xfb\x91\xb2\x78\x67\x1c\xea\xe1\xbe\x7b\xe7\xff\xfb\xfe\xd7\xfb\x7e\xf7\xf3\x7a\xdc\x27\xdb\x59\x30\x3c\x30\x4d\x9c\xa3\x7b\x43\x04\x2e\xb4\x44\x24\x6a\x26\xf5\xd9\x6e\xa5\xfc\x07\x1e\x6f\xc8\x7d\x3f\x77\xae\x1f\x3a\x05\x8e\x72\x7f\xf1\x43\xf7\xf2\xa1\xa4\x0f\xd8\xf9\x15\x64\x78\xa3\xd5\xfa\xbf\x5d\xfc\xd0\xbb\xba\x1c\x54\xe8\xc1\xef\xee\xba\x17\x37\x3f\x77\xef\x72\x95\xb5\x92\x44\xa5\xc1\x94\x99\x55\xdf\x30\xa5\x09\x8f\xd1\x70\x5e\x5d\xe1\x56\x4b\xce\x09\x78\xce\xf3\x1a\xcf\xa6\xd5\x73\xe0\x4d\xae\xd8\x70\xfe\xc5\x94\xc7\xe4\xc4\xbe\x03\xa5\x81\x8d\xcd\xc8\x48\xcc\xd5\x0d\xeb\xde\x31\xf3\xec\x2f\xa6\x6a\x6f\x46\xb8\x73\xd4\x41\x52\xbf\x98\xea\x43\x2d\xe8\x78\x0c\xf6\xd0\xd2\x50\x4d\x6b\xf6\x53\x20\x2f\x7c\x67\xd6\x7f\x26\x38\x9c\x73\xdd\xad\x35\xa4\x67\xc6\x16\xf3\x21\x94\x91\x2e\xb6\x28\x30\xd8\x51\x2a\x86\xe6\x16\x4b\x13\xa1\x96\x5e\xe6\x3c\x1a\x33\x98\x3e\x5c\xc0\xb6\xa4\x31\xe3\xce\x04\x79\xa6\x3c\xf5\x3e\xb5\x95\x8a\x0b\x2b\x5e\xd9\x7c\x4e\x00\x20\x9b\xb1\xf5\x94\x9a\xc9\xb6\x47\x65\x0b\x9a\x85\x3d\x43\x0b\x23\xc1\xa7\x15\x6d\x14\x8f\x49\xef\xe6\x5e\x09\xac\xc8\x78\x7e\x69\x59\xc6\xe6\xc7\xe3\xf2\xe6\x97\xeb\xab\x9b\xce\xe5\xa0\xdb\xf9\x54\x3c\xf1\xd9\x93\xfb\xfe\x5d\xb7\xf3\xb9\xf8\x68\x70\x7d\xd3\x1f\xb8\x37\x96\x6e\xf9\x9a\x0e\x16\xef\xe9\xe2\x8b\xe7\x48\xb3\x5c\x60\x8d\x2f\x34\x49\xf4\x65\xe2\xf1\xc7\x21\x19\x71\x61\xf8\xfc\xd4\x05\x9a\x58\x11\xc6\xd1\xd6\xea\x62\xa5\x59\x9c\x83\xc1\xaf\xaa\x49\x63\xcc\x57\x82\xe0\x29\xdc\x13\x98\xa1\x2e\x8b\x4f\x6f\x46\xa7\xf7\xe6\xc7\x29\x16\x4f\x44\x64\x9f\xbe\x08\xaa\x14\x61\x05\x95\x0e\xbb\x21\x67\x4a\x62\xde\xc1\x19\xba\xd3\x7c\x5f\xbf\x9f\x5d\x6a\x7a\xb3\xc7\x44\x61\x9a\x48\x3b\xd8\x02\x5d\xcf\xd1\x15\x16\xe3\xdc\xbc\xf8\x07\x3e\x1a\x99\xc6\xbe\x31\xc3\xd0\x77\x58\x61\x16\x15\xbc\x57\x6f\x0d\x77\x2f\x42\x7f\xf6\xe5\x4c\x1e\x5e\xdc\x55\x0f\xb3\xed\xf6\xd4\xc3\x2d\x50\xdc\x68\xec\x05\xdd\xd0\x3e\xa9\xd8\x6b\x30\x71\xf3\x78\xf9\x25\x53\xdd\xf6\xe2\x76\x2a\xbe\x58\xb1\x9d\x4c\x85\x16\xbd\xf2\x23\xad\x6d\x56\xec\x25\xf2\x85\x5a\x83\x81\x3f\xee\xd2\x16\xca\x9b\x01\xab\x31\x9e\xcd\x08\x16\xb2\x6a\xb5\x8b\x62\x60\xcd\xda\x9b\x9e\xfc\x3e\xec\x22\xbb\x7e\x4e\x10\x67\x60\x70\xc8\x84\x88\xd2\x8e\x6c\xb0\x07\x4c\x5b\x0b\x3b\xe0\x16\xca\xc7\xdf\xd8\x52\xed\x9f\xa9\xd4\x4a\xa3\xf9\xf1\x7b\x5b\x43\x7e\xb3\x0d\xf1\xb1\xd3\xbb\x2a\x09\x17\x83\xcb\xee\xc7\xce\xc3\xd5\x72\x33\x61\xe1\xbb\xf2\x12\xa3\x53\xa4\x9f\x17\xc3\x01\xe8\xc8\xdc\x19\xae\x12\xbe\x51\x69\x09\x03\xa3\x95\xad\x52\x6d\xcc\xf0\x31\x99\x25\x7c\x3e\x25\x0c\x4c\x3c\x85\x9b\x50\xd3\x73\x84\xa9\xbd\x5a\xbc\xc1\x82\x15\xc7\x9a\xdd\xe0\x1a\x3b\x75\xe5\xf7\x49\x9c\xdd\xbc\xc5\xea\xfb\x25\xd6\x7d\x6b\x9c\x82\xf6\x7f\xee\x15\x56\x1b\x9e\xb1\xce\x45\xbf\xf7\x73\xb7\xa8\x1f\x5e\xfc\xd0\xfb\xb9\x4a\xaa\x19\x7c\xea\x5e\x77\xef\x3a\xfd\x15\xc2\x49\xa9\xc9\x2a\xe1\x44\xea\x01\x97\x9d\xc2\x54\x66\x81\x4e\x91\xa9\xe1\x8f\xa8\x92\xe8\x99\x4a\x3a\xa4\x09\x55\x73\x64\x1d\xac\x0f\x3d\xe0\xac\xcf\x38\xa1\x31\x55\x73\x27\xbe\x98\x7e\x8b\xeb\xa8\x39\xa9\x6d\xdf\x98\x1d\x7c\xb7\x2b\x58\xf9\xcc\xe2\xb8\x49\x9f\x23\xd0\x6d\x9f\x41\x69\xf3\x3e\x63\x5a\x90\x66\x63\x22\xcc\x70\xc0\xa9\xe4\x8f\xc5\x7b\xae\x47\xe5\x0b\x2b\x39\xd5\x32\xa1\x75\x4c\x18\xd1\x2c\xd2\xeb\xc4\x08\x52\x82\xb0\xf7\x5a\xe6\x9a\x25\x34\xa2\x2a\x99\xa3\x08\x6c\x58\x60\xce\x9c\x62\x86\xc7\x56\x38\x00\x35\xa7\xb4\x25\x7e\x4a\xc1\x00\x7f\x33\xb2\xa6\xfd\x3e\x25\x1b\x1e\xb3\x87\xeb\xcb\xee\xc7\xde\x75\x71\x0b\xfc\xd0\xfb\x54\x10\x61\x3f\x77\x2f\x7b\x0f\x85\xdb\x5c\x4b\xb2\xcb\xe5\xfa\x72\xb3\x15\x47\x31\x7b\xe9\x1c\x5d\x9a\x4f\xcf\x35\x71\x7f\x37\x93\xd3\x5b\x46\x9a\xe9\xe5\xca\x6f\x89\x0e\x77\x2e\xd2\xd0\xfd\xd1\x65\x4a\x54\xfa\x25\x9a\x9a\x90\xac\x57\xa8\x60\x43\xaa\x8e\xc0\x58\xe8\xfb\xba\xec\x2b\x2f\x4f\xd9\xbd\x08\x21\xb2\x67\xb9\x65\xc9\x0f\xcd\x00\xa3\x41\x9d\x11\xab\xc2\x5b\x97\x33\xec\x9f\xc1\xf3\x3e\x4d\xa5\x32\x1e\x52\xd8\x9c\xe8\xe9\x6f\x52\x13\x14\x3c\xa8\x67\xe8\x9e\x90\x47\xe6\xac\x07\x63\xaa\x26\xe9\xf0\x2c\xe2\xd3\x0f\x4f\xe9\x90\x08\x46\x14\x91\x1f\xf0\x8c\x4e\xb1\x96\xa4\x89\x98\x7f\x18\x26\x7c\xf8\x61\x8a\xa5\x22\xe2\xc3\xec\x69\x0c\x81\x3d\xce\xd3\xf5\x21\x6b\x76\xcc\xff\xf3\xea\xbb\x6f\x4f\xaf\xfe\xf6\xed\xbb\x45\x0b\x59\xdd\xfa\x77\x59\x84\x67\x32\x4d\x6c\x20\xa0\xf0\x69\xe3\x8e\x7c\x4a\x56\xad\xf7\x75\x71\xb9\xb6\xd3\x5f\x2f\x6e\x1f\x0a\x16\xeb\xe2\x3f\x3f\x77\x3f\xdf\xdc\xfd\x5a\xe0\x94\xfd\x9b\xbb\xce\xa7\x02\x43\xed\xde\xfe\xd0\xfd\xdc\xbd\xeb\x5c\x0d\xdc\xc3\x6d\x6c\x6f\x3f\x32\xfe\xc2\x8a\xa4\x91\x8e\x03\x2e\xf4\x74\x8e\x3e\x72\x81\x7e\xcc\x56\xf2\x74\x88\x25\x5c\x31\xee\xce\x92\x27\x68\xc6\x63\x60\xbc\x88\xcc\x26\x64\x4a\x04\x4e\xac\xcd\x40\x2a\x2e\xf0\xd8\xdc\xf4\x32\x12\x58\x45\x13\x24\x67\x38\x22\x27\x28\x82\xdd\x30\x3e\x81\x45\x01\x55\x8b\x8f\xcb\x76\xbe\xbb\x94\x29\x3a\x25\x4e\x05\xb7\xff\xec\x9b\xc5\xd8\x60\x71\x6e\xfa\x3f\x14\x85\xbd\x8f\x57\xbf\xf6\xbb\x83\xfb\xcb\x1f\x97\xd2\xd3\x7c\x56\x18\xd9\x3d\xc4\x55\x5d\xf0\x24\x9d\x32\xff\xef\xcd\xc7\xd6\xbb\xee\x77\x3f\x95\x47\x77\xd3\xe9\x17\x77\xc6\x5d\x31\x6e\xef\xdd\xf7\x37\x37\x57\xdd\x82\xa7\xfb\xdd\x65\xa7\xdf\xed\xf7\x3e\x17\xf6\xcf\xe5\xc3\x1d\xf8\x80\x96\x4e\xd3\x8d\xa0\x62\xa2\x7a\x5a\xfe\x34\x77\xcd\x0a\x1b\x71\xa2\x8e\x0d\xff\x37\x67\xf9\xd4\xc3\xcb\x31\x51\x6e\x60\xd5\x39\xcd\x4c\xaa\x91\x19\x69\x25\x3b\x54\xc5\x65\x42\xf5\xec\x78\xe9\x42\x2f\xe3\xca\xfd\x6c\x08\x30\xae\x33\xa3\x6c\xe3\x24\xe1\x2f\x26\x42\x79\x4a\xf5\xad\x2c\x09\x04\x2a\xeb\x57\x64\xee\x21\x3c\xab\xe0\x78\xc5\x65\x21\x91\x20\xea\x33\x4f\x99\xda\x7c\xcb\x75\xae\x0b\x7c\xa7\x7b\xfd\xf3\xe0\xe7\x4e\x71\x07\xf6\xae\x96\xb3\x1a\xbf\x89\x8a\xab\xb8\x73\xfd\x6b\x76\x09\x43\x1c\xfb\x49\xa6\xa1\x1a\xd9\x35\x4a\xa8\x16\x7b\x23\xac\xb5\xd7\x04\x24\x1a\x44\x28\x98\x1c\xa6\x7a\x72\x10\x37\x3b\x33\xfe\x24\xc3\x9f\xcc\x20\xcf\xdd\x1f\xa5\xf6\x24\xd0\x05\xac\xa9\x2e\x4d\x00\xda\xb1\x5a\x35\x43\x84\x3d\x53\xc1\x19\x08\xdb\xcf\x58\x50\x2d\x8d\x9b\x96\xf5\x5c\xcf\xe1\xff\xaf\xd7\x26\x18\x46\x4b\x8c\xeb\x9e\x0b\x75\x99\xc5\x27\x6f\x66\x0d\xa9\x8a\xd3\x5d\x8c\xd0\xad\x36\x74\x2c\x7e\x5b\xb1\x38\x5b\xc6\x31\x17\x27\xfc\x7b\x72\x49\x71\xa2\x19\xc0\xee\xe4\xc5\xce\xf5\x7d\xaf\x28\x3f\x16\xd5\x0c\x8f\x2f\x6f\x2c\x2f\x82\xa1\xd2\x8c\xdc\x29\x13\xf7\x3f\x5d\x19\xed\x42\x6f\x12\x7b\x6e\x3d\xc5\x02\x04\x20\x57\x5b\x75\x86\x85\x2c\x7d\x21\x11\x20\x99\xe5\x71\x64\xfa\xce\x82\x28\xad\x67\x4e\xe3\x47\x46\xbe\xcc\x08\x93\x10\x1c\x60\xee\xb3\xdc\xd7\x2e\xcf\x50\x6f\x04\x2c\x41\xbf\xce\x50\xca\xac\x03\x4c\x5f\xb8\x66\x90\x27\x5a\x94\xb5\x43\xc8\x34\x44\x30\xbc\x30\xe2\x62\xc0\xf2\xc1\x3f\xb2\x5f\x32\x27\x1a\x3c\x1a\x71\xcd\x80\xf4\x2a\xda\xf6\xce\x11\x66\x92\x9e\x20\xad\xb0\x94\xd7\x14\x32\x22\xb4\x42\x69\x23\xd3\x34\xa7\xb1\x7f\x1e\xfe\x1a\x58\x08\x7f\xf6\x2f\x83\xea\xbb\xa0\x74\x15\xd4\x88\xc6\x89\xf1\x98\x0c\x9a\xdf\x09\x11\x17\xc4\xfa\x59\xd6\xbe\x06\x56\x31\xf6\x3e\x96\x4f\x0b\xbe\x87\x1e\x93\x0a\xb3\x88\x5c\x24\x58\x6e\x18\x84\xe4\x6c\x1c\x27\x45\x89\xe3\xee\xee\xe1\xb6\xdf\xfb\x7e\x05\x97\x2f\x7f\xbc\x18\x06\x14\x25\xa9\x73\xcf\x0d\x05\xc7\x31\xd2\xec\x73\xcc\x8d\x2b\xd0\x0a\xfe\xe6\x04\x99\x35\xa1\xd2\x8b\x13\xc5\xf2\xa9\x60\xa4\xb6\x59\x16\xd6\xce\xe1\xbb\x12\xa8\x25\x04\x8a\x34\x25\x90\x67\xf2\x70\x4b\x0d\x9e\x45\x13\x45\x67\xad\x5b\xb3\x04\xab\x11\x17\x53\xc3\xe5\x0b\x93\x36\x8d\x2f\x6f\x94\x32\x45\x84\x48\x67\x0a\x54\x76\x3d\xd6\xb2\x94\xaa\x97\xec\x8a\x8f\x3f\x13\x29\xf1\x98\x6c\xe3\x80\xae\x52\x1e\xee\x7f\xf6\xff\x09\x0e\xe6\x26\xb2\x7f\x61\x84\x2e\xa0\xdf\xed\xa7\x1b\xf6\xd1\x04\xf2\xdc\xf2\x84\x46\x1b\x06\xdc\x7d\xec\xf4\xae\x06\xbd\xcf\x5a\x89\xef\xf4\xbb\x57\x05\x51\x02\x9e\x75\x3e\xf6\xbb\x77\x83\xee\xdf\xbb\x17\x0f\xfd\xce\xf7\x57\xdd\xc1\xf5\xcd\x65\xf7\x7e\x70\x71\xf3\xf9\xf6\xaa\xbb\x22\x32\xa7\xb6\xf1\x45\xeb\x6a\xf9\xd5\xf3\x85\x5f\x60\x85\x35\x2f\xf3\xed\x65\x90\x0c\x87\x69\x02\x4e\x70\x6e\x9c\xe1\x18\x31\x1e\x13\xf8\x59\x3a\xeb\x8c\xcb\x36\x39\x43\x3d\xf5\x3e\x49\x10\x4e\x15\x9f\x62\xf0\xda\x24\xf3\x47\x86\x87\x9a\xb5\xe2\x24\xf1\xc2\xbb\x44\xca\x98\x66\xb1\xba\x31\x69\xe2\x8b\x13\xa2\xd9\xf9\xcc\xcb\x61\xb4\x7e\x83\x11\x65\x10\x40\x3c\xc5\xe2\xc9\xb8\x99\xf2\x2e\xf3\x43\x21\x11\x96\x8f\x4c\x8f\x8b\x58\xc3\x50\x13\x0a\x9f\x37\x7a\xab\x96\x3a\x53\xfc\x44\x34\x55\xa6\x69\x34\x41\x33\xc1\xc7\x82\x48\x69\x6d\xcb\x11\x66\x26\x00\xc1\xbe\xae\xaf\xa1\x47\xc6\xb8\x26\x85\x33\x61\xc7\x64\x46\x58\x4c\x58\x44\x4d\xb6\x22\xf8\xee\x33\xd3\xe6\x58\xe0\xd9\x04\x49\x0e\x4e\x6f\x20\x3b\xd8\xaf\xcc\x47\xee\x26\x33\x33\x36\x8f\x7d\x0b\xb4\x48\x35\x9f\xb8\x01\x39\xd1\x50\x19\x3e\x76\x97\xa1\x73\xbb\x18\x3b\xe0\x74\x96\x10\xe8\xd2\x92\x1c\x16\x43\xd3\xba\xb0\x1e\x7a\x99\xaa\x16\x41\x5f\xd8\x6e\xcc\x58\xda\x11\x9d\x55\x58\xb6\xed\x91\x42\x3f\x60\x16\x27\xba\x15\xe7\xc3\x28\x9e\x45\xc8\xb0\xe9\xe8\x5d\xe3\x4e\xe3\x36\xb7\x68\x84\x53\xb9\xcd\x35\x5a\x4a\x31\x35\x56\xc1\xd3\x3c\x28\x04\xb6\xb7\xcd\x2f\x05\xea\xce\x34\x8b\xc4\x09\xb7\x54\x32\xaf\xa7\x36\x68\x19\x46\x53\x73\xcd\xce\x04\x65\x11\x9d\xe1\x64\x23\xdd\xaf\x94\x63\x60\x43\xf7\xff\x40\x47\x7a\xfb\x7c\xb3\xe0\xb6\x55\x44\x4c\x21\x9d\xdc\x0e\x33\x5b\xc2\x35\x2c\x49\x36\x59\x83\xc8\x3c\x9a\x04\x0b\x9e\x1a\x7f\x1c\xd0\x85\xc4\x15\x47\xf5\xac\x6a\xb9\xf5\xc9\xc0\xc5\x00\xe8\x0d\x16\xdb\x44\xfe\xd4\xd1\xaf\xd4\x8a\xed\xdd\x04\xe3\xe1\xe4\xb6\xba\xcd\xaa\x15\xf0\x1e\xfe\x7b\xd9\xde\xf9\x8c\x67\x7a\xcf\x44\xa9\x54\xe0\x29\xce\xe6\x68\x95\xa4\x52\x28\xbb\xe7\x3b\xcf\x82\xda\x9b\xaf\x46\x4e\x42\x1b\x00\xb5\xd8\x49\x21\x86\xc0\x43\x04\xb0\x7b\x7c\x94\x6a\x59\x16\x61\x88\x42\x40\x7f\x20\x67\xe3\x33\x74\xf3\x73\xf7\xee\xae\x77\xd9\x3d\x41\x9d\xdb\xdb\xee\xf5\xe5\x09\x22\x2a\xfa\xc6\xc5\x2c\xda\x80\xa5\x47\xa6\xb8\x95\x56\xe6\x68\xc2\x5f\x80\x37\x12\x31\x26\x85\x39\xbb\xe8\x26\x08\x55\x1e\x53\xa9\x6c\xf8\xac\xe6\x2b\xf9\xb0\xb4\xbc\x5f\xb9\x43\x52\x35\xd9\x66\x6b\x60\x29\xd3\xa9\xd6\x65\x07\x14\x4f\x07\x82\x27\xdb\x30\x85\x4b\x98\x0a\xa8\xcb\x19\x98\x02\xc5\x53\xa4\x9b\xb5\xa1\x20\x99\xcb\x31\x13\xe9\xb4\x60\xa4\xf9\xb2\xbe\x37\xbd\x7b\xcb\x79\x1f\x6c\x3c\x1a\x75\x21\x10\x00\xb6\x50\xc3\x2a\x72\xb3\xf1\xc0\x5a\xea\x07\x38\x8a\xb4\xca\xbd\xe3\x49\xe5\x1d\x65\x2e\x01\xdb\xd1\xde\xa6\xb9\x6a\x9f\xbb\x61\xce\x34\x07\x83\x60\x60\x7d\xe5\x4a\x1e\xd1\xbc\xfd\x8a\x7e\x87\xf3\x85\x5e\x61\xcb\x9e\x3d\xb2\x07\x99\x99\x54\xcc\x25\x2c\x09\xac\xa4\x44\x2f\x13\x02\x47\x63\x8e\x26\xf8\x99\x14\xba\x74\x39\x24\xba\xe1\x39\x4f\x45\x15\xa3\x7b\x64\x97\x64\x26\x88\x96\xf4\xcb\x0e\x94\x6c\x4f\xdf\x15\x77\x62\xd8\xd7\x61\x5f\x1f\xfd\xbe\xbe\x48\x52\xa9\x88\xe8\x48\x49\xc7\x60\x48\xdc\x4a\x80\x33\x8d\x0d\x66\x9c\x27\x83\x06\x36\x91\xe6\x14\x2f\x78\xc2\x0a\x01\x1f\xd2\x20\x1d\xf0\x14\xe4\xa3\xc2\xb5\xc9\xf5\x5d\xe7\x65\x0e\xdb\xe1\x2d\x21\x83\x73\x99\x75\x1c\xa0\xc4\x56\x22\x0e\xae\x6a\x65\x59\x4b\x68\xef\x62\xce\x85\x91\x6f\x32\x77\x59\x3e\xc4\xd2\x61\x72\xa2\x08\x65\x8e\x6c\xf9\x47\xb0\x9f\x35\x81\x8d\xdc\xf1\x7b\xca\x15\x96\xdf\x9c\x3d\x32\x2d\x44\x3d\x91\xb9\x31\xb7\x6a\x31\xe5\x8f\x5a\x16\x3f\x95\x84\x49\x08\xf7\xfe\xa3\x71\xcf\xe9\x2d\xee\xcc\xd5\x46\x35\x25\xd3\x59\x82\x15\x04\x5d\x67\xbd\x40\x88\xae\x6d\xd4\x4a\x49\x79\x00\x34\xc8\xf9\x66\x2e\xf6\x99\x19\xfe\x98\x28\xc8\x1c\x57\x54\x81\xce\x14\xa7\x9a\x3c\x8b\x43\x5f\x69\xba\x32\xbb\x42\x70\xf0\x93\xc4\xe9\x76\x8c\x5f\x2e\xb6\xb1\x92\x33\x66\xda\xc2\xbd\x8d\x79\xff\xe0\xec\x46\x91\xe0\xac\x14\x0d\xa3\x95\x39\xb3\xd2\x43\xc3\x0e\x9c\xff\x9a\xb0\xb3\x17\xfa\x44\x67\x24\xa6\x18\x22\xe0\xf5\xbf\x3e\xe8\x79\xfd\xe7\xc5\xdd\xcd\xf5\x20\xcf\xe4\xf9\xef\x47\xd6\x49\x24\xcf\xb2\x14\x10\xe3\x2c\x0b\xb7\x9f\x09\xe2\x44\x42\x3b\x17\xb0\xba\xe6\x66\xc4\x47\x56\x37\x82\x98\x47\xf2\x0c\xbf\xc8\x33\x3c\xc5\xff\xe4\x0c\x5c\xe9\x1d\xf8\xf3\x22\xe1\x69\xfc\x0b\x56\xd1\xe4\x03\x9c\x6b\xf5\x81\x3c\x13\xa6\x8c\x9b\x4a\x93\x2b\x86\x9c\x64\x09\xd1\xfa\xff\xa9\xc7\x9c\x27\x15\x49\xad\xc9\x46\x64\xa6\xd0\xff\x2b\xc8\x90\x73\x55\x7d\x49\xf1\xd1\x48\x92\xb5\x2e\xa4\x5c\x49\xbb\xbf\x41\x7f\xfb\xaf\x6f\xff\xac\xb7\xd0\x26\x34\xee\xdd\xdf\x0c\xf4\xf7\xff\x79\x69\xbf\x97\x6b\xb0\x3b\x93\x4a\x2b\xad\xab\xd9\x50\xc3\x04\xce\xa7\x0c\x6e\x3f\x01\xce\x0b\x60\x6f\xb0\x1d\xf2\x75\xac\xe2\x6e\x97\x85\xd6\xb7\x53\xd9\x36\x22\x26\xa8\xd8\xde\x1c\xd1\x29\x62\x1c\x4d\x4d\xac\x29\x66\xe8\xaf\x3f\x7e\x5f\xbd\x80\xa9\xa0\x1b\x75\x48\x2d\x0a\x85\xd7\xa5\xa4\xff\x24\x12\xe9\x5d\xa3\x77\x31\x9f\xea\xae\x05\x91\x13\x9e\xc4\xe8\x85\x80\x9a\x64\xe3\x40\x33\xad\x5c\x90\x47\xe6\x37\x01\x21\x87\x08\x27\x8a\x8f\x09\xdc\xd5\x4e\x51\x53\x44\x68\x51\xc5\x64\x69\x28\x2e\xc8\x89\x01\x66\xbb\xff\xce\xc5\x56\xc3\x34\xe1\x91\x4b\x6a\xb1\x26\xb9\x78\x58\x3d\xf3\x51\xd9\xf4\x8a\xea\x6d\xf8\xe5\x45\xb6\x66\xdb\x6a\xd2\xd8\x24\x14\x6b\xc3\x2a\xaf\x4c\xf5\x60\x68\xc4\xd9\x20\xa1\xec\x69\xa3\xc5\x70\x89\xe1\x48\xb7\x60\x69\xa6\x5b\xcc\xec\xdc\xc6\x02\xb2\xc6\xf9\xf8\x98\x26\x89\x49\x6d\xf1\x97\x07\xe4\x2e\x43\x37\x10\x06\x66\x26\x07\x94\xc4\xd6\xef\x65\x35\x61\x41\x18\x04\xbc\x3d\xb2\xe1\xdc\xfa\x6c\xe5\x09\x92\x69\x34\x71\x99\x79\x11\x67\x52\x8b\xd1\x5c\xa0\x88\x4f\xa7\x5a\xeb\x85\x25\x53\x9c\x27\xd2\x46\xbb\xb3\x53\x85\x23\xf5\xc8\xf2\xfe\x56\x9c\x3c\x53\x94\x69\xbb\xd4\xbd\xe6\x2e\x9d\xbc\xf8\xd3\x52\x81\x9b\xc6\x3e\x14\x05\x18\xc1\x8c\x27\xca\x03\xb5\xe0\x8b\x67\xc9\x2c\x58\x8d\x66\x20\x27\x5c\xa8\x41\x5c\xc9\x73\x56\x6e\x9a\x32\x23\x64\xe4\x34\x81\xa0\x61\xfe\xac\x85\x7f\xf2\x92\x19\x5f\x97\x0d\x41\xef\xea\x65\x23\x68\x76\x8c\x96\x8e\x6c\xdd\x2d\x58\x43\x2b\x03\x4c\x12\x15\x63\xc2\x57\x8d\xf1\x1e\xbe\xba\xd0\x1f\x2d\x25\x5e\xf9\xdc\x39\x21\x88\xc7\x39\x86\x9e\xb9\xd7\x6d\x46\xc8\x32\x9a\x5a\xe8\x84\xfd\x65\x8e\x2e\x9b\xca\x43\xd1\x92\xab\xc7\x02\x26\x7b\x49\x40\xd6\xc4\x62\x48\x95\xc0\xa2\x00\x80\x92\xe9\x83\x92\x60\x01\xf1\x59\x8f\xcc\xc0\xe1\x19\x4d\x21\x46\x31\x95\x90\x20\x02\x77\xa9\xe7\x0c\x43\xcd\x94\xc0\xd2\xd1\xce\xf3\x1c\x4d\xfc\x39\x04\x96\xe5\x5b\xc3\x31\x3b\xdd\x51\x06\xfb\xa5\xf5\x33\x1e\xa5\xb9\x20\x17\x81\x84\x6b\xa1\x82\x10\x65\x92\x8e\x27\x0a\x51\x66\xed\x8e\x38\x19\x73\x41\xd5\x64\x2a\x4f\xd0\x30\x95\x5a\x0b\x35\xc1\x6a\x26\x1e\x85\xa8\xa8\x11\x17\xda\x36\x89\x38\x2e\x35\xb8\xa8\xa2\x6c\xb0\x35\x9a\x1d\xca\x6e\xe9\xae\x58\xb1\x71\x3a\x19\x7c\x62\xb9\x0d\x4a\x64\x86\xba\x89\x4c\x1c\x20\x77\x80\x55\xbf\xa7\x44\xaa\xba\x73\x00\x60\x97\x3b\xf3\x52\x1c\xa2\x92\x16\x32\xc9\xa0\x82\xb8\xd8\x6d\x90\xbc\x8a\x80\x9b\x06\x94\x2a\x73\x3a\x4d\x67\xaa\x32\x70\x6b\xd1\x55\x74\xe7\x41\x19\x35\x23\x36\x24\x63\xc1\x6e\x06\x00\xba\x47\x76\x4f\x48\x3d\x3e\xdd\xc2\xda\xff\x06\x47\x09\xa6\x60\x13\x3d\x96\x6f\xf9\x6d\x9c\xd8\x97\xdd\xfb\x8b\xbb\xde\xad\x81\x9c\xb8\xb9\xfb\xdc\xe9\x0f\x2a\xfc\xda\x15\x6f\x7d\xee\xdc\xfd\x78\xb9\xfa\xb5\x1f\xfa\xc5\xac\xec\x8a\x57\xee\xee\x97\x27\x73\x34\x18\x62\x45\x52\x58\x65\x3f\xe7\x68\x36\x57\x13\xce\xb2\x10\x85\xb8\xc0\x9b\x4e\x91\xc9\x08\x56\x10\x42\x24\xa4\xaa\x70\x1c\xf6\x21\x2e\x67\xb5\x84\x59\x5c\x2c\x83\x2e\xb7\x53\xd1\x68\x8d\x13\xf9\x29\xe1\x43\xf0\x5b\x5b\xd9\xc7\x02\xd3\x2d\x89\x40\xdf\x32\xde\xe7\x92\xca\x59\x82\xe7\x0b\x3d\xac\xba\x72\xae\xf1\x94\x40\xc4\x71\x0e\x8b\xe7\x92\x45\xf4\xca\x40\x02\x53\x76\xaf\xd3\x11\x64\x32\x29\x8a\x15\x41\x43\xa2\x5e\x20\x6f\xce\xfd\x9a\xd9\x52\x5d\xc0\x88\x3c\x7b\x64\x60\xce\x79\xd4\x44\x8e\x53\x88\xf6\x7b\x7c\x77\x82\x1e\xdf\xc5\xe4\x99\x24\x7c\xa6\x57\x5e\xff\x50\x77\xc9\xcc\x19\x9e\xd2\xe8\x9a\xc7\xc4\x85\x68\xdc\x59\x24\xc7\xad\x6c\x8a\x10\x7c\x46\xe2\x81\xbb\x32\x9b\xcb\xc0\x17\xf6\x53\x37\x9c\x8b\x84\xcb\x0c\xf0\x05\x2d\x37\xfd\x74\xa7\x98\x26\xd7\x5c\x65\x76\xc6\x6d\xe6\x20\x48\x44\x67\xa0\x67\x0c\x88\x6e\xf7\x70\x62\x54\xe1\x5c\x3a\xe6\x0c\x63\x40\x38\x8e\x05\x91\x12\x18\xb3\x1b\x5e\x1e\xd0\xc4\xbc\xa9\x17\x4a\x77\xae\x23\x20\x65\xc6\x7c\xd3\xa3\xdf\x66\xd1\x88\x5b\xb5\x9f\xba\xec\x79\x8f\x0e\xe6\x6d\xc5\x12\xbd\xbf\x7e\x24\x73\x48\x28\xb9\xc5\x54\x6c\xe8\x68\xae\x8a\xe0\xdd\x8b\xcb\xb9\x5b\xd1\x51\x8b\x9c\xcf\xd5\x74\xd8\xce\x0d\x9d\x45\x1e\x1e\x4a\xe7\x76\x7c\x26\xeb\xb8\xa1\x12\xfe\x50\xa7\x72\xd7\x06\x64\xa0\xb2\x1a\x39\x23\xd1\x1a\xfa\x63\x36\xc0\x7b\xfd\xdd\x4a\xbd\x2b\x13\x3e\x5d\x34\x61\xbe\x0a\x36\xd5\xbf\x8c\x2e\x40\x56\x8e\x38\xb2\xbc\x78\x83\x41\x3b\x36\xbe\x6c\xdc\x5d\x7f\xfb\x6a\x29\x77\xad\xf0\x8c\x0a\xc2\x97\x10\x3b\xcd\xad\xa9\xac\xbf\xcf\xbe\x7d\x82\x28\xc4\x8e\x82\x7a\x99\xe4\x38\x08\x2c\x46\xb9\x53\xe7\x91\xe5\x11\x38\x12\xbd\x90\x04\x82\xf6\xf4\x2d\x07\x0e\x0b\x3b\x5c\xdb\x12\x89\x4d\xfc\xf3\x09\xe2\xa9\xd2\x8d\x99\x0c\x23\x67\x92\xb6\xe9\x4b\xb9\x13\xc7\x78\x12\x6d\x28\x7f\x86\xfe\x6d\xf6\xba\x91\x0c\x28\x43\x9f\x88\x82\x56\xa0\x68\x84\x3f\x41\xd0\x7a\xca\x01\xa1\xd5\xb4\xdf\xe2\x44\xd9\x99\xac\xb1\xf2\x39\x0c\xcc\xf7\x09\x1f\x2e\x37\x79\x40\xe3\xe8\xe1\xae\xe7\xec\xab\x79\x34\x98\x07\x31\x5d\xf0\x8f\x76\x6f\xef\xba\x17\x9d\x7e\xf7\xf2\x0c\x3d\x48\xa2\xc9\x93\x4d\x17\xb2\xc5\x33\x05\xcb\x8c\xdc\xe2\xca\x30\xa9\x08\xae\x33\xeb\x10\x21\x0a\x39\xdd\x2b\x18\x47\x11\x74\x66\xf9\xc6\x06\xc8\x17\x6a\xcd\x8e\x00\x93\x54\x9e\xa7\x8d\x33\x5c\x75\x02\x21\xea\x6b\x70\x3c\x31\x77\x66\xbc\xd3\xc5\x38\xc3\x55\xdb\xa7\x18\x9f\xb8\xef\xc9\xc0\xd1\x52\x13\x42\x05\x6a\x34\x2d\xb3\xa9\x06\xcd\xe7\xe4\x05\xec\x7f\xc6\xb3\xe5\xc9\xb4\xf8\xa5\xb0\x69\x8d\x60\xef\x45\x22\xec\xfb\x1c\x38\xb6\x36\x30\xac\x70\xfb\x09\xe6\xee\x39\xc3\x5b\x33\xbe\x69\xf2\x57\xa4\x33\xf9\xf9\x13\x2b\x0d\xc2\x46\xe5\x4a\x04\x67\x07\x7e\xa1\x0c\x15\xae\xc4\x13\x34\xa2\x5f\x6c\xa3\x79\xb4\xbe\x7b\xd5\x0b\xdf\xa8\x89\x0e\x9d\xe0\xc5\x33\xb5\x86\xd8\x70\x0b\xdf\x2f\x15\x22\xb9\xd4\x22\x51\xa4\xc5\x25\x41\x22\x2e\xf4\x4d\x01\xdd\xe6\x3e\x95\x55\x22\x83\xc2\x42\x13\x65\xd1\xc7\xb4\xec\xf4\xe7\x85\x62\x62\xac\xc8\xa9\x16\xbd\x56\xa4\x73\xdb\x8c\x1f\xc8\x0d\xc2\xca\x03\x37\xcb\x6f\x9e\x21\x19\x63\xe6\x02\xcd\x6b\x86\xeb\xae\xbc\x2d\x58\x95\x56\x81\x30\x24\xbb\x81\x7c\x05\x89\x4c\x85\x71\xc8\x19\xd0\x73\xe9\x38\x6c\x2c\x4f\x1b\xc8\xf6\x82\xb3\xd0\xa2\x9a\xc1\xa6\xb3\xb8\x4d\x83\x4d\xb0\x54\xc8\x8e\xa9\xce\xb0\xe2\xa9\x88\xfb\x35\x29\x17\x74\xfb\xa6\xca\x9b\xde\x42\x45\x2d\x96\x80\x9f\x47\x3a\x14\x18\x83\x79\xa3\x75\x1a\x27\x08\x5f\xc0\x0a\x65\x67\xfb\xce\x48\x59\xee\x96\xf0\x99\x09\xa4\x1c\x2c\x36\x7d\x86\x3a\x6c\x01\xfd\xcb\x45\x99\x15\xe8\x65\xee\x24\x9c\xbc\xe0\xb9\x44\x33\x61\x80\x72\x4c\x1e\x82\x9b\x3c\x68\x60\xc5\x8f\xb2\xc0\x0e\xe5\x12\x41\x10\x58\x96\x56\x87\x00\x3a\xb9\x77\xb0\x07\xc7\x64\x29\x46\x3e\x13\xc8\xf3\xe6\x72\x5b\x45\x03\x56\xa7\xc8\x20\x9a\x60\x36\x26\x03\x67\x32\xde\x44\x5b\xd2\xed\x5c\x40\x33\x97\xb6\x95\xea\xcb\xe9\xd6\x28\x4c\xb6\x48\x8f\x79\x35\x33\x87\xea\x43\x20\x15\x1e\x13\x64\x46\xd4\xc8\xc8\x5e\x88\x7f\xb3\xd0\xc9\xa0\x27\xd8\x56\xbb\xc5\x9c\x80\x3a\xe1\x1d\x02\xb9\xae\xf0\x90\x24\xaf\x13\x07\x02\x5d\x5b\x57\x03\xf8\x1e\x4d\x6e\x03\x41\x2f\xe0\x9d\x28\xb1\x0c\xeb\x8b\x10\x69\x55\xa6\xc3\xb2\x79\xc2\x91\xb3\x27\x6d\x9b\x89\xba\x82\x2e\x9b\x4c\xb5\xae\xcc\x8b\x7f\xed\x79\xe5\x50\xaa\x0c\x6c\xfe\xf5\x57\xb6\x90\x6f\x36\x10\xaf\x2a\x4b\xcd\x38\xb6\x2e\xcb\xb2\x72\x2a\x1b\x43\x26\x34\xac\xc0\xd8\x1b\x21\xc6\x19\x41\x54\xe6\x2f\xab\x62\x72\x57\x06\x38\xa4\x45\x7c\x63\x7c\xc9\x4a\xa9\x65\x15\xb2\xf6\x6d\x69\xc9\xa1\x20\x32\xdb\x80\xcb\x56\x67\x44\x2b\xaa\x58\xcc\x01\xb0\xd4\xf0\xe1\xa2\x4c\xb7\x72\x9c\x3b\x17\xb8\xfb\x0e\x8f\xd6\x8b\x3b\x56\x1c\x81\x18\x59\x1a\x1c\x32\xa8\xae\xf6\x25\xfb\x91\x05\xdd\x79\x64\x99\x65\x03\x36\x22\x95\x68\x8a\x67\xe0\xa1\x64\x5c\xe5\x5f\x19\x10\x29\x95\x2d\xe1\x89\x13\xc4\xa5\x29\x74\xb6\x9a\x02\x5c\x8c\xb7\x09\x3c\x69\x5e\xcc\xa2\xb9\x61\xc9\x5d\xfe\xf9\xaa\x16\xa1\x42\x1d\xcc\xf1\x98\x3e\x13\xe6\x4e\xd4\x89\x3b\x91\x9a\x24\x6e\xca\xc9\xfc\x14\x43\xc8\x36\x89\x7d\x2f\xd2\x72\x7e\xb8\xbd\x33\x66\x57\xd6\xd0\xe6\x24\xeb\x57\x86\x24\x19\xc0\xb9\x42\xa5\x00\x17\x64\xef\x9f\x11\x8b\x81\x6c\xb2\xea\xb1\x44\x7f\x64\x5c\xfd\xd1\x43\x89\x76\xa6\x13\xf8\xd4\x19\xc0\x4e\x16\xaa\xfa\x00\xcb\xb0\xdb\x16\x61\x0f\xad\x6c\x25\xe5\xb7\x8d\xb3\xc8\x93\x08\xf6\x2a\x0b\x77\x17\x33\x0a\xeb\xca\xa2\x85\xe8\x07\x54\xbe\x94\xca\xe6\x56\x53\x79\x31\x3f\xe9\x05\x33\xab\x5c\x15\xee\x90\xad\x45\xa3\x30\x87\x05\x74\x85\x6d\x76\xdb\xb4\x71\x14\xda\x0a\x40\xe9\x6a\xab\xc8\x26\x39\xb3\x75\x5a\x81\x28\x86\x01\xda\x92\x22\x35\x88\xc9\x67\x8f\xec\x23\x17\x56\x00\x90\xb6\x66\xc3\x10\x47\x4f\xa7\x84\xc5\x08\xa7\x6a\x62\x90\x8b\xad\x57\x63\x6e\x77\x83\x96\x73\x60\xdb\x64\xb0\x24\x54\x46\x58\xc4\xae\x7a\xc8\x33\x77\xa3\x78\x64\x5e\x23\x50\x15\x02\x6a\x81\x41\x91\xe6\x3a\x45\x97\x48\xad\xdd\xd5\xd1\xa2\xaa\x4e\xef\x42\x95\xde\xe5\xe7\xac\x50\x77\x18\xea\x59\x40\xb0\x18\x1f\x2d\x52\xa7\xe7\x6c\x9d\x4e\xbb\xd4\xfb\x79\xd1\x07\x72\x62\xf5\x19\x63\x10\xb3\x33\xd0\x72\xd6\xb7\x8e\xd7\x16\x10\x98\x47\xa9\x80\xd0\xe7\xaa\x36\xff\x10\x4d\x68\x92\x7b\x4e\xbe\x39\xc9\x86\xa9\x9b\x4c\xc8\x33\x49\x0c\xfe\x7f\x24\x20\xcb\xc1\xd8\x2c\xbf\x45\xff\xdb\xd4\xae\x45\x7f\x7e\x64\x9f\x80\x0d\x27\xc9\x1c\xd0\x49\xb3\x96\xb1\x2a\x35\xf3\x54\x39\x00\x65\xd3\xaa\x50\x71\x20\x66\xad\x27\xf8\x99\x3c\x32\xd7\xcc\xff\x46\x4f\xe8\x4f\xe8\xcf\x75\xca\xa5\x4b\x56\xd8\xb3\x95\xe5\xa3\x97\x0a\xe0\xdd\x72\x96\x51\x5a\x7e\xe3\x8c\x30\x05\x13\x68\x05\x4a\x49\x06\x32\x4e\xd9\x33\x8f\x16\x32\x62\xfc\x53\x8b\x05\x61\x6a\xc0\x78\x4c\x06\xa4\xc2\xa1\xba\x84\x49\x68\x21\xe0\x9a\xc7\x64\xa5\x3b\x34\x63\xa6\xbf\x80\xe1\x48\xa6\xc3\x6c\x39\x00\x2c\x21\xcb\x8c\xcf\x6c\x1f\xc5\x9d\x56\x3d\xf2\x0c\xc9\x77\x93\x71\x6f\xea\xca\xcd\xc5\x46\x9c\x43\x09\x57\xbb\x13\x13\xac\x9c\x34\x59\x3e\x8e\x65\x37\x84\x7e\x59\xcf\xdc\x5e\x56\x1e\x46\x31\xd4\x91\x11\x74\x4c\xb5\xf6\xd0\xdc\x5d\x0c\x9c\x70\x13\x5f\x8a\x01\x6c\x6d\xe4\x4c\xc9\x49\xe1\x40\x6b\x4e\xb3\xfd\x97\xbb\x40\x87\x3c\x2d\xab\x0f\x96\x00\x54\xfa\xc1\x06\x56\x53\x98\x6b\x3e\x3c\x36\xd9\x94\x64\x42\x0d\x7e\x41\xe7\xe2\x0a\xe9\xd3\xc1\xa7\x06\xe4\x0b\x88\x96\xaa\x09\x17\xf4\x9f\xcb\xf6\x36\x16\x8a\x8e\x70\xa4\x06\x3b\xa9\x89\x53\xbf\x99\x3a\xb6\x9f\x5e\x7d\xdd\xbd\x05\xbc\x06\xfc\x4c\xbc\x70\x4a\x08\x96\xb4\xad\xc8\xcc\x91\x5b\xe6\xb7\x5c\x20\xc6\x5f\x72\x90\x2f\xf7\x3d\xe0\x5a\x7b\x69\x28\x58\xab\x5c\x33\x88\x87\x96\x14\xf6\x27\x40\x6e\xbd\x57\x26\xc5\x14\xe0\xda\x0d\x58\x96\xde\x9e\x13\xcc\xe2\xc4\x5d\x21\x88\x9b\x88\x9e\xf9\x0b\x9e\xaf\xe5\x53\xf7\xa3\x44\xf3\x9c\x43\xb3\xfc\x45\x25\x08\x78\x80\x91\xd4\x54\x41\xd5\xac\x52\x84\xd1\x30\x05\x98\x60\x4d\x93\x51\x9a\x98\xda\x22\x11\x17\xf1\xd9\x23\xb3\xe1\xe1\x5e\x6f\x5a\x04\x74\x5a\x13\x56\x59\x83\xd4\xa2\xa9\xda\xea\x25\xc6\x2c\xb7\x54\xae\xff\x29\x25\xe9\x8e\x92\x44\x5f\x35\xac\xbe\x8f\xc7\x32\x8f\x93\x37\xb4\xd1\x57\x5e\x4e\xdf\xdf\xf5\x4c\xa5\x97\x56\xed\xcc\xc5\x19\x4a\x99\xb1\xb3\x98\x92\xbc\x6b\x99\xe9\xee\x4c\x75\x86\x1d\xd8\xe9\x0e\x11\xa4\xb3\x28\x7a\x56\x70\x75\xbb\xfd\x9e\xb3\x24\x63\x74\x18\xe3\x97\x2b\x73\x51\x12\xea\xf6\x68\x07\xdb\xe0\xee\x58\xd4\x55\x96\x06\xee\xe7\x56\xb1\xec\xb6\xa8\xc8\xf7\x57\x1c\x32\x83\x5e\x04\x05\x10\xc3\x79\xfe\x72\x56\x7d\xda\xdd\xc2\x3e\x8f\xd1\xc2\x9f\xd1\x16\x20\x58\xc7\x91\x70\x5e\x7d\x75\xae\x61\xd7\xb1\x0d\x15\xbb\x5e\x0c\xc6\xa8\x3b\x11\x86\x25\xb5\xf5\x48\x2c\x22\x18\xad\x3c\x0c\x59\xb1\x9a\xd7\xb1\x0a\x67\x12\xe3\xe1\x4e\x46\xb6\x1d\x07\x11\x8e\x26\xb5\x93\x1a\x72\x9e\x10\xcc\xea\x94\x82\xca\xc7\xe5\x23\x62\xf0\x77\x81\x75\x27\x09\x80\x50\x3b\x12\xd8\xc2\xa5\xb9\x56\xc4\x62\x28\x1e\x60\x78\xb8\x09\xf8\x74\x03\x55\x84\x39\x83\x1a\x65\xe3\x84\x94\x69\x65\xab\x3c\x9c\xd8\x4e\x92\x28\x4d\xbc\xca\xa5\x33\x22\xf4\xa8\x35\x89\x9f\x09\xd3\xaa\x98\x1d\x87\xf3\x50\xbd\xb8\x9c\xfd\xac\x5e\xd9\x49\xd6\xb5\x73\x92\x42\x62\x6c\xfc\xc8\xe0\xe0\xf2\xe2\x61\xd5\x7b\x55\x6a\xed\xcd\x37\xf7\x6d\x7c\x3a\x3d\x21\x62\xed\xe3\x79\x5f\xb4\xfd\xaf\x7d\x26\x4d\xdf\x03\x08\x1c\xd9\xda\x5f\xea\xf9\xd4\x72\x34\x11\xb3\xb0\x0e\x31\xee\x40\x9e\x01\x08\xc5\x29\xc6\x12\x7b\x91\x38\x75\x88\x65\x7b\xbd\x4b\xf2\x0a\x2d\xee\x36\x68\x38\x94\xa5\xf1\x07\x0d\xa3\x09\xc0\xe8\xbb\xec\xdc\x5e\x59\xa9\xbe\xe8\x87\xcf\x52\xd0\xf2\xd8\x55\x5b\x83\x58\x09\x0c\x00\x1a\x00\x7b\xf0\x8b\x31\x5c\x50\x69\x84\x7b\x57\x89\x65\x3a\x53\x73\x5b\xb8\x0f\xee\xc5\x82\xbc\x0f\xa0\x84\x55\x3e\xff\xf2\x1d\x19\x17\xbc\xfe\x55\x9d\x41\x47\xd6\x5a\x53\xd9\xa4\x23\xb4\x0f\x72\x53\x02\x15\xa9\x0b\xf1\x31\x35\x90\x07\x38\xa9\x35\x11\xee\x80\x69\x82\x72\x94\x03\x89\x58\x7c\x62\x25\x52\xa2\x79\x17\x4e\x92\xd2\xbc\x30\x64\xec\xab\xac\x0e\xe2\x30\x2f\xd6\xdc\x3c\x02\x21\xc1\x43\xb2\x56\xcc\xc1\x95\xf9\x60\xe9\x2e\x82\x57\x20\x5c\x7f\x36\x4b\xe6\xcd\xd2\x04\x7c\xed\xb7\x12\xc7\x6f\xd5\xc0\x7c\xf4\xbf\xa5\x77\x53\x11\x41\x6f\xb3\x21\x4a\x12\xa5\x82\xaa\xf9\xc0\xda\x52\x9b\x33\xad\x7b\xfb\xe5\x85\xfd\xb0\x89\xa1\xe2\x1c\xb9\xfe\x9c\xed\x16\xee\x29\x41\x4d\x91\x27\x3b\x85\x26\xcb\x8d\x53\x35\xa9\xc4\xf7\x5a\x46\x58\x07\x30\xd6\x6c\xa8\xba\x8b\x4d\x87\x67\x8b\xc7\x0c\xf8\xc8\x41\x77\x35\x27\x6c\xb9\xaa\xce\x1a\x46\x68\x87\x10\x3e\x13\x94\x0b\x5b\xbc\xa6\x49\xa4\xe2\x14\x7f\x19\xcc\xb0\xc0\x49\x42\x12\x2a\xa7\x9b\x9b\xcc\xbf\xfb\xcb\xd2\xd1\x5e\x98\x22\x4b\x66\xb0\x53\xfc\x85\x4e\xd3\x29\x62\xe9\x74\x68\xa5\x5c\x2c\x9f\x7c\x7c\x56\x87\x26\x61\x60\xc6\xdc\x00\x0b\x98\x16\xc2\x43\xdc\x7d\x64\x1e\xf6\xba\x35\x55\xe0\x68\x42\xc9\x33\x20\xc3\x0a\x46\xa4\x3c\x43\xd7\x5c\x91\x73\xf4\x19\xcf\xfa\x20\xa8\x99\xaa\xa7\x63\xe3\x74\xc0\x12\x69\xa9\x35\x65\x54\x9d\x3c\x32\x0b\xd8\xee\xa8\xf2\x21\xe2\xcc\x80\xf6\x46\x40\xd8\xac\x09\xb0\xa2\x3b\xf4\x5a\xe5\x72\x6f\xa9\xac\x21\xb6\xc0\x2f\x03\x2f\x24\x79\x60\x52\x3e\xd6\xd8\xc7\x77\xf8\xc5\x04\xe1\x5f\x62\x85\x4d\x41\xe3\x65\x92\xbb\x8d\x72\xb3\x45\xae\x0c\x56\xb5\x8b\x06\xe2\x16\x30\x25\x2b\xcf\x67\x42\x8e\xff\x40\xcf\xc8\x19\xfa\x3e\xe1\x43\x79\x92\x9b\xaa\xcc\x43\x49\x94\x3c\x31\x7e\x3f\xf8\xb7\xc9\x56\xfc\xc6\x51\x3f\xe7\xfb\x50\x99\x72\x44\xbf\x18\x9c\x16\xf9\xdd\xf9\x87\x0f\xd3\xf9\xe9\x30\x8d\x9e\x88\xd2\x7f\x81\x4c\x51\x49\x21\x07\x72\x86\xab\x20\xd3\x56\x51\x67\x11\x6e\xad\xd1\x8e\xb4\xb9\x52\x92\x00\xb4\xbf\xbe\xd2\xb3\xda\xbf\x0e\x9d\x8b\xb3\xea\xc2\xa6\x76\xca\x22\xad\x3b\x5e\x05\x4c\xf0\xc3\x68\x2b\xa6\xb6\xb1\x0f\x45\x3e\x4a\xf0\xb8\xa4\xb2\xac\xa1\xa4\xdc\x4c\xa9\xdd\x45\x7a\xee\x10\x44\xa3\x4f\x59\x31\x74\xf0\xbd\xf3\xf2\x82\xb7\xd6\x7a\xb1\xce\x1e\x59\x47\xa2\x17\x62\x4a\x16\x43\xda\x2c\x38\x7d\x52\x2a\x27\x59\xd2\x2c\x98\xa1\xa1\x51\x83\xd8\x6c\x80\x3d\xac\xe2\xe8\x34\x2b\xe7\x16\xb3\x1a\x28\x4e\x24\x39\xd1\x0d\x83\x49\xd5\x45\x87\xa2\x17\x81\x67\x33\x22\x1e\x99\x45\xdf\x05\x8c\x79\xce\x6d\xe4\x4f\x5d\x8a\x40\xd0\x28\x0f\xab\x51\x7a\xb4\x27\xc5\x2c\xd4\x55\xe7\x1b\x92\x56\x97\x51\xb8\x2a\x0f\xd3\x91\x4f\xcb\xa2\x4d\xc3\xf7\x5f\xdf\x6c\xdc\x70\xcc\xab\xb4\xf3\x4e\x29\xf7\x02\x2a\xa2\x4f\x41\x81\x94\x79\xe1\x57\x67\xeb\xcb\xd4\xf7\x82\x98\x03\xe0\xed\xf0\x71\xcc\x89\xf4\x8c\xf8\x28\xb3\xc5\x25\x74\x44\xb4\xf4\xf1\xc8\xf4\x36\xf6\x1d\x0e\x06\x03\xde\x41\xc2\xeb\x4e\x23\xc1\xa5\xb4\xe9\x14\xa6\x9d\xe5\x49\x71\x5b\x94\x9b\x34\x40\xf6\xbd\x9b\xeb\xc1\x62\xe1\x49\xef\x99\x2b\x41\x69\x1f\x56\xe2\x40\xd4\x36\xb5\xb2\xe0\x64\x4e\x8b\x35\x4a\x4e\x7e\xb8\xb8\xea\x65\x75\xd6\x4a\x5d\x2f\xd6\x9c\xf4\xc1\xff\xeb\xab\x4e\x2e\xce\xd8\xab\x3f\x59\x6a\x62\x49\x05\xca\xd5\x8b\x55\x0c\xe2\xde\x06\xd9\xb1\xb4\xf4\x2b\xf9\x43\x71\xcf\xac\xca\x35\xd8\xd1\x32\xd5\x5c\x2b\x11\x08\x8c\xfb\x0e\x5c\x00\xc1\x4b\xbf\x25\x15\x9e\xce\xfc\x3c\x5a\x07\x6d\x6b\xa7\x69\x8e\x5a\xdd\x25\x78\x50\xc8\xfd\x08\x9b\x20\xa1\xf2\xe0\x16\x96\x62\x3d\x8f\x57\xdf\x22\xf9\xef\x22\x36\xfd\x70\x89\xe9\xc9\x3c\x0f\x86\x94\x56\x76\x73\x55\xe2\x6b\xec\xfe\x43\x92\x55\x2d\xa8\x5d\xd0\x6d\x33\x4f\x33\x74\x33\x41\xb0\xb4\xee\x6f\x48\xd0\x2c\x25\x6f\xad\x61\x1e\xce\xc6\x6c\x52\xbc\x4f\xb3\x3a\x21\xde\x55\x63\x4b\xdf\x45\xee\x20\x52\x21\xc8\x33\x11\xb0\x77\x6c\x28\x15\x2b\x1e\x55\x9c\x08\x82\xe3\xb9\x47\x91\x2c\x8e\xc3\xf4\x0c\xe6\x31\x49\xa7\x5a\x81\x07\xd5\x84\xf1\x53\x3e\x73\x3a\x4b\xe1\x2d\x28\xf2\x42\x47\xfa\xc6\xf2\xa2\x40\xf4\x17\xec\x94\x7c\xa1\x52\x69\xb9\xa2\x22\x04\xd6\x35\x02\x12\x0f\x94\x7e\x9b\x10\x7b\xc3\x3d\xbe\xeb\x7c\x7f\x73\xd7\xef\x5e\x3e\xbe\xcb\x53\x2e\x5c\x4e\x61\x06\x5a\xe6\x6a\x50\x70\xf6\xc8\xb2\x38\xe5\x0c\xa3\x1b\xd6\x12\xe1\x38\xce\xe3\xa3\xad\x12\x69\x64\xb6\xa5\x1c\xd9\x3b\x15\x2b\x23\x94\x97\x34\xf3\x00\x89\x65\x6d\x3d\x59\x4b\x5c\x67\x85\x93\x63\xd2\xe3\x96\xe4\x31\xed\xe8\xb2\xf1\xe1\x85\x95\xd1\xb5\x89\x72\xf8\x97\x8c\xbc\x38\x5d\x09\x6e\xe7\x0f\xd8\x5c\xc2\xeb\x71\x3b\xb7\x20\x1b\x2c\xea\x47\xfa\x85\xc4\x77\x35\x52\xd5\x4e\xd2\x94\x1a\x05\x58\x56\xae\x42\xca\xe8\x3a\x1a\x7f\x36\x95\x07\xfd\x5d\x73\xb6\x74\x93\xa3\x06\xe6\x08\xc0\x00\xff\xab\x10\x46\x11\x11\x0a\x53\x86\x46\x70\xb0\x59\x34\x47\x80\xc2\x42\xc0\x87\xfd\x17\x34\xa5\x0c\xe0\x20\x96\x91\xf6\xa1\x38\x8f\x35\x84\xd6\xcf\xbd\xeb\x87\x7e\x41\x54\xfd\xe1\xe6\xa1\x50\x6a\xf3\xb2\xf3\xeb\x52\x59\xb5\xd4\xc2\xb2\x60\x21\x6f\x8a\x79\x6a\xa9\x05\x42\xce\x28\x53\x39\xd1\x64\xae\xc8\xc3\xdd\xd5\x56\xf2\x5d\xb5\xb3\xac\x16\xc6\xde\x97\xae\xaa\x61\x2e\x9a\x7c\x1a\x93\x68\x15\xd0\x6e\xf3\x7d\x64\xa2\xa0\x34\x1d\xac\x35\xd1\x82\xf0\x61\x89\x66\x58\x58\x3f\x54\x6c\x02\xa0\x8a\xc5\xeb\x8c\xe6\xb5\x0c\x16\xe4\x13\x51\x3f\xeb\xab\x8f\xb3\xdd\x20\x7d\x81\x28\x0b\xfe\x51\x32\x78\x36\x0d\xaf\x71\xd2\xec\x50\x96\xe4\x2f\x39\x61\x19\x7a\x40\xb6\x07\x1f\x4c\xe3\x0c\xc1\xae\xe9\xe8\xe6\x80\x22\x2e\x4c\x53\xab\xa4\x9c\xe9\x1d\x69\x10\x7f\x1d\x4c\xb0\xd7\x1c\x1f\x99\x8f\x1b\x82\x26\x7a\xc9\x02\xba\xad\x9c\x94\xa8\x73\xdb\xab\xa0\xf5\x55\xd9\x85\xf4\xb6\x2a\x2e\x25\x99\x37\x6b\xd7\xc8\x57\x5e\xce\x69\x2b\xa0\xae\xec\x4c\xb7\xc3\xb6\x32\x4e\xff\xdb\x62\x24\x41\x1b\x00\xa5\xab\x54\x86\x42\x2e\xf9\x0a\xec\xe8\xf5\xd2\x2b\x73\x32\xac\x89\x64\xe5\x0f\xc8\x66\xd7\xf8\xe8\x4d\x8b\xa1\xdb\x27\x3e\x9a\x13\x37\x35\x9d\x6d\x6c\xc1\xce\x10\xae\xf2\xd9\x34\x81\xb8\xfa\xd9\xec\xe8\x0c\x01\x05\x30\x5d\x5c\xcd\x50\x17\x72\x6d\x01\x09\xfc\xe9\xfa\xbb\x6d\x3d\x54\xac\x7c\x7c\xce\xfc\x6d\xe1\xd2\xf1\x0c\x5b\xbb\x03\x28\x51\xae\x98\x47\x55\xed\xc7\xb3\x47\xe6\x05\xac\x48\xa3\xf6\xe8\x33\xe2\xea\xe7\x40\x51\x66\x06\xd8\xeb\x90\xfb\x94\x09\x3f\x85\x15\x28\xe3\x1e\xa8\x49\xb1\x02\xce\x42\x3f\xf6\x74\xca\x09\x76\xd9\xa5\xce\x82\x62\xe3\x00\x7d\xfb\x12\xb4\xe7\xd5\xbc\xb0\x1d\x83\x39\x1a\x8c\x16\xd8\xab\xa8\xe8\x21\x12\xc4\x9c\x48\xf6\x5e\x65\xf9\xbb\x34\x99\xbb\x90\xea\x92\x7b\x40\x4b\x75\x98\xda\x96\x97\x1f\xf0\x1d\x40\x6e\xad\xab\x38\x78\xc7\x6a\xa5\x99\xca\xf9\x78\x61\x27\xf8\xb1\x48\xd0\x69\x9d\x55\xfd\xcb\x8c\x44\x9b\xe0\x02\xdd\x62\x81\xa7\x44\x11\xb1\x2c\x1c\xa9\x58\xef\x1c\x44\x1c\xb7\x82\xb6\x5f\xb3\x8a\xa6\x18\x4c\xb9\x6a\x50\xa6\xdd\x5e\xad\xc2\xf9\xc9\x66\xb1\x16\xa4\x99\x9e\xc6\xcf\xd6\xf2\xbf\xe6\x2c\x6c\x3f\xf9\x34\x6c\xb4\x95\x07\xeb\xb4\xed\x9c\x0e\x83\x6f\xd3\x5f\x40\x8a\x29\x84\x0b\xb5\x04\xd8\x66\xf5\x28\xeb\x10\x6d\x56\xf1\xd2\x9d\xf0\x6e\x97\xe1\xe0\x32\x93\x4b\x87\xaa\x90\x3b\x01\xbb\x04\x54\x2a\x03\xee\x52\x8d\x4a\x03\x42\x4b\x55\x84\xa4\xe7\xf6\xb3\x98\x85\xb9\x41\xd7\x4a\x56\xe5\xfa\x67\x25\x72\xad\xe0\x71\xbb\x42\xec\x08\x12\xcd\xae\x25\x9a\x55\x5b\xb9\x10\x5d\xab\x77\x27\x11\x25\xf0\x20\x5b\x97\xdc\xa2\x3e\x14\x27\x08\x29\x5d\xf6\x8a\xb4\xc5\x8d\xe1\xea\xa7\x2c\xfb\x57\x91\x83\xbb\x4d\xed\x6f\xd5\xaa\x5c\xd5\x33\xcf\x05\x05\x1e\xa8\xc4\x97\x06\x6c\x5c\x0d\x8c\xd6\x84\x41\x1a\x2b\x7f\xef\xda\x38\xb0\x20\x67\x7c\xce\x53\xf4\x42\xe5\x04\x29\xfe\xc8\x20\x4e\x30\xf3\x06\x28\x8e\xcc\x8b\x27\xf0\x16\x60\x5b\xc8\x74\x38\xa5\x0a\x61\x6f\x86\x05\x93\xe4\x89\x3d\xcf\xfa\x03\x98\x71\x25\x7c\x41\x15\xee\xd2\x8a\x43\xb3\x81\x7d\x2d\x6f\x64\x5b\x84\x02\x2f\xa6\x79\xbf\x18\x05\x9e\xc6\xe3\x6b\x98\x95\x67\x2e\x80\x14\xa0\x6a\x6b\x83\x45\x82\x05\xb8\x5e\x2a\x55\xe9\x6e\xb1\x86\x9e\x15\x00\x05\xf9\x42\x34\x42\x28\xc8\x5f\xdf\x05\x44\x41\x5d\x25\xbd\x65\x29\xab\xee\x93\x1a\xfb\xb7\x4b\x85\x56\xdc\x05\xce\xfb\x92\xd2\x6d\xad\xa4\xd4\x36\xa8\xba\x3c\x21\x60\xf3\xf0\xf2\xba\xe8\x65\x38\xe3\x11\x67\x31\x5d\x23\x5e\x18\xaa\xa5\x0d\xd3\x51\x87\xcd\x57\x23\x1f\x4d\xfd\x40\x7d\x6b\x2f\xf1\x24\x91\x6a\xcc\xcd\x95\x2a\x6b\xde\xbe\xbf\xd3\xbd\x94\xd0\x22\x18\x11\x29\xdf\x4e\x8c\x2b\xc8\xfb\x89\x54\x32\xaf\xc8\x45\x7d\x64\xd5\x52\xd2\x72\xbe\xbd\x6d\x1a\xc9\x4e\x61\xf7\x3c\x1e\xe1\x66\x61\xad\x6e\xbf\x64\x81\x78\x46\xa1\x27\x16\x64\xa3\x24\x06\xe7\x6e\xc8\xba\x00\x2a\x2d\x1c\x6d\x92\x6b\x5e\xc1\x39\xaa\x87\xbe\x90\xe4\xb1\xf2\xec\x5a\xc1\x60\x87\xea\xe7\xc2\x0d\xd2\x38\x27\x26\x93\xe3\xed\x8d\x61\x83\xba\xe3\xcc\xd6\x50\x72\x27\x6f\x52\xac\x19\xe0\x6c\x77\x06\xc2\x5b\x46\xa6\xd0\x8d\x9f\x80\x0b\xda\x8e\x1d\x9b\x70\x9c\x0c\x1a\xbe\xb4\x26\x85\x19\x9b\x90\xca\xbd\xcc\x7a\xdd\x0a\xdb\x9e\x4f\x54\xd8\x98\x64\xea\x5b\x37\xa0\xb4\xb6\x0d\xe5\x2c\xdd\x16\x99\x00\x9a\xb2\x98\x08\x46\xb0\x9a\x1c\x2e\x13\xe4\x62\x5b\x13\xba\x37\xbe\xfd\x66\x85\xd8\x91\xe2\x62\x72\xc8\x36\xc3\x4d\xd5\x64\xcd\x24\x8b\x35\x32\x16\xf2\x62\xdb\x0b\xea\x6d\x85\x69\xd3\xc3\x1f\x5a\x67\x97\x6e\x95\x2c\x52\xad\x72\xee\x27\x6d\xa6\xc2\x36\xb5\x90\x30\xa3\x4f\xbb\x5f\xa2\x7c\x05\x49\xde\x44\x7e\xca\xfe\x53\x26\x96\x15\x43\x4f\xbd\x2c\x0a\xa8\x48\xaf\x30\x65\x96\x7b\x2d\x4b\x9c\xd0\x72\xef\x14\x57\xe5\x4a\xb4\x3e\x0b\xe7\xcd\x27\xe1\x84\x94\x8c\x90\x92\x51\xb1\x46\x21\x25\x03\xa1\xb6\xa5\x64\xac\x52\x41\x97\x19\x69\x33\xbf\x21\x14\xad\x2d\xd4\x56\x32\xeb\xbb\x42\x8f\xdc\x3c\xed\xc0\xd9\x39\xfd\x98\x2d\xfb\x8b\xfd\xa1\x32\x6c\x6b\xe1\xb3\xf2\x6c\x7d\x9b\x2b\x9b\x97\x5d\x17\x58\xc4\x89\x85\x20\xb4\x41\xd5\x45\x1b\xd9\x32\x73\xee\x23\xfb\x81\xbf\x90\x67\x22\x4e\x10\x56\x68\xca\x01\xd7\x2a\x8f\xe1\x81\x83\x50\xc0\xd2\x37\xb1\x1a\x18\x5d\xe3\x29\x89\x4d\xe1\x50\x2f\xf4\xd2\x1a\x95\xad\x3b\xb8\x0a\x69\x17\x40\x63\xcd\x32\xb8\xd8\x8e\x47\x66\xc2\x21\x4d\x08\x1e\xc8\x0a\xd4\x4d\x0c\x36\xcc\x1f\x33\x67\xf5\x1f\xcf\x50\x5f\xdf\x4f\x54\x16\xc7\xeb\x01\xef\xd5\x8d\xed\x91\x8d\x05\x4f\x67\x99\x9d\x8f\x0f\x4d\x05\x69\x13\xa1\xb5\xe8\xac\x86\xc1\x38\x4f\x75\x84\x63\xad\x8b\x2f\xdf\x38\xaf\x12\x29\xbb\x11\xcc\x92\xbf\x81\xf4\x31\xcc\xc2\xff\x6c\x38\xbe\xf1\x31\x7b\xe0\x32\xcb\x2a\x00\xec\xc9\x01\x7e\x49\x24\x58\x85\x32\xcf\x40\x21\xd7\xbd\x88\xa7\x50\x39\xce\x65\x76\xdb\xcc\xb7\xe2\xfc\x0f\xd5\x50\x0d\x79\xe7\x36\x2e\xcd\x24\xd2\xda\x7b\x62\x6f\x16\xdd\xc6\x11\xbe\x75\xfc\xe2\x36\x15\x33\x0e\x92\x58\x32\x77\xd0\x12\x16\xe4\x6f\xc6\x67\xa9\x89\xbd\xa3\x7e\x28\x56\xe5\xce\xa6\x52\x7d\xc6\x2a\x9a\x68\xce\x9d\xa3\xb2\xed\x28\x26\x31\xe7\xca\xfb\xb5\xf2\x56\xcc\xe0\xc2\xef\xbd\xc6\xed\xb1\x6c\xf7\x78\x31\x86\x59\x20\x67\x26\x49\x4c\x75\x7f\xc6\x35\x68\xeb\xc2\x7b\x76\x51\xf7\x89\x7d\xa2\x27\xba\x6a\x17\xad\x1a\x7f\xb3\xbd\x55\x2c\xf5\xb6\xf3\x68\xc7\x2d\x60\x6e\x2e\x2d\xa8\x58\xfe\xa2\x2d\x74\x5c\x13\xa2\x20\xe8\x66\x99\x4a\xb6\x3c\xc3\xb3\x16\x47\x32\x8b\xeb\x14\xcf\xb4\x12\xa1\xb8\xbe\x25\xc5\xd8\xc8\xb1\x26\x96\x17\x61\x94\x0a\xea\xce\x7e\x29\x6f\xbd\x7e\x77\x80\x85\xf2\x83\x5f\xca\x2b\xc2\x5e\x95\x43\x13\x94\x80\x23\x95\xe2\x2c\x78\x12\xf6\x44\x42\xd9\x93\xee\xcc\xe4\xe8\x3b\xe7\xbf\x70\xe2\x5d\xc5\x9a\xae\xdc\xd8\x5b\xac\x32\xae\xc2\x60\x6c\x74\xd2\x28\x1b\x7b\x00\x8e\xd5\x56\xe2\x26\x45\x37\x2a\xbf\x6c\x56\x38\xa4\xf2\xd3\xba\x32\xc7\x4d\xbe\x5d\x02\x30\xd5\x28\x64\xbd\x8d\x15\x13\xbc\x4c\x00\x1b\x2a\x6c\x65\x37\x1f\xd8\xd3\x76\x04\xb0\xc7\x14\x42\x19\xb0\x93\xe5\xfe\xe0\x97\x4d\xd0\x43\xfb\xe6\xbf\xf3\x87\xa0\xbf\xdb\xe2\x2c\x15\x2f\x3e\x32\x2e\xec\xab\x27\xd9\x7b\xfa\xb5\x1c\x9f\x58\x4b\x89\x8b\x5f\xe6\xe8\xa3\xa2\x88\x53\x08\x68\x2d\x16\x67\xce\xc0\x53\x67\x65\x2d\xf4\xe0\x9f\xd2\x21\x11\x8c\xe8\x39\x39\x5c\x87\x8c\x07\x4f\x31\xc3\x63\x00\xc3\x3e\x81\xa0\x43\x90\xb2\x73\x0d\xca\x9c\x44\x53\x1f\x14\x98\xac\xe6\xf1\x36\x95\x39\xaf\xfa\x0d\x7d\x1a\x09\xdc\x62\xf1\xe6\x91\x2b\xd5\x87\xf6\xce\xf6\xbf\x99\xa2\xd1\xef\xdc\xff\x38\xb8\xeb\xde\xdf\x3c\xdc\x5d\x14\xb4\x8d\x8b\xab\x87\xfb\x7e\xf7\xae\xf2\x59\x9e\x06\xfc\xd3\x43\xf7\xa1\xe6\x91\x6b\xe0\xaa\xf3\x7d\xb7\x50\x42\xff\xa7\x87\xce\x55\xaf\xff\xeb\xe0\xe6\xe3\xe0\xbe\x7b\xf7\x73\xef\xa2\x3b\xb8\xbf\xed\x5e\xf4\x3e\xf6\x2e\x3a\xfa\x4b\xff\xdd\xdb\xab\x87\x4f\xbd\xeb\x81\x8b\xe8\xf6\x1f\xfd\x72\x73\xf7\xe3\xc7\xab\x9b\x5f\x06\x5e\x97\x37\xd7\x1f\x7b\x9f\xaa\x66\xd1\xb9\xbf\xef\x7d\xba\xfe\xdc\xbd\x5e\x5e\xaa\xbf\x9a\x1a\xb5\x75\xb3\xbd\xfb\xd7\xb3\x75\x79\xd2\xdd\x70\x6e\xcf\x04\xfd\x27\xb8\x5c\x6e\xcd\x16\x3d\x3d\x71\x7f\x99\xc2\xfa\xa7\x9a\x73\x3b\x77\x5e\xce\xf4\x1e\x59\xe6\x13\xce\x64\x01\x85\xc7\xd2\x65\x75\x17\x46\x7b\x8e\x3a\x70\xc8\x40\xcf\x29\x74\x0a\x49\x23\xd9\x48\x5d\x14\x01\xec\xc3\x84\x4e\x29\x04\x14\xa0\x53\x54\x5e\xf0\x62\x83\x76\x4e\x30\x04\xeb\x6e\x8c\x97\x9d\x06\x59\x4e\x18\x87\x9d\x72\x8e\xdc\xc5\x42\x8c\x15\xc4\xc0\xfa\x9a\xc2\xf4\xe5\xec\x16\x40\xb6\x45\x39\x8a\x4b\xb9\xc5\xc2\x06\x2b\xb6\x3c\x21\xe8\xc7\xbf\xe5\x83\x02\xc7\x8b\x35\x18\xa4\x0b\x15\x28\xed\x03\x91\x1a\xaa\xae\xda\x9e\x85\x9e\xdc\x31\xb7\x16\x71\x38\xb7\xb6\x6e\x3f\x78\xc9\x52\xe6\x21\xb9\x15\x5c\x66\xfa\x78\x9b\x19\x95\xf6\xf8\x39\xba\x07\x14\x19\x99\x5b\x1c\xf4\x2a\xce\x92\x74\x4c\x19\xa2\xd3\x59\x02\x3c\xc6\x98\x21\x86\x64\x82\x9f\x29\x77\x05\x57\x4c\x5d\x1a\xa0\xa3\x95\x08\xd1\x29\xaa\x3d\x28\xe7\xa8\x13\xc7\xb2\xc8\xe0\x0a\x3b\xc7\x71\xd1\xd3\xe2\xb0\x7d\xf0\x35\xcd\x58\x2d\xdb\x2c\xed\xa3\xfc\xc8\x01\xc5\x76\x8f\x93\xb3\xc8\x0e\x8b\x22\xc3\x16\x52\x8b\xa6\xe0\xc0\x6d\xe5\xc1\x46\x32\x4c\x1f\xcb\x27\xc7\x9a\x57\xc9\x31\x0e\xb1\x68\xbb\x1e\x2d\x74\x51\xd3\x4e\x33\xca\x0e\xe0\xa0\x6d\xd6\x67\x2d\xe0\xf6\x8a\x2e\xdd\x8c\x93\x52\xa9\xbb\xc6\xfd\x15\x4a\xe5\x55\x76\xb6\x53\x27\x55\xb5\x10\x09\x47\x72\x90\xed\xff\x35\xe6\x71\x0b\x9f\xde\x64\x5f\x2e\x95\x34\x07\x1e\xdd\xd6\x75\x5d\x2d\xe4\x3f\x5b\xf7\xd5\xd2\x7d\xb8\x23\xe4\xac\xe6\x52\x24\x94\x0a\xa1\x11\x78\x29\x31\x65\xb6\x80\x14\xc9\xdc\x68\xae\x60\xbb\x3e\xc7\x59\x49\x45\x3c\xe4\xcf\x05\x9d\x78\x4a\xa4\xc4\x35\x58\x30\x9e\x25\x6f\x1b\xc6\x90\x9d\x50\xfb\x61\xc3\xfd\xe4\xce\x64\x5f\x7f\xb5\x4c\x46\xbf\xf3\x15\x7a\x37\x51\x2d\xc3\xc6\x2e\x88\x19\xdd\x98\x54\x46\xcd\x5f\x4e\xf2\x18\x20\x2e\xbc\xd0\xa8\x3a\xaf\x55\x43\x6b\x60\x99\x60\x95\x75\xc1\x7c\xcf\xe3\xfa\xa1\x43\x5e\xeb\x1b\x83\x8d\x5b\x77\x10\x2e\xd2\x67\x8d\x5d\x57\x70\xd3\xfa\x15\xdb\x23\x3e\x9d\x1a\xb9\xa0\x60\x02\x3e\x41\xd8\x64\x90\xe6\xd2\x94\x4c\xa3\x89\x71\x8e\xe9\x2b\xe3\xe4\x91\xbd\x78\x0b\x52\x88\xb1\xee\xf8\x2d\x01\x50\xeb\x17\x7d\xdc\xe8\x73\x21\x72\x1d\x44\x46\x0a\x61\xd4\xde\x46\x30\x7e\xcc\xbc\xe0\xd9\x8a\x0d\xee\xad\xd7\x16\x5b\x7d\x83\xda\x9a\x25\xfa\xd6\x55\xd8\xcc\xe6\xe6\x15\xb6\xdc\x42\xc1\x6f\x3a\x04\xaf\xb6\x66\xd5\x08\x76\x50\x5a\xf3\xa0\xc8\xe9\x59\x26\xac\x49\x9c\x9e\x0e\x2d\xfc\x87\x9e\xae\xa3\xf6\x9f\xdc\x8c\xfe\x64\x14\xe1\xb4\x06\x2f\xc6\x6b\x2d\x03\x4f\x47\xa7\x5a\x66\x75\x38\x06\x36\x7e\x44\xa2\x53\x03\xc8\xf8\x1e\x82\x58\x3b\xb7\xbd\xf7\x27\xe8\xbd\x9f\xc8\xf7\xfe\x68\x4c\x17\xf9\xf1\xb7\x54\xb3\xc5\x3d\x41\x97\x2b\xe4\x92\x14\x0f\x3d\xec\x94\x12\x1f\xb0\x3b\xc6\xb2\x01\x54\xc7\x05\xf4\x97\x85\x6f\xc0\xa3\x0f\xe5\x22\x8d\xd3\x3b\x8b\x64\xb7\x7e\x33\x23\x61\x53\x59\xb1\x72\xf1\x23\x1b\xce\xcb\x9e\xb1\x93\xcc\x35\xd6\x98\x47\x6c\x5d\x02\x51\xb7\xb7\x98\x77\xbe\xe3\x08\xeb\xe5\xb7\xd1\x8a\x4c\xf6\x4e\x56\xa7\x27\xe7\xa1\x75\xa1\x1d\x21\x35\xa1\x6a\x56\x05\x33\x9f\x23\x66\xe5\xa2\xac\x92\xbe\x8e\x6d\xbb\x35\x88\xe7\xef\x54\x51\xc4\xa6\x72\xd4\x88\xf6\x61\x97\xed\x77\x97\xed\x22\x95\xa5\x38\xb8\xf5\xaf\xef\x0b\x23\x45\x7a\xcd\x38\x73\xaf\x56\x65\x32\x06\x5f\xa8\x93\xb9\xba\xbc\xf5\x9a\x8e\x72\x8f\x26\xab\x3d\xe5\xf7\x26\xda\xc2\xf8\xaa\x17\xc7\x5a\x1e\x6a\x47\xd9\xea\x52\x9c\x9a\xb4\x55\x45\xa7\xe4\xc4\x94\x33\xcb\x23\x44\xec\x79\x85\xed\x66\x02\xbb\x26\x84\x0a\xd7\x89\x05\x8f\x5c\x0b\xe7\x60\x4d\x5d\xa0\x6e\x8f\x6c\x11\x9e\x73\xdd\xf9\xdc\xbd\x1c\x74\xaf\xfb\xbd\xfe\xaf\x15\xc0\xa0\xc5\xc7\x0e\x1b\xd4\x7b\xe1\xfe\xd7\xfb\x7e\xf7\xf3\xe0\x53\xf7\xba\x7b\xd7\xe9\xaf\xc0\x0d\x5d\xd6\x59\x1d\x26\x65\x2a\xab\x94\xc7\x75\x70\x29\x9d\x91\xb9\xa2\xf7\x45\xf4\x50\xaf\x13\x4a\x6a\x10\x44\x0d\xa6\x03\x8b\x89\x40\x31\x79\x26\x09\x9f\xe5\x46\xdd\x4a\x82\x79\xd0\xa2\x15\xed\x2f\x83\x17\x85\x36\xcb\x34\x3e\x47\xa6\x36\xa2\x57\x1e\x3a\x6b\x10\x44\x3e\x2c\x08\x7b\xaf\x10\xf9\x32\x4b\x68\x44\x95\x97\xf3\xc9\x85\x75\xee\x18\x9f\x2b\x84\xf4\xae\xd8\x5c\x3b\x0b\xe1\xd9\xb9\xc5\xc1\x0f\x3f\x58\xb4\x35\x64\x27\x2a\x83\xba\x5b\x59\x19\x6a\x07\x66\x85\x1a\x4f\xfb\x02\x12\xdf\x06\xa3\xdb\x87\x71\x62\x31\xb1\xc9\xe6\x6d\xd6\xa0\xf4\x55\x0f\x72\xf5\x6d\xb8\x2c\xb8\xa8\x70\xae\x97\x47\x17\x35\xdb\xa9\xaf\x1c\x23\x54\x28\x44\xbb\x03\x48\x15\x1b\xf0\xbf\x66\x94\xc7\x42\x21\x20\x66\x02\x75\x31\x12\x64\xca\x95\x56\xc0\x4c\x18\xc5\x89\x16\xaa\x28\x4e\xe8\x3f\x01\x7c\x4c\x90\x33\x2f\xec\xc4\x41\xb6\xe5\xce\x0b\x0b\x0c\x72\xf6\xc8\x2e\xbb\xb7\x77\xdd\x0b\xcd\x90\xce\xd0\x83\x04\x5c\xb1\xc2\xd4\x2f\xed\xf6\x36\xe2\x98\x1f\xfe\x41\x99\x54\x04\xd7\x45\xd0\x11\x21\xb8\x68\xce\x1f\xb2\xfe\xba\xf0\x5d\xf5\xf6\x86\x67\x05\xcb\x98\x33\x3f\x5c\xd7\x56\x11\xf7\x12\x2d\x76\x9e\xc8\x76\x87\x5f\x0a\x14\xf1\x71\x55\x40\x12\x29\x52\x7d\x8f\xd4\x06\x64\xd6\xe6\xf3\x2b\xf4\x79\x0b\xdf\x2e\x9b\x67\x1f\xe2\x12\xa5\xca\x61\x5e\x0d\x12\x6c\x56\xce\xa8\x34\xcf\x5a\x51\x51\xbc\x06\x06\x4b\x69\xeb\x0f\xc9\x18\x33\x24\x52\xc6\x4a\xb8\xbf\xbe\x9d\x6f\x31\xd2\x68\xdd\xa3\xaa\x69\x86\xa7\x3c\x65\xa6\x1e\xaf\x1e\x55\xc5\x60\xe4\x8c\x30\xb5\x62\x30\xaf\x85\xb0\x53\x1a\x6a\x7b\x41\x76\x2a\x06\x5a\x87\xb3\x53\xe5\xcd\x82\x52\xe5\xeb\x5d\xcb\x2e\x92\xb1\xe0\xd2\xd2\x87\x2a\xbb\x9f\xab\xb5\x6c\x2c\x9f\xb6\xee\xae\x8f\xe5\xd3\xea\xae\x62\x12\x3d\xad\x7b\xd9\x94\xd3\x59\x13\x5b\xe9\x7d\xc1\xd8\x37\xd7\x4f\x6d\xcd\x1d\x28\xf0\x1f\x3d\xa1\x1f\xfa\x9f\xaf\xd0\x88\x6a\xb9\x57\x5f\x2b\xd7\x58\xcb\xd8\x0f\x22\x71\x56\x69\x6b\xd9\x4d\x45\x92\xdd\xbd\xb0\xf0\x4e\x94\xf2\xa4\x04\x7d\xa3\xe1\x31\x71\xa6\x66\x61\x61\x14\x4b\x35\x77\x04\x66\x31\x9f\x9a\x79\x7c\x90\xe9\x68\x44\xbf\x9c\x29\x2c\xbe\xa9\xa1\x87\x89\xe9\x18\xfc\x83\x0f\x07\x7a\x44\x5b\x5e\xc4\x55\xcd\x21\x5b\x80\x3c\x23\x9b\x9d\xd9\xa5\x79\xf7\xff\xf0\x21\x40\x04\x00\xca\x81\xf3\x0d\xda\x38\x09\xfb\x8a\xdb\x49\x79\x45\xee\x02\x7a\x4d\xc4\x85\x20\x16\x59\xc0\x14\x8d\x9d\x61\xa1\x28\x58\x6b\x1d\xfa\x4d\xa1\xec\x41\xbe\x44\x7e\x89\xfc\x09\xce\x21\xc6\x87\x84\x80\x7b\x69\x46\x93\xf5\x94\xde\x8b\x82\x67\xb4\x74\x02\x6d\xb8\xae\x05\x44\x05\x83\xcc\x4a\x11\xab\xfb\x4c\x98\xda\x89\x7e\x02\x4d\x54\x60\x1d\x34\xf3\x71\x98\xda\xad\xbd\xcb\xfc\x72\x73\x71\xd0\x7e\x4c\x95\x12\x18\xee\x79\x9b\x5d\x66\x1d\xfa\x75\x61\x06\xcf\x8d\x3d\xd7\xf0\xea\x22\x5d\x56\xe4\x13\x58\x6a\xe7\xa5\xf1\xf3\x58\x60\x57\xb3\x61\x43\x84\x26\x49\x8c\x15\xc3\x43\x06\xb1\xca\x69\x79\xcd\x4d\x9f\x7a\x6f\x95\xba\x5c\xb9\xe4\x1b\xc0\x11\x15\x9a\xf9\x44\x20\x0f\x76\x17\xd1\xfb\xeb\x00\x1e\xc0\x40\x1e\x44\x02\x71\xe7\x4b\xad\x58\xa6\x7e\xbc\xe6\x7c\x99\x64\x87\x1b\xc8\xe8\x66\x30\x5a\x68\x24\x33\x41\x22\x7d\x95\x9d\xa3\xdb\x84\x68\xc9\x2b\xd5\xd2\x57\x9a\x24\x0e\xba\x6d\xb9\x74\xb8\x16\xdc\xe0\xde\xe7\xe5\xe9\x1e\x4b\x26\xe6\xa0\x0b\x97\xcf\xcc\xa3\xc1\xee\x61\x2a\x3c\xfa\x82\x09\x19\x0c\x89\x45\x2d\x12\x38\xfc\xdc\x44\xed\x82\x29\x09\x17\x2e\x32\xfa\x4f\xcd\x7e\x05\x91\x13\x5e\x9b\x19\xea\xcf\x76\x3f\x73\x70\xa4\xdc\xe3\x24\xdc\x7d\x58\x17\x8c\xde\x40\xae\x29\xdd\x81\x05\x11\xa7\x89\x2f\x36\x8f\x3d\xb1\x40\xba\xf6\x6e\xb5\x43\x83\x5b\x32\x37\xb5\xf9\xa0\x76\xb9\xeb\x22\x57\x66\xe6\xc6\xf7\x9a\x7d\x9e\x1b\x90\xf3\x3c\x0a\xaa\x64\x5e\x4e\x10\xe9\xbb\xb6\x6e\x89\xf5\x3c\x07\xa9\x58\x0b\xc7\x23\x87\xa3\x5f\x87\x73\xdb\x0c\x9e\x7c\x58\x9a\x08\xd5\xec\xd2\x56\x09\x01\x31\xda\x06\x3a\xc9\x02\xc4\x9f\xdd\x36\x86\x8c\x95\x2a\x5e\x3d\x53\xde\xd6\xad\x06\x52\x72\x2e\xca\xec\xcb\xbb\x56\xd8\x81\x85\x09\x04\xd0\xb8\xf5\x41\xe3\x6c\xc9\x98\x6c\xef\x01\xc4\xa3\x12\x80\x96\x90\x3b\xd0\xca\x82\x83\x35\x7a\xaf\x4a\x17\x2b\xac\x4e\xa3\xdc\xb0\xc2\x17\x9a\x97\x5c\x6e\xe9\x81\xd3\x93\x99\x0f\x20\xdb\x76\x9b\x18\xa0\xc2\xfc\x8d\xf7\x00\xda\x24\x31\x32\x90\x0f\x06\xd2\xda\xd2\x2e\xf3\x9c\xcc\xb0\x20\x4c\x3d\xb2\x3b\x3d\x0a\xf3\x45\x1e\x89\xe1\xa2\x80\x5c\x99\x01\x28\x46\x3c\x42\xd8\x7e\x05\x44\xaf\x0b\xc3\x93\x03\xf3\x12\xa8\xa6\x7b\x44\x26\xf8\xde\xbc\x63\x80\x22\x2c\x50\x92\x9e\x2a\x1d\xe5\x6a\xbc\x16\x20\xa3\x09\x05\x9c\x86\x98\x48\x7b\x21\x51\x65\x81\x38\x32\xf1\x3b\x25\x0e\x58\x1b\x3e\xcb\xf8\x57\x15\xc3\x76\x86\x02\xe6\x0c\x74\xf2\x91\x79\x7d\x2c\xc1\x61\x35\xca\xfa\x86\xaa\x04\xac\x33\x8d\x33\xc7\x17\xfc\xd3\xac\x10\x17\x74\x4c\x99\x57\x0d\xcb\x4e\x6f\x8a\x67\x60\xde\x35\x67\x90\x8f\xb2\x3b\xad\x6f\x73\x1c\xce\x60\xc4\xff\xf7\xef\xff\x73\x46\xeb\xbc\x1f\x72\x60\x29\xd0\x86\x95\x5c\x6f\x59\xfc\x95\xf7\xa0\x57\x6a\x20\x3d\x3c\x9d\x56\x16\xf2\x36\xf2\x5f\xed\xe5\xa6\x37\x0d\x57\x13\xe3\xee\x2d\x6e\x77\xf0\x8d\x88\x74\xc9\xd9\x30\x57\xcc\xeb\xd2\x92\x4a\xc8\x4d\xd0\x23\x31\x27\x39\x33\x10\xf8\x95\xe6\x17\xcc\x34\x8f\x2c\xff\x44\x1a\x10\x19\x83\xdb\x6b\x7e\xc8\xa9\xd3\x90\x30\xcb\x78\x7f\x1e\x29\x91\xbb\xc3\xbd\x58\x68\x57\x17\xc5\xc4\xb0\xea\xf6\x4b\x37\x6d\x89\x73\x7b\x00\x96\xdb\xc4\x8c\x4e\xb0\xdc\x5f\x68\x4e\x65\x3d\x2f\x63\x4d\xf7\x85\x87\x55\x41\x3a\x66\x90\x26\x45\x56\x2f\x48\x2a\x89\x30\x9c\x2e\xc3\x10\xb3\x3b\xc1\x87\xe7\x84\x08\xd1\x15\xbe\x46\x32\xc5\x74\xad\x6c\x06\xfd\x7e\x35\x78\x68\xc1\xd9\x80\xc7\x44\x0c\xe2\x54\x2d\x1c\x8b\x65\x19\x06\xfa\xa3\xcb\x54\xcd\x57\xb7\x2f\x13\xbc\x58\xcf\x68\x19\x60\xab\x7e\xbf\xa6\xd9\xd5\x12\xb3\x17\xe2\x53\x94\x9a\x6b\xe0\x50\x49\x09\x0e\xd5\x46\xbc\x16\x4c\x24\x70\x03\x33\x05\x38\x84\xb9\x26\x65\xaf\x68\x03\xda\x0e\x23\x47\xc3\x34\x37\x29\x65\x65\x30\xe2\xb3\x47\xf6\xd1\xd4\x91\x01\x2d\xcf\x0c\x20\x82\x74\x23\xf2\x65\xc6\x25\x29\xe4\xbf\x55\x94\xb6\xb0\x89\xaf\x76\x18\xd5\xc2\x7a\xfe\xd1\xf6\xb2\xfa\xab\x03\xdb\x2e\x2e\xf8\xe2\x94\xab\x77\xe0\x56\xe2\x60\x44\x67\x54\xef\x9d\x41\xe5\x49\xdb\x5f\x79\xe5\x3c\xa6\x0b\xc0\xc3\x54\x32\x3f\x41\xd9\xf4\x4a\x1b\x22\x21\xcf\x04\xcc\xe9\x30\x46\xbf\x80\x49\xd1\xae\x57\xc3\x4e\x56\x1d\xa0\x3c\xf9\x14\xd8\x02\x8a\xcb\x23\x28\xa6\xe8\x55\xed\xc5\x62\xf2\xd1\xd6\x79\x72\x55\x81\x29\x6b\x88\xe7\x1d\xbf\x90\xcb\x9c\x28\x44\xbe\x28\x62\x4b\xbd\xf6\x5d\x26\xe3\x62\xf2\x03\xaa\x4e\xc6\xaa\x97\x1d\xf7\x5e\x74\xbb\xe3\x12\xdf\x5d\xaa\x66\xec\xae\x7c\x9b\xba\x38\xc1\x2c\xb6\xf9\xb8\x56\xc9\xd0\xc2\x16\xcc\xce\x18\xdd\xb2\x4c\x05\x9b\x55\xea\x21\xe0\x9b\x36\x0d\x54\x3f\x5c\x64\x4e\x61\xd4\x2a\x0b\x84\x57\x70\xa1\x25\xf7\x94\x29\x9a\xe8\xcd\x61\xc7\x20\xd1\x08\x22\xe3\x2c\xba\x23\x44\xb6\xd7\x01\x08\x52\x29\x29\x1b\x0f\x2c\x25\x5d\x6a\x69\xb3\x8b\xa1\xb8\xa7\x3e\x9b\xa6\xcc\x8f\xdf\xbb\x86\x96\x1b\xd5\xcd\xb6\x06\x70\x37\x97\xd4\x0a\x1a\x07\xe3\x6e\x32\x16\x95\xcf\xe5\xc2\x0e\x68\x6c\x48\x41\x4d\x45\x71\x98\xe8\x3a\x76\x77\x90\xe9\x16\xc1\x2f\xf2\x2b\x44\xda\x44\x55\x93\x7e\x06\x91\xfa\xaa\x26\x13\x57\xd6\x66\xe0\xf6\x58\x26\xa2\xd9\x7a\x66\x19\xce\x40\x29\x99\x17\xbb\xee\x6c\x3a\x02\x4e\x92\x21\x8e\x9e\x32\x2d\x2c\xb3\x45\x70\xe1\xea\x41\x68\xb9\x12\x0a\xde\x99\xcd\xa5\x07\x1a\x81\x74\xe3\x7b\x0b\x0d\xfc\x91\x1d\x76\xde\xb9\xa1\x9a\xc5\x95\x33\x78\x57\x66\xf4\x26\xb7\x21\x26\xb3\x84\xcf\xa7\x35\xf7\x59\x39\x81\x71\x9b\x48\x9d\xba\xfc\xc9\x9d\x5e\x65\x25\xa6\xb7\xf6\x65\xb6\x90\x0d\xb5\x03\x30\xae\x35\xb8\xe4\xa7\x84\x0f\xc1\xa4\x6a\xcd\x0f\x2e\xc3\xc7\x4b\xf5\x28\x9f\xe7\x75\xf3\x8e\xca\x27\x92\xca\x59\xa2\x95\x99\xfa\x1e\x4c\xce\xc9\x7e\xd7\xcd\x20\x24\xac\xb6\x0e\x36\x8f\xd6\xae\xfc\x7c\x1f\xb0\xcf\x57\x4e\x12\x30\xef\x1a\xfe\x55\xb2\xb2\x99\x54\xc3\x33\xe3\xa4\x56\xfc\x91\x29\x3c\x76\x8b\x6b\x85\x4b\xfe\xc2\x88\x90\x13\x3a\x2b\x14\xc2\xdc\x3a\x3c\xdc\xee\x68\xfb\x3f\x26\x18\xba\xb2\xcd\x16\xa6\x6e\xf5\xf9\xec\xd4\xa0\xb3\xe8\xdd\x29\x67\x38\xca\x6d\xb2\x51\x82\xa5\xa4\xa3\xb9\x07\xaa\x92\xc5\xf9\x42\xea\x5a\xd1\x88\xe1\x55\xbe\xab\x62\x73\x86\x3a\xbb\x41\x15\xd8\x3e\xa3\xf2\xa1\x78\xf8\x69\xec\x83\xee\xe9\xdb\x6c\x11\x7a\xc7\xc9\x09\x96\xea\xb5\xe0\xc1\x06\x3e\x61\x33\x14\x80\xa6\x78\x4d\x7b\xde\x49\x15\x69\x98\x0b\x1b\x29\x47\x0b\xcb\xe4\x68\x4b\x33\xab\xc3\x65\x48\x2b\x3e\x7c\x91\x2a\xe4\xb0\xc2\xce\xd3\x3a\xa3\x33\x89\xeb\x73\x99\xa1\xb4\x00\x98\x45\xfe\xf1\x09\x92\x5b\x81\xb2\x35\xd9\x94\x97\x24\x21\x3b\x09\x36\xdf\x60\x87\x96\x23\x39\xbc\xbd\xb9\x74\x5f\xe6\x65\x29\x56\xdb\x55\x36\x88\x81\xaf\xc1\x48\xaa\x1e\xfa\x2f\x66\xa0\x36\x0c\xbe\x6a\x15\xc1\x26\x0a\x54\x5e\x3d\xda\x36\xed\x72\x2f\xb4\xc4\x0c\xdf\xee\xf7\x7c\x8e\x85\x4d\x9d\xcf\x38\x93\x13\xdb\xb8\xcf\x5f\x39\x54\x7d\x61\x5c\x9f\x48\x93\xb0\x9a\x95\xa7\x6f\x23\xde\xbb\x78\x43\x35\xdb\x17\xd6\x71\xad\x38\x1a\x13\x40\xe2\xa1\x2c\xa6\xcf\x34\x4e\x71\x72\x54\x7b\x62\x67\x89\x36\x3b\xa2\x7e\x35\x87\x69\x64\xe9\xc9\xe3\x41\x89\x92\xee\x3e\x5a\xc0\xfc\xb4\x8b\xd3\xc2\x25\x68\xc7\xb1\x34\x0a\xc3\x9b\x97\xd8\xb6\x86\xc6\xb0\x23\xb3\x00\x11\x41\x94\x2c\x5c\xb2\xf9\xd8\x77\x2f\x4d\x1a\x1a\xc7\xf6\x8b\x0c\x0e\xa2\x00\xc3\x86\x0b\x68\x96\x66\x8d\x5e\x9f\xeb\x96\x8f\xd6\x5b\x97\x3b\xd7\x3f\x63\xe5\x51\xe5\xa7\x2b\x08\xc3\x6d\x38\xa7\xcd\xe5\x61\x07\x40\xdb\x42\xe1\xa7\xee\x18\xb6\xf3\xfe\x6d\x81\x70\xbc\x20\x12\xec\x4e\x44\x3e\xa2\x6d\xd2\x0a\x49\x79\x61\x29\x0e\x25\x2f\x9f\x3a\x6c\xaf\x1c\x29\xab\xbd\x4b\xd4\x8e\x93\x7c\x67\xdd\x8f\xfb\xbb\xe0\x57\xef\x97\x9d\xec\x0f\x80\xb9\xc5\x90\x8f\x9f\xda\x72\x3f\x70\x78\xbd\x18\xce\x05\x9f\xd7\x8a\xe8\x58\x3b\xbc\x46\x71\xb1\x0b\xe4\xdc\xc7\xf2\xda\xe4\xcb\xc6\x8b\xbb\xcf\xad\xb6\xee\x58\x76\xa1\xa3\xed\xd9\x7b\x68\x77\xa3\xf7\x41\x08\x52\x6f\x76\x8b\x56\x40\x3a\xb9\x25\xdb\xe5\x21\xab\xaa\xd1\xb8\x3d\x7c\x84\xcb\x2d\x1d\xcc\x04\x19\xd1\x2f\x1b\xa9\x02\xb7\xf0\xa9\x55\xaf\x35\x99\x4b\x55\x1f\xc1\x2d\x08\x55\x22\xbd\x40\x5a\x4b\x69\x5b\x19\xee\x91\xe5\x19\xb9\x36\x1d\x57\x0b\xc3\x5c\x14\x7e\xda\x14\xfa\x74\xf7\x15\x2a\xcd\xba\x4e\x94\x9a\xc9\xf3\x0f\x1f\xc6\x54\x4d\xd2\xe1\x59\xc4\xa7\x26\xff\x83\x8b\xb1\xf9\xe3\x03\x95\x32\x25\xf2\xc3\x5f\xfe\xfc\xe7\x7c\x89\x87\x38\x7a\x1a\x1b\x38\xa7\x45\x7f\x67\x71\xc9\x09\x96\xdb\x45\x94\xb9\xd4\xc9\x3d\xa7\xd0\x7b\xdd\xb8\xa4\x65\xfd\x8d\x54\x78\x3a\xf3\x43\x90\x4d\x8d\x47\xa9\x70\x5e\x59\x06\xf2\x61\xf5\x34\xd1\x04\xcf\x66\x84\xd5\x9b\x5d\x4c\x82\xf3\x16\xac\xc7\xa5\x48\xdb\x11\x92\x2f\xb3\x04\xb3\x22\xec\x07\x94\x49\x13\x24\x22\x4c\x59\x48\x8a\xbc\x36\x3d\xec\x46\x03\x3d\x65\xf8\xff\x7a\x29\xb0\x30\x47\x2a\xf3\xfa\x87\x6e\x38\xb6\x16\xb1\xab\x50\x8b\x3d\xd2\x95\xeb\x3f\xe7\xb4\x23\x8e\x6a\xcb\x92\x63\xef\x6d\xad\xb7\x6d\x76\x50\x24\x38\x1b\x90\x2f\x9a\xc9\xc9\x4d\x81\xe2\x1e\x24\x91\xa8\xf3\xcb\x3d\x92\x73\xa6\xf0\x97\x73\xf4\x99\x32\x10\x60\x7f\xe0\xa9\x90\xe8\x12\xcf\x4f\xf9\xe8\x74\xca\x99\x9a\xa0\xcf\xf0\xff\xed\x4f\x2f\x84\x3c\xa1\x5f\x09\x16\x96\x3f\xd8\xfa\x91\xae\x84\x1d\x6c\x21\x91\x32\x89\xc8\xb3\x3e\xa1\x7f\xfe\x5f\x68\x6a\x5a\x3e\x47\xdf\x7e\xf8\xf3\xff\x42\x7f\x84\xff\xfb\x7f\xd0\x1f\x6b\x2c\x0d\xeb\x41\xcd\x41\x99\xf1\xbb\xda\x30\x02\xa0\x94\x5c\x24\xf9\xaa\x66\x2f\x04\xcf\x57\xaa\xb2\xe5\x27\x1a\x3d\xf1\xd1\x68\xa0\x37\x86\x49\x20\x1d\xe0\xad\xcc\x0e\x3e\x6a\x30\xb5\x85\xe2\x4d\xd9\xc9\xbc\xe0\x93\xed\xd4\x20\x8d\x38\x76\x2d\xd3\xdc\x3c\x01\xc1\x6b\x85\xd2\xe3\x54\xc2\x57\x24\xd6\x5c\x75\x9d\xd3\xe1\xac\x8b\x0e\x74\xc0\x59\x90\x7c\x64\x1e\x27\x10\x17\x02\x4e\xfd\xe8\x69\x13\x60\x66\x09\x59\x79\x1c\x16\xc2\xba\xdf\x4c\xac\x2e\x4c\xed\xb5\xe2\x74\xe5\x42\xe7\xab\x43\x74\xef\xb9\xd8\x4a\xdf\x7a\x22\xb5\x29\x34\x2b\x8a\x9b\xb9\x82\xdb\xd8\x37\x6a\x28\x8e\x24\x17\x19\x7a\xb7\xb1\x8b\xd8\x12\xa8\xab\xad\xa8\x54\x98\xa0\xc6\x66\x87\x5e\x4f\xfd\x32\xfb\x64\xd5\x30\x21\xc2\xd1\xbd\x9d\x17\x77\x84\xd1\x6a\x11\x49\xb3\xc4\x8a\x11\x57\x80\x6c\xae\x5a\xd0\xfb\x0c\x57\x05\x1a\x87\x70\x5b\xc8\x1b\x62\x4e\xb2\xb5\xc0\x15\xd5\xeb\x99\x8a\x88\x5c\xf0\xed\xc2\xad\x13\xca\x16\xf2\x34\x6a\x83\xdb\xea\x65\xf2\x2b\x5b\x21\xce\xe1\x50\xf3\x38\x57\x16\x8c\x5b\xc2\xd6\x5e\xf1\x00\x70\x8b\xb3\x01\x20\xc5\x5d\x60\xac\x2e\x54\x04\xd9\x82\x6b\x1b\xc3\x75\xce\xf0\x5c\x41\x99\x52\x1d\x19\x81\x35\x2f\x5c\x12\x33\x09\xe1\x64\x5b\x8f\xc3\xab\x8d\x94\xc7\xa8\x15\xaa\x14\xc3\x48\x20\xdf\x72\x43\x8c\x5c\x53\xa6\xec\x04\x09\x0c\xc1\xc0\x6a\xa2\xdb\x93\x44\x9c\x8e\x70\x44\xd9\xf8\xc4\x83\x47\x05\xa8\x12\xff\x3a\xa8\xda\xa4\x7d\x2c\x9f\x76\x1b\xe0\xba\x75\xb5\x59\x1a\xe7\x15\x0f\x2d\xa0\x91\x71\xac\xd0\x05\x6c\x48\x85\xe5\x53\x1d\xa2\xd7\x02\x9c\xe0\x92\xd1\x65\xa4\x70\x20\x84\xcb\xc6\xe7\xa0\x0f\x88\xaf\x4f\x41\xa5\x12\x57\xff\xdc\x82\x8b\xba\x4c\x53\x9c\xa1\xff\x94\x51\x75\x97\x8c\x5f\x4e\xb8\x50\x83\x0d\xf1\x88\xcb\x2e\x15\x46\x4e\x13\x00\x12\xe2\xcf\x44\x3c\x53\xf2\x52\x84\xf5\x5d\x67\x2f\x1a\xa3\x99\x17\x4f\x09\xb8\xaf\xd3\x19\x87\xd4\xad\x11\x9a\x62\x36\x37\x8c\x52\x33\x17\x2c\x9f\x64\x56\x75\x19\xc9\x29\x4e\x92\x13\x24\x48\x2a\x4d\x35\x72\x49\x92\xd1\xa9\x2b\x00\x13\xa3\x84\x8f\x69\x84\x13\x34\x4c\x78\xf4\x24\x4d\x66\x25\x1b\x1b\x26\x35\x13\x3c\x22\x52\x7a\x92\x55\x8e\xa2\x60\x73\x5b\xa1\xe4\xb2\x22\x62\x4a\x19\x95\x8a\x46\x4e\x64\xca\xc1\x50\x4c\xe1\xff\x08\x83\x49\x18\x32\x85\x61\xb8\x5a\xd2\x23\x06\x14\x36\x65\xb6\x54\x18\x5c\xd7\x16\xeb\xd1\x25\x27\xd4\x1d\xa0\x1d\x40\x57\xba\x1d\x32\x50\xc5\x03\xb9\xe2\x48\x5d\xd8\xcf\xe0\x18\x2f\xdb\x02\x77\xc5\x13\x95\x6d\xc8\xec\xa4\x15\xe0\xb4\x20\x97\x21\x4b\xbd\x28\x48\x2e\x59\x46\x42\xcb\x90\xf4\x60\xc8\x35\xf8\x79\xab\xf6\xb4\xa6\x22\x88\x3c\x50\x9d\xae\xec\xb5\xa7\x2c\x4a\xd2\x38\x2b\xab\xaa\x45\x80\x67\xbd\x49\x1c\x79\x34\xed\xb5\xa0\x70\x82\xb0\x44\x2f\x24\x49\xf4\xff\x9a\xcc\x8b\xd3\xac\x5c\x88\x66\xc9\xa6\xa4\x0b\x74\xe2\xb8\x74\xdd\x8e\x6a\x1d\x2a\xea\x2d\x56\x13\x83\x35\x31\xe5\xca\x54\xb4\x35\xa8\xa8\xce\xbe\x65\x60\x34\x87\x09\x1f\xc2\x49\x07\xc0\x54\x97\x5f\xed\xa5\x73\xa6\x51\x44\x48\x4c\x62\x53\x9f\x33\x03\xf3\xb4\x47\xf4\x9b\x6a\xf8\xce\x02\x45\x5a\x00\x96\x5a\x36\xac\xd5\x42\xa6\x16\xab\x1b\x9e\xa1\xdb\x12\x20\x90\x47\x99\x11\x2e\xc3\xc3\x9d\x2c\x2c\xe1\xeb\x00\xac\x96\x26\xb1\xbf\x15\x5a\x13\x60\xb5\xd0\xe7\x0e\x00\x56\x4b\xf3\xac\xc9\x19\xe1\xe3\xbd\xe6\xba\xeb\x49\x5d\xf1\xe6\x09\x88\x06\x98\xce\xdc\x9d\x85\x2d\xe8\x0e\xe4\xbc\x6a\x23\xb6\x0b\x3c\xb6\x54\x03\xf4\x75\xc1\x63\x4b\x83\x69\x33\x78\x6c\x69\xa8\xed\x05\x8f\xad\x18\x68\x03\xf0\x58\xe3\xdc\x1f\xe8\x4d\xdd\x8c\x29\x40\x42\xd5\x30\x1d\xdd\x03\xc4\xc0\xd2\x31\x5e\x98\xc0\x01\x73\x8d\xb9\x3b\xda\xc6\x17\xc1\x68\x6d\xee\x6d\x5d\x38\x56\xc9\x09\xb1\xee\xde\xcb\xbc\x6f\x06\x74\x64\x5d\xb3\xfb\x89\x6f\xed\x06\x3b\x64\x84\x67\x16\xcb\xa0\xae\xc4\x51\x7b\xb2\xb6\x37\xc3\xe5\x05\xec\xcb\x02\xcb\x6f\x84\x5c\xf7\xb9\x54\x2d\x64\xc2\x5f\x6c\xc5\x2e\xd8\x86\x66\x53\xd6\x6e\x41\xe8\x74\x60\x95\xb6\x3a\xca\x51\xa6\xc8\xb8\xac\xd3\xe6\x87\x86\x32\xf5\xdd\x5f\x56\x72\x22\x03\xed\xe9\xd4\x43\xaf\x66\x47\xe6\xec\xb0\xcf\x48\x8c\xa2\x89\xd6\x8a\xa4\x56\x5f\xf4\x74\xcc\xcd\x2a\xd1\x14\x53\xa7\x48\xa5\xd2\xb8\x96\xa8\x7c\x64\x05\x2c\xdc\x33\xf4\x11\xca\x20\xe3\xe9\x4c\xeb\x5f\xd9\xfc\xa8\xde\x49\x8f\xe9\xb7\xdf\x7e\x47\xd0\xb7\x68\x4a\x30\x2b\xe8\xb0\xa0\x36\xe9\xab\x0f\xb0\x23\xd5\x84\x3c\xb2\xca\xa5\x40\xdd\x2f\xa6\xb6\x99\x8b\x37\xec\xb1\x11\x77\x3a\x31\x94\xf7\xc4\xd1\x04\xc9\x74\x68\xea\x53\x7b\x36\x0c\x27\x48\x5f\xf1\x31\x38\xaa\xe1\x46\x76\x83\x5e\x76\x0a\xf7\x1b\x03\x60\xdd\x8d\x4d\x6f\xe3\x0e\xdc\x23\xa7\x92\x14\x30\xc5\x2a\x9c\x66\x86\xf3\xf9\x07\x5f\x1a\xbc\xa1\x13\xe3\x43\xd0\xfa\x19\xb6\x96\x7d\x2d\x4b\x43\x38\x31\x78\xc9\xd2\x04\x0b\x7b\xf4\x1f\x99\x56\x34\x04\x79\xa6\x3c\x95\xc9\x1c\xc5\x9c\x91\x13\xd8\x09\x69\x34\x31\x8e\x55\xad\xb3\x60\x5b\x28\xe5\x99\xca\x54\x2b\xb4\xd0\x96\xab\xcb\x22\x15\x36\x58\x68\x13\x0a\xfd\x68\xf5\x9b\xc0\x57\xca\xcb\x8f\x44\xcd\xb4\x28\x1f\xae\xb8\xc4\xf3\x1b\xc2\x15\x17\x76\x55\x80\x2b\xce\xe0\x8a\x17\xe9\xd2\x46\xb8\xe2\xd2\x9a\x37\x83\x2b\xae\x5a\xf2\x0d\xe0\x8a\x0b\xcd\xbc\x19\xb8\xe2\x12\x45\xdf\x0c\x5c\x71\x69\x5e\x01\xae\xf8\xed\xc1\x15\x6f\x09\xc8\x5b\xcd\x8b\x0d\xae\x97\xa2\x6c\xbe\xf6\x26\x7b\x2f\x51\xef\x46\x6f\xb0\xe8\xa9\x18\xd4\x96\x5d\x57\xdb\x83\x00\x57\x33\xa1\xf5\x40\x80\x2b\x55\xf5\x7a\x56\xb7\x2d\xb0\x18\x28\x06\x07\x06\x01\x2e\x4c\x20\xc4\x57\xae\x1f\x5f\x59\xb9\xf9\x6c\xdf\x7a\x78\x2e\xe8\xb2\x7c\x21\x37\x84\x01\x2e\xac\x4f\xa3\x48\x4c\x10\xdd\x77\xb0\x13\xf7\x2b\xcd\xf7\x0b\x87\x7c\xa5\x2c\xef\x53\x51\x5a\x40\x72\x2d\xe1\x39\x94\x42\xa3\x84\xfb\xfe\xff\xb0\x73\x37\x88\x0c\x2e\x91\x37\xf3\xab\x98\xbd\xd8\x60\xab\x36\xde\xa1\x4e\x2b\xdd\x4d\xa2\xb0\x4b\xde\x5c\xd3\xc5\xec\x06\x71\x3f\x23\x51\x8d\x8d\x99\x4e\xe9\xae\x9a\x5d\x75\x91\x65\x18\x6c\xa0\x90\x2f\xe4\xa5\xea\xeb\xc9\x0c\xc7\xc8\xf8\xa5\x74\x60\x40\x49\x31\x5f\x8e\xa9\x54\xa2\x36\xb6\x69\x61\x84\xdb\xb8\x4a\x67\x69\xe3\x80\x18\x8f\xaa\xe3\xcd\x3e\x9b\x92\x29\x17\xab\x02\xab\x2a\xbf\xb4\x25\x96\x36\xf9\x94\xcc\x26\x64\xaa\x25\x99\xc1\xba\x8d\x34\x5d\xef\x2c\x69\xd9\xe6\xae\x99\x40\xc7\xc2\x26\xf0\x1c\xa1\xfa\xdd\xd8\x20\xa1\x36\x5e\xee\x6d\x97\xd9\x62\xb5\xae\xe9\x10\x72\x20\xde\xcb\x0d\x6e\xf6\xa5\x82\xbb\x1b\xf6\x77\x65\x4c\x47\x16\x52\xb3\x3a\x6a\x63\x49\xbc\xc6\x32\xbc\xb3\xfc\x2b\x5b\x80\x7c\x0d\x57\x7e\xd1\x3b\xaf\x39\xa1\x5f\x7d\x7a\xfd\x00\x8f\x1a\xb4\xde\x45\xf2\x40\x64\x8e\x24\xe2\xd4\xd7\x0c\x0a\x83\x59\xa4\x57\x61\x97\x38\x8d\x72\x8b\x4d\x92\x8a\xda\x28\xd3\x26\x06\xed\x48\xa5\x38\x01\x4d\xc2\xaf\x9a\x5a\x5e\xd4\xe1\xbc\x22\xed\xb1\x99\xc7\x84\x32\xf5\x5f\x7f\x5d\x6b\x35\xb5\x6a\x65\xe9\x06\x95\xde\x70\x14\x11\x69\x6c\xec\x36\x0a\x19\x0f\xf9\x33\x14\x79\xdb\x66\x55\xf5\x51\xd6\xf3\xd6\x0c\x3e\x83\xc0\x8e\xf3\xad\x6e\xc4\x85\x89\xe0\xe9\x78\xe2\x6c\x48\xfa\xcc\xe8\xa9\x55\xad\xe5\xcf\x0b\x36\xf2\xb5\xd7\xf2\xfb\x94\x26\x9b\x59\xe8\xee\x0b\xe5\xef\x3e\xf5\xfa\x48\x4e\xb2\xd3\x3a\x84\x66\x2b\x17\x76\x71\xd0\xcd\xfb\xb4\xdf\x66\xfe\x1a\xe8\xe6\xc4\xc1\xbe\x8e\x78\x92\x80\xa7\x41\x92\xe9\x33\x11\xd5\xdd\xc3\x84\xfb\x74\x3d\xc4\xc6\x6c\x00\xf0\x75\x9e\x18\xd1\x48\xfe\xba\x35\xa2\xa1\x44\x6e\xf4\xe5\xa0\x05\x13\xaa\xc6\x19\x61\x55\x36\xb6\x5f\x16\x2b\x0f\x1d\x59\xc0\xa0\x8b\x1e\xdb\x59\xd0\xa0\x23\xc9\x81\x03\x07\x57\xcc\xa3\xad\xc1\x83\x25\x66\x97\xc5\xf2\xe5\xd7\x8c\x0b\x1c\x32\x8a\x4f\x47\x93\xf8\x91\x75\x0a\xf9\x14\xae\x42\xfb\x70\x9e\x07\x64\x1b\x1d\xc2\x67\x66\x50\xdf\xc5\x1a\x56\xc0\x8d\xa6\xff\x02\x4d\xc7\x80\x26\x9b\x90\x42\x17\x36\x08\xd1\xe4\x24\x3e\xc5\xd1\x3c\x4a\x68\xe4\xe9\xcc\x63\x81\x67\x93\x2a\x8e\xe7\x56\x3e\xa0\x0e\xbd\x16\xea\x50\x5d\x21\xb4\x75\xe2\xb6\xdd\xbe\x62\x78\x4a\x02\x1a\x52\x1b\xd1\x90\x4e\x32\xbc\x0d\x96\x97\x94\x7b\x45\x18\x87\xc5\x73\x1f\x20\x91\x5e\x01\x12\x69\x93\xc3\x9f\xe3\x1d\x15\x8e\x7d\x80\x69\x6a\x42\xbc\xd7\x87\x69\xca\x84\x80\x56\x21\xef\xd4\xf3\x83\x57\x46\x74\x59\x1c\xd8\x6b\xc2\x32\x55\x88\x4b\xeb\xc8\x8d\xcb\x70\x99\x96\xed\x8b\x46\x74\x79\x5d\x94\xa4\xf5\x28\xb3\x16\x00\x52\xe5\xdd\xd9\x12\x38\xa4\xfa\x65\x68\xc9\xb9\xd9\x65\x56\xcf\x7a\x35\x7b\xfd\xcc\x9e\x75\x14\xcc\xf5\x92\x7c\xb2\xfd\x70\x5c\x89\x3e\x79\x71\xc3\xcd\x92\x7d\x3a\xce\x07\x4f\x04\x9a\xf0\x24\x76\x20\x1c\x19\xb5\xb2\x0e\xb2\x4c\x88\x8c\x40\x6e\x31\xee\x67\x24\x32\xda\x66\x5e\x88\x6f\x59\x4a\x4f\xb6\x88\x30\xdc\x1d\x30\x9a\x5d\x58\x51\x32\x4e\xb2\x89\xfd\x64\xa5\x74\x21\x8b\xe6\xff\x25\x63\x2c\x50\x08\xbc\x06\xd5\xc3\x5c\x69\xf7\x5e\x31\xb8\x65\xa2\x87\x67\x1c\x15\x55\x25\x76\xcd\x3e\x83\xa7\xcf\xd4\x19\x62\xb0\xdf\xe3\x52\x2f\xa5\x9b\x5d\x23\x4f\x65\x79\xb3\x6c\x10\x0c\xb7\x50\x31\x71\x7b\x70\xa4\x29\xfe\x32\x98\x61\x81\x93\x84\x24\x54\x4e\xf7\x16\x0c\x7d\x51\x74\x57\xeb\xb3\x2a\xb8\x31\x91\xb1\x74\x3a\x34\x5b\xd1\x0d\xc4\x16\xd9\x54\x1c\x89\x94\xf9\xd0\x6e\xd9\xc2\x64\x45\x3c\x53\xb8\x17\xc0\xaa\x16\x4d\xa0\x5a\xf2\x08\x53\xc1\x88\xac\xad\x4d\x4b\xa2\x54\x50\x35\x1f\xd8\x52\xbf\xcd\x0f\xdc\xbd\xfd\xf2\xc2\x7e\xb8\xdc\xc3\xef\x50\x0d\x5c\x7f\x59\x69\xe1\x19\x11\x50\x9e\xcb\x15\x9a\xf2\xca\x19\x5b\xd4\x0a\x92\xd5\xf8\x82\xf0\xef\x85\x6b\xbb\x2e\x70\x1a\xbf\x0c\xbc\x8c\xb2\x41\x54\xde\x1c\xab\x0e\x6b\x15\xee\xd6\xb2\x49\xee\x19\x79\xaa\xc6\x8b\xbe\x87\xea\x3e\x36\x6d\xc4\x34\xad\x07\xec\xb9\xc2\xc1\x5e\x9b\x2f\x8c\x97\xf2\x5f\x51\xec\xc6\x1b\xa7\xc5\x3a\xaa\x0a\xbe\x5a\x32\xd8\x8e\xf7\x55\x83\x11\x7b\x9d\xec\x68\xd8\xfa\xa0\x0b\x91\xce\x14\x1d\x2e\x42\xfb\x38\x6e\xb0\x83\xd2\xbd\x9d\x04\xd2\xcc\x9d\x9b\xa5\xd0\xad\xa9\xe7\x5b\xe0\xc4\x76\x76\x5a\xfe\xb7\x38\x6a\x0e\x21\xc9\x20\x4c\xf9\x79\x8c\x37\x53\xaa\x94\x4b\x94\x30\x06\x78\xbd\x3b\x8b\xb6\xe9\xf7\x2e\xdc\x05\x43\x85\x65\x63\xa2\x3a\x7b\x64\x1d\x89\x5e\x08\x62\xc4\x42\x68\x54\xd4\x4e\xce\xac\xfa\x50\x73\x6d\x48\x74\x4f\x59\x6c\x8e\x16\x1e\xa8\x92\x59\xd9\x3f\xd3\xc7\x08\x27\x92\x9c\xe8\x86\xa1\x5a\xb0\xe2\x10\xfc\x8a\xd1\x8b\xc0\xb3\x19\x11\x8f\xcc\x66\xb1\x80\xc3\x89\xf3\xc4\xb4\x5f\x17\xe2\x6b\x69\x40\x06\x11\x8e\x26\x07\x5a\x23\x0c\xc9\x48\xd1\x84\xc4\x2e\x5f\xba\xb8\x3c\x6e\xde\xc6\x60\xbf\xc6\x62\xf5\x46\xae\x6c\xdd\x89\xed\x24\x89\x34\x47\xc9\xca\xbb\xcf\x88\xd0\xa3\xd6\x7b\xf8\x99\x30\x44\x47\x6e\x1c\x36\x76\x09\xbd\x80\x67\x4e\x6f\xfd\x67\x4c\x13\x03\x40\xe0\xba\x76\x42\xa0\x71\x3f\x3c\x32\xe3\xee\x67\x51\x21\x43\x97\x32\x2a\x27\x9a\x53\xa7\xe0\x93\x05\x35\xa3\x2e\x71\x88\x3d\xaf\x73\x9a\xbb\xfa\xf5\xe5\x1c\xf4\x99\x0a\xce\xa6\x90\x24\x64\x71\xa9\x1c\xf9\x24\x51\xd9\xf1\xa8\x4c\xf1\x5c\x29\x11\xc7\xb1\x2c\x1a\x5f\x8d\x5a\x49\xff\x59\x30\xbb\x9c\x16\xb2\x22\x23\x0f\x56\x09\x82\x58\x5d\x45\xbf\x65\xf2\x6f\x48\xed\x58\x4c\xed\xa8\xa6\x4d\x1b\xd3\x3b\xb2\x43\xbc\x6e\x8a\x47\xdd\xf2\xef\x42\xb2\xdd\x61\xaa\xc7\x2b\xe7\x44\xec\x27\x1d\xe2\x75\xf3\x57\xf6\x91\xba\x12\x12\x3c\x5e\x31\xc1\xa3\xb1\xa5\xb6\x18\x9b\x5e\x7f\x6c\xd7\x4a\x8e\x58\x01\x66\x55\xd5\xcb\x67\xa2\x04\x8d\xe4\x2e\xf8\x83\x9c\xe1\x86\x51\x7d\xa0\x05\xce\x56\x48\x4d\xfa\x85\xcc\x09\x0a\x71\x72\x59\x85\xcb\xa1\x20\xf8\x29\xe6\x2f\x0b\xb6\x3a\xe9\xa3\x89\x7c\xe6\x5a\xec\x11\x24\xa2\x92\x14\x22\x79\xa8\x44\x8c\x48\x6b\xec\xc4\x8f\x6c\x42\x89\xc0\x22\x9a\x40\x76\x67\xbe\x30\x26\x4b\xd8\x00\x3a\x99\x58\x0e\xdf\xdb\xb5\xc6\xa2\x37\xa0\x7b\xd9\xc2\x94\xe1\xf3\xd9\x35\xd7\x23\x99\x9a\x4f\x32\x61\xc6\x4a\x19\xbe\x49\xae\xd1\xf2\x6f\x9b\x88\x90\x11\x7b\xaf\xc9\x08\x59\x30\x95\xf7\x45\xc3\x84\x84\x7c\x37\x84\xa4\x84\x3d\x25\x25\x54\x90\x78\xbd\xc4\x84\x8d\x4c\x7e\x87\x8f\x99\x76\x3d\x1f\x22\x6e\x7a\x55\xd0\x5a\x3a\x1c\xec\xfd\xe8\x55\xce\xb9\xe9\x09\xfc\x25\xdb\x14\x46\x22\x16\x7a\x9f\x0d\x49\x1c\x03\xa7\x55\xdc\x56\x68\xcf\xf7\x8e\x33\x0f\xe8\xbb\x17\x4b\xbd\xd9\x71\xc2\xd9\x58\xd2\xd8\x80\xcd\xcc\x30\xd4\x2a\xf6\x8d\x17\x00\xae\x00\xeb\x9b\x24\x44\x38\xaf\x84\x40\x7f\x90\x94\x59\x34\xc9\xec\xb7\x98\x13\xc9\xde\x2b\x63\x2c\xc0\x6c\x8e\x9e\x18\x7f\x49\x48\x3c\x86\x15\x2a\x0f\xe6\x14\x51\x72\x82\xa8\xca\x3e\x13\x80\xc6\xc0\x53\xf5\xa8\xc7\x0e\xb1\x76\x46\x03\x20\xf6\x5b\x61\xab\x57\x78\x1c\x58\x7e\x73\x86\x50\x8f\xa1\x11\x8e\xd4\x09\x92\xe9\x30\x6f\x3f\xe6\xa6\xb8\xbc\xd6\xbe\xbd\x89\xe7\x8d\x84\x98\xf9\x8a\xce\xab\xcf\x86\xe3\x0e\x7a\xbb\x76\x12\x8a\xb7\x8a\x2d\x7c\xc6\xdb\x40\xac\x7e\x4e\xa5\x0d\xc2\x40\x9c\x65\x47\xdf\xc2\x4b\x65\x18\xd9\x80\x77\x6a\xf0\xa6\x19\x8f\x6b\x6d\x9d\xa5\xa9\xac\x3b\x96\x3c\x10\xd4\x0a\x4a\xd6\x51\x05\xed\x1a\x72\x6b\xa9\x49\x2a\x41\xf0\xd4\x3a\x07\xf4\x55\x03\x62\x8d\x09\x03\xd5\xa3\xa7\xc2\x48\x98\xeb\x2c\xf1\x15\x65\x4f\x7a\x75\x73\x54\x70\x0e\x78\xc9\xba\xe7\xaa\x45\x9b\xe9\x1b\x8f\x5c\x70\x66\x1c\x84\x5b\xc9\x9d\x74\xcc\x70\xb2\xa6\x8d\x63\x81\x72\x8b\x3e\x3d\x27\x67\x59\x71\x41\x4b\x11\xc6\xd8\x87\x4c\x8f\x6b\xd9\x90\x4a\xf3\xf5\xe5\x3d\x8c\x62\x32\x23\x2c\x26\x2c\x9a\xc3\x16\x61\x80\x1c\x24\x18\x4e\x10\x86\xef\x70\x72\x86\x2e\x4d\x7e\x51\x26\xe1\xd9\x6b\x1d\x2e\xf4\x29\x66\x74\xa4\xf5\x04\x30\xc2\xda\x51\x3e\x32\x33\x4c\xe7\x03\x21\xb9\x75\x35\xa3\x58\xd5\xca\xe8\x1b\xe4\x7a\x4b\x54\x66\x56\xfc\x1e\x2d\xbf\x70\xa0\xb7\x65\xab\xa3\x9b\x73\x35\x18\x64\x3a\x3c\x85\x7f\x17\x12\xee\x1c\x50\x51\x8e\xa2\x43\x12\x02\xe6\x40\xeb\xf1\x82\x8b\xb1\x0e\x58\x6f\x17\x7e\xbb\x15\x79\x2c\x5e\x1f\x05\xa5\x66\x4a\x19\x9d\xa6\x53\xcf\x79\x67\x2a\x36\x44\xd6\x7e\x69\x32\x51\x66\x5a\x0f\x88\x1c\x78\x3b\xd2\x97\x2b\x9b\xa3\x31\x7d\x26\xec\x91\xcd\x38\x65\xea\x0c\x5d\x73\x45\xbc\x12\x19\x06\x3a\x8b\xcf\x14\x9d\x1a\xb4\x57\x41\xf4\x39\x30\xa0\xe0\x00\xb4\x39\xc1\xea\x04\xc5\x29\x1c\x55\x46\x94\x66\x1d\xfa\xc6\x55\xb0\x32\x10\x1f\x2e\x1e\x99\xb9\xe9\x46\x98\x26\xa9\x20\x56\x66\xc5\x26\x2f\x28\x1f\x72\x3e\x32\x8b\x04\xe7\x4d\x62\x4a\xc7\x13\xa5\x97\x48\xcb\x78\xd6\xdf\x38\xd1\xdc\x88\x3f\xb2\x21\x41\x18\xcd\xb8\xa4\x8a\x3e\x67\xfe\x4b\x3a\x42\x58\x4a\xb0\xa0\x9c\xa1\xcb\x82\xfd\x9f\x4a\x50\xbd\xeb\xe2\x8a\x29\x1b\x58\xdb\x73\x7d\x3e\xd2\xd6\x0b\x59\xe8\xc5\x52\x19\x0f\x25\x4f\x52\xe5\xbb\x60\xab\xd7\x36\x37\x8d\xbb\xc2\x05\x60\x20\xe6\xa3\x47\xe6\xf6\xb5\x3c\x43\x1d\x89\x24\xd7\xab\x24\xcd\x52\x46\x82\x2a\x22\xa8\x41\xb1\x22\xca\x2c\x42\x76\x4e\xb3\x33\x30\xc5\xe2\x49\x8b\x50\xbe\x05\xde\x60\xaa\x16\xac\x1d\x43\x23\x21\x01\xac\x97\xbf\x1c\x60\xfa\x47\x8c\xb3\x53\x46\xc6\x78\xd5\x8a\x3c\xb2\xc2\x92\xa0\x3f\xd0\x51\xae\x90\xd6\xf9\x1c\x3d\xda\x0d\x20\xf2\xa9\x6e\x95\x4c\xc7\x75\x8b\x34\x4a\x38\x5e\xe1\x36\x1e\xe5\x87\x1e\xfd\x83\x0f\xcd\x18\xb5\xde\xcf\x15\x48\x81\x5a\xbd\x1a\x71\x41\x26\xff\x1f\x7b\xef\xda\xdc\xc8\x91\x9c\x0b\x7f\xf7\xaf\x28\xcb\x6f\xc4\x0c\x8f\x41\x50\xa3\x3d\x76\xc8\x74\x28\xe2\xe5\x90\x1c\x09\x2b\x0e\xc9\xe5\x45\x5a\x9f\xc5\x06\xa6\xd0\x5d\x00\x7a\xd9\xa8\x6a\x75\x75\x93\x83\xb5\xf7\xbf\x9f\xa8\xcc\xac\x4b\xdf\x80\x6e\x82\x1c\xc9\x3e\xfb\xc1\x5e\x0d\xd1\x5d\x55\x5d\x97\xac\xbc\x3c\xf9\x24\x97\xf1\xc8\x2e\x56\x75\x6c\x70\x33\x92\xab\xcd\x1a\x63\xa0\x09\x5a\x12\x65\x81\x5c\x54\x5c\x06\x6b\x41\x86\x1b\x2d\x85\x5f\x87\x41\x77\x85\x6b\x0d\x6a\xbf\xa0\x03\x02\x45\xde\x26\xa3\x23\xae\x93\x75\x96\xfa\x9c\xae\xc0\x37\xba\x30\x2a\x96\x95\x91\xea\x11\x5c\x57\xd6\x6a\x83\x5b\x9d\x56\xce\xec\xb3\x96\x91\x3b\x41\x0a\xb7\x86\xf5\x79\x61\x19\xd0\x40\x84\xbd\xd5\xc2\xfc\xb3\x10\xde\xec\x43\x65\x7d\x2a\xad\x0a\x72\x00\x52\x86\x9a\x0d\x9c\x67\x46\x85\x46\x9a\x5b\x9a\x3f\x16\x61\x90\xbb\x72\x4e\xe8\x30\xd8\x47\x5b\x2f\xaa\x22\x31\x6a\xf6\xfb\x04\x18\xba\xce\xf6\xa4\xdd\x4f\x64\x2c\x3a\x8b\x59\xf5\x92\x1a\x5d\x77\x0b\x0a\xd4\xd9\x73\xeb\x4f\xdc\xad\x84\x16\xac\x78\x72\x44\x71\xc6\xae\x02\x97\x65\x2e\x52\xf1\xc8\xfd\x1d\xe7\xfa\x22\x71\x19\x71\xdd\x51\x1e\x06\xd8\xd6\xcc\xf8\x9f\x9f\x38\xed\xc6\x77\x65\x86\xf2\xc8\x53\x4a\x5c\xa1\x58\xb9\xee\x5e\xb0\xc9\xd9\x5e\x18\x52\x6a\xa5\x6d\x3e\xbb\x55\x0c\xdb\xf7\x8f\x62\xd3\x3e\x23\x3b\x48\x0c\xb7\x65\xa3\xbb\x69\x1f\xe0\xab\xbe\xf6\xef\x34\xe7\xb8\x73\xe6\x7e\xac\x7c\xf2\xaf\x90\x44\x75\xdd\xa8\xc0\x0d\xff\xa9\xcb\xc5\x22\xf9\x0c\x56\xad\xbd\x49\xac\xe5\x11\xe5\x4a\x1b\x29\x06\xba\x0a\xb3\x8b\x87\x81\xe4\x7d\x12\xaa\x5a\xdf\x34\x56\xd6\xe0\x1d\xdd\x39\xdb\x7f\x28\x45\xbe\xd7\x7c\xbb\xad\x3a\x04\x8e\x18\x9c\x92\x76\x1b\xd1\x36\x5a\xf0\x9e\x98\xa4\xb0\xd5\x3b\xde\x31\x75\xbb\xe9\xcf\x5b\x5f\x9b\xa3\xf0\x1d\x3e\x90\x50\x6a\x6f\xf5\xa9\x79\x7c\x9a\xc3\x26\xdb\xa2\x44\x46\xbe\x8d\x88\x60\x9d\x13\x70\xca\xa5\x3a\x73\x8b\x19\xc1\xc0\x08\xd6\xac\x2a\xa8\xd2\x42\x60\xa9\x53\x63\x89\x5c\x4e\xa5\x9d\x5b\x3d\x62\x08\x13\xaf\x09\xd4\x0a\xb7\x3d\x0f\x5e\x75\x1b\xbb\x9f\x4b\x15\xe3\xea\x52\x68\x6d\x2e\x46\x5d\xe4\x3c\x91\x14\xc3\xb1\xf3\xa3\xa7\x92\x1d\xd6\x71\xea\x23\xf0\x23\x8c\x6c\xb6\xeb\xc8\x0f\x50\x4f\x25\x66\x9c\xb0\x6f\xd8\xdb\x82\x2f\xf1\x96\x00\xf6\x4a\x9e\x02\xef\x25\x58\x09\x64\x95\x07\xc9\x01\xee\x44\x26\xf1\xc1\xf1\xb6\x3e\xd1\x87\xf0\x16\x9a\x81\x43\x6e\xe6\xd0\x4f\x50\xb2\xf0\xff\x10\xf1\xc1\xb6\x96\xfc\x4b\x0f\x62\x33\xaa\x4f\x72\xf7\xbd\x71\xc7\xf7\x42\x68\xbe\xd6\xc5\x01\x83\xee\x1f\xa4\xe4\x73\x91\xfe\xe4\x3f\x94\x6d\x15\x45\xef\x13\xc9\xf7\x93\x41\xad\xc3\xeb\x87\x40\x9f\x6f\xba\xea\xd6\xb5\x88\x9e\x67\x33\xae\x9c\xa0\x2e\x2b\x98\xe9\x8e\x34\x76\x5b\xd5\x90\x03\xea\x71\x25\xd2\x8c\xc5\xc9\x02\x42\x6f\x05\xec\x17\x47\x1e\x8b\xf5\x7e\x8c\x41\xb3\x2e\x25\x12\x01\x23\xea\xe3\x89\x4e\x3a\x89\x0c\xdf\xf8\x78\x2a\x27\xc5\x1b\xcd\x74\x91\x2b\xb9\x34\xc6\x74\xfc\x98\x68\x5f\xc8\xce\x1c\xc8\x72\x2d\x72\xea\x22\xd1\xa8\x75\x53\x11\x28\x6e\x2f\x36\x33\x36\x73\xf5\x81\xe2\x63\x8b\x2d\x9a\x3f\xa2\x5d\x61\x46\xa9\x2d\x6a\xaa\x05\xf6\x4e\x8b\x5b\x93\x9d\x5f\xd8\x75\xf9\x53\xe8\x9d\x64\x6b\xef\xc8\xb4\xf2\xf2\xa8\xee\xc6\xa4\x59\xdf\xe2\xc2\x1c\x7c\x21\xf4\xbd\x08\x6c\xd5\x84\x12\xd3\x9c\x4c\x3f\x36\xb0\x85\x83\x1b\x64\x62\xd5\x3e\x90\x46\x8d\xd6\x53\xe8\xc2\x14\x09\xf8\x3f\x74\xc1\x8b\x24\xa2\x5b\x40\xe5\xe4\xc5\x25\xbb\xba\x7b\x69\xf7\xb5\x49\x74\xc4\xd3\xe6\x0a\x6f\x89\xa9\xe3\xf3\xdb\x1d\x9d\x74\xdc\xb0\xed\xad\xc4\x2e\x91\x4a\xd3\x21\x65\xea\x6a\x5f\x7e\xea\x5f\xdf\x3e\x22\xdf\x8f\x59\x00\xbb\x16\x70\x6a\xd0\x41\xc1\x53\x0a\x17\xe9\x82\x56\x29\x7c\x08\x2f\xb5\x0d\x99\x8f\x53\xa9\x16\x50\xc8\x30\xed\x42\xae\x67\xb9\x5a\x27\x43\x2a\x69\x20\x98\xfb\xc6\xc6\xfe\x77\x44\x52\x2c\x42\x00\xdc\x6f\xb8\xbd\xa8\x47\xe0\x64\xe0\xe4\x52\xdb\x72\x86\xd6\x3c\x7b\xd6\x84\xef\x42\xbe\x9c\xb0\x35\xc2\x8e\x68\xf6\x80\x53\x5b\x40\x72\x2a\x4c\xf2\x13\xdf\x78\xfa\x9b\xae\x1a\x09\x72\xd0\x76\xb8\x37\x8f\x4f\xe4\x42\x0d\x38\x9c\x9e\xae\x86\x4e\x1f\xb7\x7b\x36\x38\x7f\x0e\x89\x81\xab\x8f\x73\xda\xe7\x3c\x9e\xb6\x6d\xea\xc1\x27\xd3\xce\xe0\x6b\xc6\x61\x43\x21\x12\xbc\xf3\xb7\x21\x77\x6b\xf5\x68\x05\x2d\x32\x18\xce\xf6\xa9\xfa\x58\xd9\x87\x2f\x3e\x47\xb5\x76\xe0\x37\x9f\x30\x76\xdd\xde\xea\x17\x98\x33\x3a\x24\xbd\x26\x6b\x4f\x7e\xae\x61\xb5\x1e\x6c\x8f\xae\xb2\xc3\xde\x96\xdc\xae\xc9\x00\x69\xa6\xc9\x6a\xf0\xd9\x27\x44\x03\xb0\x48\x52\xa1\xc7\x6c\xd2\x12\xc4\xb5\x49\xf8\x0e\x34\x8e\xe9\x80\x56\x7b\x2a\xf3\x24\x28\xfe\x6e\x75\x24\x96\x40\x11\xba\x10\xc8\x12\x04\x2d\x20\x7c\xba\x52\x4f\x98\x81\x97\x27\x46\x66\xa1\xb2\x5a\x40\x48\xcb\xc8\x82\x84\x22\x42\x18\x50\x73\x2f\x28\xcc\x8b\x30\x66\x8e\x0b\x86\x85\x1e\x88\xfa\x92\xbe\x44\x19\xcf\xfe\x1c\x07\xb6\xd7\x3b\xf3\x46\x1f\xa3\xc0\x3e\xbb\xc7\xe8\x9c\x96\x3f\xdc\x1f\xf9\x01\x5e\xb5\x8e\x5d\xce\x16\xb9\x00\x2b\x7b\xed\x78\xd3\xb0\x70\x82\x52\x70\xdf\xdd\x9e\xfd\x78\x74\x3f\x61\xa2\x88\x58\x9a\x3c\x88\xa9\x8c\xf4\x23\x18\x7d\xbf\x94\xa2\x30\x7f\xee\x70\x02\x25\x6b\x21\x35\x48\x82\xa4\xe8\x69\xaf\xd9\x89\x31\xff\x7b\x56\x7d\xbf\x8f\x55\xee\xb8\x3e\xcd\xde\xb5\x35\x0d\x61\x9b\x42\xd9\x36\x9c\xda\x16\xbf\xe6\x7b\x8c\xb7\x9e\xb7\x55\x3c\x7f\x46\x4a\xb4\xfc\x4b\x29\x07\x2a\x5d\xa7\xfe\xa5\x60\x14\x1d\x3a\xdd\x3a\xe3\x50\xcf\x64\x58\xae\x35\xbe\xd3\xda\xfa\x2e\x21\xe2\xa9\x6f\x6c\xfc\xdc\x17\x87\x67\x45\x2e\x04\x88\x10\xb7\x9f\xe8\xae\x27\xb6\x35\xf7\x61\xc1\x4b\xe3\xa9\xfc\x68\x51\x75\xfe\xaf\xda\xc7\x1a\xd6\xf3\xa0\xcc\x4b\xb5\x15\x68\x36\x4e\xb4\xfb\x03\x14\xed\xd3\x65\x5a\x60\xd5\xe2\x45\x22\x79\xea\x06\x8a\xbf\xb4\x49\x89\x9c\xcb\x68\xb5\x6f\x98\x3c\x59\xcc\x44\x3a\x44\x13\x9d\x2c\xce\x53\x6d\xf6\x77\xf4\xd0\x71\x3a\x9f\x53\x97\xdb\x7f\x0c\x46\x9c\xa8\xb6\x27\xf3\x61\x76\x9e\x62\xd5\x60\xc1\x00\x87\x55\xcf\x90\x47\x12\x30\xb3\x8a\xa4\xa9\x23\x0c\x0b\x53\x53\x5d\xda\x19\xf4\xc2\x78\x31\x95\x79\x29\xa1\xa0\x98\x43\x65\x72\xe6\x6b\xc2\x44\x16\x23\x41\x88\x95\xa5\x11\x13\x58\x72\x05\x1f\x36\xf6\x99\x2a\x35\xc4\xa3\xd6\xa2\x30\x17\xd4\x5b\xa8\xf5\x8f\xb0\xe8\x11\xcb\xf2\x64\x0d\x21\x65\x7d\xd0\xb2\x74\xa7\xbc\xe0\xa9\x5a\xbe\xb4\x57\xe9\x99\x29\x36\x76\x18\x6c\x72\x66\x26\x7f\x29\xa4\xc8\xe1\x43\xc1\x97\xdd\x7a\x84\x7b\x78\xb9\x3b\x24\x37\x44\x12\x29\xf8\xab\x9d\xc7\x82\x97\x85\x5a\x1b\xfb\x96\xa7\xe9\x66\x84\x51\x67\xc1\x56\x5c\xaf\xec\x42\x63\xc0\xb0\xcf\xdd\x44\x93\x7b\xca\xa3\x95\xb8\x2d\x78\x51\xb6\x22\xb3\x6a\xa3\xfc\x4a\xc8\x72\xfd\xd5\x31\xfb\x93\xff\xc6\xd3\x93\xd3\x1f\xce\x67\x67\x93\xdb\x93\xf7\x17\xe7\x67\xc1\xf7\xd0\x2f\x1f\x27\xb7\xb7\xcd\xbf\xfe\x30\xb9\x6b\xfe\xf1\xfa\xea\xfa\xfe\xe2\xe4\xae\xad\x95\x8b\xab\xab\x1f\xef\xaf\x67\x1f\x4e\x26\x17\xf7\x37\xe7\x2d\xaf\xde\xdf\x75\xff\x78\xfb\xe3\xe4\xfa\xba\xad\xd5\xf3\x9f\x26\xa7\xa6\x3b\xfa\xfb\x9f\x83\x63\x07\xa1\x73\x33\x03\x1d\xdf\x57\x3f\x99\x87\xac\xfa\xe0\x31\xbb\xaf\xd7\xbd\xa2\x44\x2c\x24\x11\x7b\xe2\xda\x08\x37\xc8\x03\x04\x17\xac\x9f\xad\xae\x57\x11\xab\x1c\xad\x04\x4b\x95\x7a\x28\x33\x92\x79\xe8\x6d\x97\x0a\x3d\x42\x42\x07\xad\xfd\x30\xb9\x3b\x6e\xd6\xdf\x72\x8d\x05\x74\xa9\xce\xb9\xfc\xc4\x91\x3a\x00\xe4\x2c\x38\x59\x6c\x5d\x26\x1f\xba\x0e\x7a\x70\x4b\xb6\xad\x1f\x6c\x8d\xcb\xa2\xd6\x4d\x1c\x7b\x92\x25\xf8\xb0\xa0\xe1\xea\x82\x6f\x9b\x4d\x37\x1d\x58\x78\x94\xcd\x45\xc4\x4b\x44\x74\x9b\x0b\x2c\xcf\x55\x1e\x0e\xd8\x6f\x94\x97\x6b\x94\x36\x58\x6b\x83\xb5\x35\x33\x1f\xae\x1f\x92\x2c\xab\x2c\x3b\x6d\xc4\xdd\x2b\x0f\xa5\xde\x1e\x93\xa8\x10\xf1\x57\x4d\xbd\xc8\xe7\xe0\xa3\xde\x6c\x4e\xb5\x19\x72\x70\xd6\x13\xb9\x44\x5f\x82\x2d\xba\xb7\xda\x38\x14\x12\x80\x5e\x3d\x0c\x18\x8a\x80\x98\xbb\xc6\x15\x45\x4b\x00\x56\xc4\x0b\xf6\x24\x80\x8e\xa6\xa4\xaa\xa3\x68\xd3\x1b\x99\x01\xdd\x21\x1e\xc0\xd6\x10\xae\xd0\xd4\x74\x0a\xf9\x97\x50\xe4\xcd\xfb\x5a\x0c\x0b\xe2\xed\xe4\x14\x39\xc3\x46\x41\xea\xdb\x7c\x01\x18\xf1\x4b\x06\xfd\x5a\x6e\xba\x1d\x97\x90\xb9\x0e\xfa\x8c\xc7\xf2\xb8\x55\x8a\xa3\xf4\x1f\x58\xa5\x80\xc6\xce\xb9\xba\x53\x31\xdf\x98\xcd\x01\xa0\x07\x5d\x66\x99\xca\x0b\xd6\xd1\x06\x42\x20\x71\x7c\x70\x97\xd1\x77\x38\x11\x09\x8d\x18\xcd\x45\xb7\xd4\x61\xeb\x47\x2d\x45\xf3\x1a\xc4\xce\x82\xe4\x22\x30\x30\x5d\xcd\xcc\x75\xc5\x54\xaf\xec\xd0\x36\xa5\x7a\x9f\xec\xcc\xcc\x28\x0e\x7d\x4b\x38\xb7\xf5\x7e\x65\x5b\x68\x5d\xf2\x54\x2c\x8a\xd9\xc0\x60\x17\xb4\x28\xbb\xd8\x00\x93\xe5\xea\x05\x5a\xec\x6f\x7d\x7c\x43\xa0\x68\x63\x72\x04\x9e\x87\x5c\xa9\x02\xf5\x5e\x6f\x1b\x31\x3b\x9b\xe0\xb6\xa0\x4e\x29\x8f\xde\x29\x97\xc6\x96\x40\x2c\x99\x4b\x39\x1f\x4f\xe5\x39\x80\x4f\xbd\x81\x63\xd3\xeb\xc1\xba\xd8\x69\x57\x54\x0a\xd6\x7f\xd1\x4c\x97\xee\xea\x00\x7e\xdf\x23\x64\x51\xa4\x1b\xc7\x11\x15\xb3\xca\x7b\x7d\x4e\x0f\x7a\xd3\xad\x6a\x89\x1f\x8c\x47\x47\x17\x22\x23\x8f\x3f\x7e\xa7\x47\x49\x43\xb4\xd9\x74\x35\x66\x3f\x5b\x8f\x12\x24\x0d\xb9\x24\x1a\x8b\x7b\x4d\xf9\xc6\x12\x8a\xb7\x4d\xec\x4b\x70\x74\xbf\x74\x1a\xd1\xf6\x09\x76\x64\x9c\x2d\xb3\x5c\x31\xec\xa5\x44\x4f\xef\x00\x18\xd2\xa9\x7b\xe9\x56\x6c\x47\x54\x7e\x80\x12\xee\x84\x4a\x07\x9d\x45\xa6\x9b\x7f\xc4\xc5\x42\x16\x0f\x0b\xd2\xa0\x92\xda\x14\x99\x35\xe7\x07\x22\x8b\x48\xf2\xc1\x16\x49\x9a\x82\x1e\x30\x66\x27\x72\x63\x49\x30\xcc\x55\x68\xc1\xa9\xc9\x52\xaa\x5d\xf9\xf9\x1d\x9b\x29\x0a\x36\xd3\x6d\xf7\x66\x42\xfc\x87\xe7\x40\x7a\x99\x1d\xf5\x02\x7c\x78\x46\xb6\xf0\x66\x35\x95\xfe\x2c\x78\x03\x9c\x02\xe1\x6d\xfe\xa5\x32\xcb\x1a\xc3\x0d\x5e\xfc\x5b\xfb\xd0\xbf\x2f\x79\xce\x65\x01\xf9\x52\xa4\xb4\xe6\x22\x48\xdb\x16\x9f\x01\xdb\x2a\xd1\xc1\x0c\x7f\x0a\x17\xd7\x42\x09\x10\x7e\x96\xc4\x23\x96\x8c\xc5\x18\x2a\xfb\xe6\x46\x97\x98\xfb\x27\x57\x46\x73\x98\xca\x46\x1e\xc8\x98\x9d\xa4\x5a\xd1\x1b\x42\x46\xa9\xd2\x00\xed\x9d\x87\xa4\xeb\xb0\xf3\x29\x5c\x35\xdf\x80\x7d\x03\x4b\xe9\x9b\x57\xf4\x43\xf0\x22\x14\xa8\x85\x58\x7b\x0a\x27\xdd\xff\xfd\x9f\x15\xd1\xc7\x76\xe1\x2f\x5e\xb1\x14\x58\xe3\x1a\x7a\xb5\x45\xc2\x32\xd3\xdb\x16\x08\x9e\x80\x85\xf1\xf9\x39\x01\x7b\x1d\x7b\xcb\x0b\x96\x0a\xae\x0b\xf6\xee\x60\x10\xe6\xc4\x7e\xa0\x97\xae\x74\x7c\x7d\x92\xbd\x4d\xd3\x0c\x95\x3b\xd7\x31\xd4\x1d\xe6\x79\xc1\x38\x93\xe2\x29\xcc\xca\x51\x90\x48\x65\x8b\x09\x8b\x80\x17\x04\xb1\xf8\xc8\x6a\x04\x99\xae\x68\x32\x75\xc8\x11\x5b\x2a\x83\xc2\xb2\x34\xac\x96\x9d\x35\x72\xa8\x36\x80\xb1\x9b\x87\x7c\xc2\xe4\x8a\x17\x53\x49\x92\xd5\xc2\x51\x82\x14\xf9\x93\x34\xad\x26\x29\x72\xc8\xc3\x95\xe6\x83\xcd\xe8\xe3\xb1\x9b\xa0\x4b\x30\xbf\x5c\xa6\x58\xc5\xff\xe7\x0f\x0b\xe6\x32\x38\xae\xc8\xb0\xed\x56\x6d\xa7\xcd\x6f\xfd\x05\x95\xe0\x96\xee\x2f\xd4\x32\x89\x78\xda\x43\x19\x16\x6d\x43\xde\x71\xb0\x9a\xb1\x82\x2d\xba\xf1\x4b\x77\xd0\x5f\x55\x6e\xf7\xbb\xc3\x35\xfb\xa4\x5a\xdc\xf8\x1d\x8b\x1b\xe8\x16\xfb\x18\xe0\x2e\x65\xf1\x4b\x45\x92\x2b\x43\x9f\xc4\x40\x98\xb0\x5b\x0a\x7a\x02\x02\x2b\x3a\x30\x6f\x2d\x0e\xf2\xa1\x83\xf4\x4b\x02\x91\xa2\xe0\xa3\x27\x3b\x22\xba\xd9\x7f\xef\xcf\x1f\xf9\xef\xb7\x9f\xe2\xc0\x75\xcd\x0f\xef\x56\xf6\x4e\xe2\xbf\xf0\x08\xb2\x24\xa1\x27\x9b\x9f\xd9\x24\xb3\xb4\x25\x50\x38\x04\x09\x5a\xd5\xc3\x2c\x57\x91\xd0\x7a\xcc\xce\xe1\xa2\xa1\x7f\x32\xbe\xb0\x81\x8e\xe0\xe1\xa9\x34\x96\x89\xe5\xbe\x0b\xda\xaf\x6e\xf1\xb6\x13\x80\x44\xba\x7b\xc5\x88\xd6\xbb\xeb\xdb\x75\x59\x13\x96\xc7\x17\xda\x80\x92\x58\xec\x7c\x79\xcc\x62\x15\x3d\x88\xfc\x28\x17\x71\xa2\x8f\x21\x66\x5f\x74\x06\x0b\xd7\xc6\xda\xde\x5b\xd3\xe8\x02\x20\xec\x20\x14\x38\xc5\xfe\x29\xa5\xc0\xa6\x26\x8d\x58\xb2\x00\x73\xc2\xe6\xb3\x62\x02\x97\xa5\x0a\x14\xb2\xc8\x37\x88\x76\xb6\xae\xac\xda\x44\x58\x4b\xc3\x28\x6d\x5d\x99\xd8\xf9\x4b\x60\x7b\x9e\xf9\xd9\x98\xf1\x43\x40\x06\xfc\xa8\x42\x51\x1e\x18\x8a\x8b\x8c\x17\x2b\x0d\xb4\x1f\xd5\x39\x20\xa3\x0b\x5e\x35\x33\xc4\x33\xc0\x41\xa0\x97\xc2\xbf\xe4\xc8\x29\x74\x91\xa4\xe9\x54\x62\xe2\x06\x30\x74\xbc\x69\x65\x17\x32\xaf\x8e\x18\x8f\x63\xf6\xff\xbd\xfd\x70\xf1\x1f\x77\xe7\xb3\xc9\x25\xf8\xbc\x27\x17\xe7\x07\x23\xf7\xc7\xab\xfb\x3b\xf7\x57\xf4\xb0\x3c\x8a\x9c\xad\xf9\x03\x98\x78\x52\x0b\x4a\x3c\x15\x53\x19\x8e\xd4\xf2\x2e\x99\x5f\xb4\xb0\x08\x5a\x52\x53\x1c\xfd\x34\xad\x61\x17\x69\x2b\xd1\x71\x0e\x30\x7e\x6f\xdc\x2b\xdb\xf7\xa0\xdd\x3c\xae\x0b\xab\x06\x42\x7e\x31\xd7\x01\x11\x0f\xd9\xbe\x7e\xc3\x09\xb9\x4c\x64\x17\xce\x4f\xc8\xc7\xd7\x54\xe2\x7f\x14\x1b\x00\x9a\x5f\xf3\x24\xef\xbd\xf7\xda\x99\x14\xed\x89\x31\x76\x3a\xd7\xf5\x43\xa5\x51\x17\xc6\x4c\xe5\x4e\x2c\x69\x1b\x89\xee\xaf\xfe\xb9\x44\xcd\x29\x3e\x17\xb9\x65\xf8\x72\xb9\xb0\x96\x06\xd3\x5d\x34\x7e\x0f\x4e\xe5\xdd\xd5\xd9\xd5\x31\x13\x29\x9f\x2b\x48\x83\x24\xa8\x91\x6d\x82\x26\x2c\x52\xeb\xa0\xa1\x0a\xbb\xdb\x88\x65\x9e\xdd\x2d\x74\xa2\x8d\xb1\x8d\x1d\x2c\x6f\x99\xca\x9b\xdc\x68\x2f\x6b\x02\xd2\xc7\x5e\xab\xbc\xcf\xf5\x6f\x1e\xc3\xbc\x90\xcc\x18\x72\x35\xc9\x4b\x77\xf3\x42\x70\x60\xfe\xa0\xb0\x10\xf9\xf2\x09\x18\x9b\xa6\x95\x5a\xdc\xe6\xe0\xe8\x31\x85\xf6\xfd\x93\x4a\xb2\x1f\xbf\xd5\x6c\x5e\x16\x53\x59\x6d\x43\x49\x76\xf2\xf3\x2d\x7b\xcf\x8b\x68\x75\x30\x95\x90\x97\xf8\xe3\xb7\x1d\x34\x94\x83\x99\x9d\xcd\x9c\x9c\xf1\x82\x5f\x28\x1e\x27\x72\xd9\x46\xeb\xec\x6b\x0f\x9e\xdf\x9d\x1c\x33\x5b\x02\xc6\x67\xd1\x16\x96\x4e\x25\x68\x08\x04\x32\x7c\x88\x95\x22\x20\xca\x65\x95\xfa\x16\x2d\x33\xb8\xb0\xa6\xf2\x0e\xf9\xac\x8d\x54\x4d\x0a\x96\x29\xaa\x7f\x69\xac\x32\x64\xfa\xe6\x36\xbb\x5c\xa4\x1b\x66\x66\x07\xb6\xb1\x5b\x0c\xd2\xc7\x40\x9f\x69\x0a\xfb\xa9\x04\x03\xdd\xe5\xf5\xa6\x2a\xe2\x29\x60\xfd\x0e\x03\x9f\x9e\x31\xdb\x55\x09\xdc\x3a\x00\xb2\x91\x9b\x2a\x24\xd7\xd1\x3d\x39\xa5\x2c\x5c\x28\x70\x00\xc0\x3a\x52\x1c\x72\xad\x8c\xc4\x41\x1e\x5b\x70\xbe\xa5\x38\x3b\xe6\x45\xc7\x6b\x8b\xd3\x62\x7e\x75\x29\xef\xaa\x94\x96\xc7\x2d\x02\xf7\xbd\xdc\x00\x2c\x1c\x0a\xd6\x29\x80\x94\x78\xe9\x4c\x9b\xb2\xb1\x8a\xee\x4e\x0c\x5e\x9b\x4a\x44\x20\x56\xd6\x25\x64\x3e\x0c\x7a\x57\x12\x00\x92\xcd\x3c\xfb\x32\x23\xc0\x24\xe9\xfa\x59\x2e\x0e\x5d\xf6\x78\x5c\x99\x53\x73\xc3\x8e\xd9\x4d\x68\x5e\xc7\x2a\x2a\xd7\xb6\x2a\x05\x64\x9e\x13\xb2\x8e\x2e\x51\xb7\x43\xf0\x62\xdf\xb5\xe3\x81\xe1\xae\x10\x40\xbd\xd3\xdb\x3e\xc6\x0d\x73\x12\xbe\xda\xd4\xd4\xbb\x15\x5f\x90\x1d\xfb\xa1\xe1\xb0\xa1\x59\x56\x6d\xa9\xd2\xda\xde\x9c\x0e\x97\x9e\x39\x5f\xe5\xa0\x6c\x89\xcf\x99\x02\x27\x37\x26\x56\xab\xf8\x8d\x66\x93\x6b\xa3\x01\x19\x8b\xd7\x9d\xc1\x52\x17\x08\x5a\xc3\xfc\x67\x78\x1b\xd3\x10\x46\xec\x6b\x36\x2d\xbf\xfe\xfa\x77\x11\xfb\x6c\xff\xe3\x5f\xff\xe5\x5f\x7e\xf7\xaf\x43\xd2\x54\xac\x41\x0e\xed\xfa\x39\x72\xa5\x48\xab\x2a\x51\xb8\x02\x4d\x49\xb5\xc7\x2a\xd0\x01\xec\x9a\xfe\xe7\x30\x64\x07\x98\x24\xbe\xa4\x13\xae\xc3\x93\xc9\x2a\x47\xd3\x23\x09\xb4\x28\x46\x55\x09\xe1\x94\x5d\xd2\xe8\xff\x71\x0b\xd1\xeb\xcc\x1c\x95\xe7\x61\xa7\x92\xd4\xa9\xd7\xa6\x11\xf6\x96\xfc\x7f\x05\x04\x10\x0f\xec\x05\xa7\xd2\x58\xe4\x38\x26\xe7\xb2\x73\x8e\x44\x10\x0e\xe2\x73\x96\xaa\xd8\x52\xcb\x7b\x1e\x85\x04\x14\x84\xf3\xcf\xdc\x48\xee\x11\x51\x90\x52\xde\x2a\x44\x5e\x16\x3c\x12\x94\x63\xfd\xf6\xf3\xb1\xf9\xdb\x88\x6d\x8e\x01\x9c\x3a\x62\x7f\x3d\x26\xa6\x41\x9e\x17\x33\xf3\xa7\x03\xab\x6b\x53\x13\x30\xe8\x44\xb3\x37\x47\x8f\x3c\x3f\x02\xf1\x7c\x84\x23\x7a\x43\x92\xd5\xd5\x54\x0e\x75\xf3\x54\xa9\x07\x02\xee\x36\x5e\x3c\xb2\xa4\xb5\xb0\xbd\x5d\xdc\x04\x97\xde\x91\x1a\x15\xec\x10\x1e\x10\x6c\x9c\xcd\xd9\xf8\x2f\x5a\x49\x36\xde\xf0\x75\x4a\x7f\xb5\xbf\x12\xae\x98\x6b\xca\xb5\x8b\x1d\x46\x28\xdd\xa0\xa7\xf4\x7d\xaa\xe6\xf0\x55\x1f\xed\x97\x22\x32\x17\x06\xea\x6f\x1f\x7f\x61\xd1\x87\x58\x12\x0f\xe0\x5e\x5c\xab\x02\x1f\xa1\xb4\xd9\xe6\x57\x7d\x76\x43\xfa\x23\xc6\x85\x61\x52\x6c\x72\x20\x3a\x87\x1d\x2a\xce\x34\xfa\x99\xbd\x25\x11\x74\x60\xee\x18\x82\x41\xe3\x34\xb4\x75\xb0\x71\x1d\xfc\x47\xd0\x41\x22\x19\xa6\x7b\x6e\x79\xf3\xaf\x47\xe3\xf1\xd8\xbd\x0d\x8c\x3f\xff\x87\x25\x85\x16\xe9\x02\x5b\xb2\x37\xd8\x66\x2a\x3f\xda\xa2\x55\xd6\x79\xed\xe9\xb0\xb3\x5c\x15\x2a\x52\x29\x3b\xf4\x0e\xdd\x58\x45\x9a\xfd\x93\x51\x6b\x83\xa9\x84\x3f\x1a\x3b\xae\x83\x42\x1f\xab\x64\x7c\xa1\x43\x45\x0e\xf1\xfa\xb1\x0a\x19\x70\x9d\x61\xcb\x75\x98\xe4\x0c\x7b\xc1\xec\x9c\x23\x62\xc9\xcd\x73\xf3\xb0\xf8\x5c\xc0\x4f\x1d\x24\xc4\xad\x10\xf9\xf6\x9b\xb2\x21\x6e\x3d\x17\x31\x6e\xeb\x8e\x09\x20\xae\x50\x92\x0c\xf8\x9d\xa3\x30\x7c\x62\x2e\x17\x19\x96\x51\xd2\xe5\x7a\xcd\xf3\xcd\x91\x3f\x6d\xcd\xcd\xe9\x59\x6a\x41\xc6\xa4\x76\x02\x20\x84\x9b\xd2\xd1\x22\x14\x03\xa9\x97\xf6\x46\x73\x67\x37\x82\x3a\xd8\x01\xdb\x93\x90\x91\x8a\x69\x5f\xfb\xac\xd6\xaa\xc6\xe2\x9e\x69\xea\x2a\x16\x11\xa3\xbd\x33\x4e\x16\x48\x7f\x46\x4f\xd8\x97\x3b\xc4\xb7\x9a\xe9\xc2\x08\xca\xe5\x80\xf0\xe8\xe4\xea\xd6\xbe\xd3\xff\xd2\x85\x79\xa8\xaa\xec\x3c\x0d\xb9\x85\xe5\x92\xe5\xfc\xc9\x5f\xbf\x80\xed\x40\xef\x4c\xe9\x72\x7e\xf1\xdf\xa7\xea\x3a\x49\xcd\xad\x05\x7b\x7c\x3c\x95\x95\x3f\x8f\x98\x48\x93\x75\x22\x1d\xb6\x0e\x85\xbb\x5a\xa0\xf6\xfc\x90\x14\x66\xc9\x74\xfc\x60\x24\x98\xe5\xc4\x0c\x4c\xaa\x13\xb9\xb1\x5b\xc7\x05\xa6\xc8\x03\x51\x6a\x33\x2e\x6f\xa3\x03\x1b\x40\x12\x8b\x43\x52\x48\x93\x60\xe3\xc1\xf9\x9d\x4a\xd3\x9a\x3d\x4b\x1e\x86\x1c\xb4\x17\x34\x77\x68\x8b\x09\x05\x12\x00\xfa\xa8\x60\x89\x9d\xfe\xdb\xa2\xa0\x9c\xcb\x72\xbd\x6f\x12\x0b\xc1\x92\x7f\x2d\x37\xdd\x75\x2e\xec\x4d\x45\x09\x51\x42\x96\x6b\x7b\xa0\x06\xec\xb8\x73\x52\x7f\x62\x11\xa5\x1c\x59\xfe\x4c\x43\x80\x7c\x1c\x61\x80\x34\x0b\xfa\xc2\xeb\x05\xbb\xc1\xfa\x84\xa9\x90\x6f\xf1\xdf\x07\x8c\xee\x86\xaf\x47\x74\x9f\xe7\xda\xb1\xa7\xe1\x9a\x43\x7d\x6f\x11\xa3\x0f\x1d\x2a\x3a\x2c\x79\x1e\xa3\xb7\x3c\xb4\x2a\x30\x33\xd8\xe8\x5f\x1b\x55\xb2\xa7\x44\xaf\xa6\xf2\x4e\x59\x87\x23\x93\xca\xd5\xc4\x18\x81\x31\xda\xe8\x8f\x6b\x10\x02\x30\xea\xb6\x1d\x60\x84\xf0\x5e\x39\x4c\x00\xa2\x9d\x49\x15\x8b\xfd\xc8\x1f\xef\x7c\xac\xc2\xc6\xaf\x73\x81\x79\x66\x70\x53\x74\xa5\xe9\x0a\xad\x07\xfa\xe6\xeb\x0b\x0f\xf7\x10\xb5\x63\x7a\x55\x4f\x83\x2a\x93\x84\xbc\xaa\xee\x56\x83\x56\xac\xc5\x19\x64\x19\x57\xe6\xde\x55\x9a\xd8\x77\x11\xa2\x16\xa6\xc7\x5e\x77\x3f\x7e\x7b\x04\xd3\xee\x00\xc6\x9c\x2d\x73\x55\x66\x2e\x15\xdf\xa6\x11\xe2\x32\x90\x4e\x33\x91\x0b\x75\x4c\x36\xd5\x45\x22\x1f\x70\xc7\xbf\xd6\x1a\x61\x31\x11\x11\x57\x28\x70\x6d\x85\x7f\xf8\x86\x43\x96\xc8\x28\x2d\xe1\xe2\xd3\x05\x8f\x1e\xb0\x20\x4a\x97\xd3\xd7\xbc\x33\xdb\x9d\xa4\xd9\xa1\x31\x95\x69\x4a\xdd\xfa\x0b\x14\x48\xe6\xc0\x05\xf4\x98\x70\xc6\xd9\xfd\xcd\xa4\xbd\xef\x87\xa4\x19\xcc\x69\xbf\x3d\xab\x1b\x04\xfe\xdf\x8f\xc9\x20\xdc\x65\x8d\x52\x58\x54\xb6\xba\x73\x2e\x75\x11\xd6\xe3\x26\x2d\x8c\x01\x11\xdf\xb4\xb8\xf6\x07\xef\xd3\x65\x56\xce\xcc\x44\xa5\x43\x00\x02\x66\x14\xdf\x5f\xdf\x9f\x04\xef\x6d\xdb\x2a\xdf\x5f\xdf\xb3\xa0\x0f\x24\x8b\x4e\x45\x54\x38\xa4\xf1\x98\x9d\xfa\x1a\x0e\x75\xcd\x3c\x16\x8f\x49\x84\xa9\xb3\x23\xa3\x15\x4d\x25\x50\xa3\x1b\x5b\xe7\xd0\xf2\x69\xb2\xef\xaf\xef\x89\x85\xd3\xf3\xe6\x60\x39\x0a\xa0\xc6\x18\x76\xed\xd4\x48\xc9\xa5\x92\x87\x48\x19\x94\xc7\x3e\xda\x31\x02\xe3\x3a\xe2\x59\x51\x92\x82\xf1\xf8\x6e\x6c\xd7\xe4\xc6\x47\x42\xcc\xb0\xd4\x54\x1a\x5d\x09\x73\x0c\xa0\x72\x9a\xf9\xe8\xe6\xd2\xd6\x26\x75\x1f\x70\x00\x4c\xda\x5e\xc2\x3f\x71\x99\x83\x5c\x6e\x18\xcf\xe7\x49\x91\x1b\x33\x0c\x5f\x1e\x21\xc3\xd9\xca\x56\xc7\xc2\x75\xf3\x9a\x11\x15\xbb\x83\x05\x4e\x64\xa1\xa7\x32\x48\x80\x71\xd9\xc6\x98\xbc\x90\x48\x06\x54\xc2\x80\xbd\xb1\xd4\xa6\x51\xaa\xca\xd8\x5e\xab\xb9\x2b\x9e\xb7\xc9\x50\x89\x9a\x4a\x60\x3c\x31\x77\xab\x32\x6a\xa8\xbf\xfb\x8f\xd9\x27\xf9\x98\xc4\x09\x3f\x2c\x84\x4e\xf9\x61\xf1\xbf\x3f\x8d\x6a\x7f\xe2\xef\xbe\xfe\xfa\x13\xd6\x01\xec\xa2\x73\x08\x58\x9b\xf6\x74\xf0\xb4\xc7\x29\x1c\xff\xa1\xd9\xa5\x7b\xac\xd3\x45\xf2\x20\xd8\x27\x5c\xee\x4f\x44\x7e\xfc\xbc\x65\x9b\xca\xb6\x75\x63\xcf\x59\x36\xa0\xa2\x6f\x5f\x37\xb6\x65\xd9\xde\x2d\xc7\xff\xb2\x9c\x9b\xd5\xfa\x66\x39\x7e\xf7\x35\xfc\x67\x6d\x8d\x76\x1d\x5e\x97\x3d\xd3\x36\xec\x16\x41\xd4\x72\x2c\x9d\x2c\x9a\xca\xdd\xc2\x88\x0d\x93\x45\xb0\x6b\xdb\x0e\x3e\x2f\xc4\xbe\x59\xb3\xc8\x89\x3d\x00\x7d\xdd\x20\x1b\xdf\x1a\x11\xdc\x93\xa9\xdb\xb3\x6c\x03\xdc\xb3\x9b\x32\x3c\x04\xe0\xc2\x8f\x03\x78\x7e\xe0\xf9\x7e\xdf\x53\x7b\x76\xc7\xe7\x6c\x1f\x66\x2a\xc4\x00\x66\x9a\x5b\xf3\x78\xcf\x41\x56\x1e\xdd\x36\xc6\x27\x8e\xd5\x08\x9b\x45\x70\x62\xb2\xd6\x87\x9c\x22\xbb\x1d\xd1\x65\xa2\x5d\xda\xa0\x1b\x89\x85\x56\x3a\xfb\xda\xf6\xbb\xa4\xb3\x14\x16\x3b\x74\x51\xb7\x96\x8d\x1f\xb8\x22\xf6\x84\xc2\x19\x93\x7a\xb6\xee\x4d\xb0\xee\x3b\x3e\xa3\x97\x3f\x36\xe8\xd6\x9d\x7a\xf9\x11\x32\xbe\x1d\xc9\xd6\x9a\x4b\xa3\xad\xd9\x5e\x3b\x02\x4b\x68\xe5\x3f\x6b\x48\xf7\xd9\xb3\x06\x84\x3d\xf6\x4b\xd6\xb2\x5d\xd9\x56\x9e\x30\xb6\xca\x53\x8c\x1d\x14\x2b\x70\x2b\xfb\xfa\xb9\x56\xcc\x79\xf7\x32\xd6\xda\x4d\x79\xbe\x44\xa7\x97\x16\x85\x3e\x68\x59\x61\x9f\xc7\xb6\xc7\x0a\x5b\xb5\x6b\x36\x8c\x3f\xc4\xea\x63\xe0\x52\xd9\x76\xd2\xdc\x28\xab\xc5\x36\x9c\xa5\x65\xfb\x0f\x99\xe4\x7d\x72\x5d\xa4\x72\xac\x4c\x05\x3c\xaf\xdd\xfc\x5a\x7b\xd2\xcc\x5e\xf2\xb5\x63\x8f\xa1\xd6\x6c\xca\x2f\x0e\x6e\x2e\xa0\x4e\x4c\xf7\x18\x7a\x71\xc8\xf6\x1d\x02\x31\xdd\x76\x8d\x60\x2a\x4f\xec\x23\x9e\xed\x5a\x27\xe8\x65\xc1\x74\xc4\x72\x8e\x19\x2e\xe0\x33\xe3\x7e\xd6\xe9\xe3\x3a\x3e\x62\x68\xa2\x7f\xed\x13\xee\xb5\xc8\xfd\x6d\xe4\xd9\x50\xc3\xef\xe8\xe8\xb9\x1f\x1b\xf2\x56\x89\x6e\x3f\x91\x9a\xb2\x73\xb9\xa3\x63\x95\xef\x62\x96\xdc\xd6\xa9\x45\x5c\x98\x53\xbc\x04\x5e\xb3\xa0\x8c\x73\x57\xef\xfd\xcd\x24\xfa\x0c\xa2\xe1\x46\x0c\x05\x64\x25\xa4\x1b\x7f\x48\x3c\x65\x7b\xad\xb3\xa6\xac\x28\xf6\xba\x0b\x12\xbe\x9e\xe5\xaa\xbb\xb8\x74\x8f\xf9\xb2\x4d\x54\x22\x06\x2b\x2c\x36\xb9\x61\xbf\x94\x3c\xc5\xab\x55\xd2\x61\xb0\xc3\x06\xe7\xcb\x37\xff\xca\x4e\xe0\xee\x63\x1f\x41\x2a\x03\x64\x0c\x5a\x2b\x14\x4b\xd6\x99\xc8\xb5\x92\xbc\xb3\xca\xfa\xc3\xb7\x7a\x46\x95\x62\x8d\x61\xae\xca\x66\x55\xd8\x01\x5f\xd2\xd2\x5a\xf8\x51\x9c\x3d\x94\x73\x91\x4b\x81\x95\xe4\xe1\x39\x66\x9f\xeb\x35\x5c\xc5\xcb\x62\xf5\xcd\x2c\x4a\x93\xde\xe5\x6b\x21\x5f\xf5\xc4\xbc\x76\x8a\x6f\x6d\xfb\x80\x4a\xfb\x95\xa1\x4b\x86\xbf\x31\xfc\x6d\xcc\xde\xf3\xe8\x41\xc8\x98\x65\x69\xb9\x4c\x88\xf6\x06\x8d\x8d\xa4\xea\x56\xa8\x7e\x18\x6a\x36\xd8\xbe\xb9\x04\xa7\x72\xcd\x1f\xb0\xa4\x0c\xa9\xb0\xc6\x6e\xe9\x22\x4d\x74\x8e\x9a\x59\xd2\xdc\xbb\x3b\x57\xcb\xdd\xc6\xcd\x66\xea\x7b\x4f\x97\x98\xad\xf7\xb4\x52\x84\x71\xaa\xf8\x89\x06\x1c\x5c\xb7\x5b\x1b\xec\x64\x96\x41\x46\x8b\xa8\xcc\xcd\x13\x34\x18\x3c\xbd\x10\x40\x84\xb2\x48\xa5\x64\x1c\x08\xce\xde\x68\x56\x66\x56\x88\x40\x64\x2b\x05\x9c\x11\x2e\x81\xf9\x21\x4b\xa2\x07\x44\xb6\x42\xee\x06\x73\x9f\xd7\x28\x3d\xcd\x84\x87\x58\xb6\x89\x86\x05\xd2\xfb\xec\x87\x9a\x69\x54\x55\xda\xb1\x4f\x7b\xe6\xa5\x14\x2b\x21\x67\xcf\x28\xee\xd3\x7f\xd1\x2a\x39\x28\xa4\x84\xbb\x08\xa1\x9b\xc2\x52\x26\x44\xe6\xed\x2d\x7c\x57\xb9\x22\x59\xd4\x94\xf8\x44\x33\xcd\x8b\x44\x1b\x59\xd6\x3a\xe3\x9e\x54\x69\x9f\x59\xe7\xc3\x98\x9c\x5a\x58\x9c\x6a\x73\xe1\xf2\xdc\xc6\xec\x03\xc4\x55\x02\xbb\x44\x39\x4e\xa4\x2e\x81\x55\xac\x44\x27\x39\xf0\x4b\x00\x44\xed\x17\x04\xcf\x6f\x0d\x97\xb9\x9c\xc6\x31\x3b\xf1\xf1\x6c\x64\x85\xc2\x48\xf5\x8e\x2f\x12\xa9\x16\xcf\xd9\x7c\xbd\x42\x3f\x80\xf9\x82\x0d\xc4\x40\x8f\xd3\xe6\xef\x9e\x25\xde\x0d\xf3\x09\x68\x03\xf8\x83\x90\xdb\xfc\xfb\xfd\x47\x88\x01\x98\xad\x0e\x09\x17\xd9\x51\x18\xdc\x79\xce\x00\xfb\x1f\x3b\x4f\xc4\x95\x2c\x8e\xcc\x94\x1b\x23\x28\x7a\xa0\x64\x45\x8c\xef\x11\x95\xd7\xd3\x4a\xe9\xf0\x9c\xd9\xf5\x43\x3b\x3a\x2f\x5d\xcd\x2e\x48\xf6\x74\x13\x8c\x28\x4f\xa9\x42\xa6\x2f\x18\xb5\x3b\xa4\xe8\x54\x72\xeb\xcd\xac\x08\x85\x69\x00\x5c\x84\x6d\xaa\xe5\x34\xcb\xac\x7c\xa9\x9a\x30\xbb\xe9\xb5\x9b\x33\xdc\x18\xd0\x8f\xdf\xea\x2b\xe8\xef\x25\xc8\x68\xd0\xcb\xf8\xf2\x89\x60\xcf\x0c\x81\x3b\x88\xb3\xf5\x7e\x2a\x48\x12\xa1\x8b\x32\x53\x31\xf3\xfb\xbd\x2b\xd3\x46\x4a\x85\x10\xd7\xdf\xe0\x67\x05\x83\xeb\xfd\x6d\xbb\x8e\xda\xc7\x00\x27\xc7\xe6\x65\x92\xc6\xc8\x52\x18\x68\xa8\xca\xaa\x40\x50\x1e\x09\xf4\x91\x44\xbb\x0b\xae\x65\xd3\xff\xf8\xad\xbe\x56\xf1\x3e\x1b\x6b\x38\x13\x6d\x73\x5f\xf7\x48\xa3\xd1\x21\x96\x69\xbd\x7b\x26\x32\xd5\x9d\x00\x11\xcf\x74\xb5\x1e\xf0\x96\x01\x03\xe2\x6d\x5e\x2e\x6e\xa1\xf8\x68\x17\x29\x53\x50\x97\xcf\x66\x59\x9b\x75\x36\xdd\xb8\x9c\xbf\xae\x45\x21\x00\x95\xd7\x47\x38\xfb\xfd\xed\xd5\xe5\xe1\x9a\xe7\x7a\xc5\x81\xf4\xc2\xb6\x35\xb2\xf5\xdc\xd1\x5b\x60\x81\x1d\x89\x9c\xca\x43\xb6\x54\x23\x84\x11\x1d\xb3\x55\x51\x64\xfa\xf8\xe8\x68\x99\x14\xab\x72\x3e\x8e\xd4\xfa\xc8\x4f\xcd\x11\xcf\x92\xa3\x79\xaa\xe6\x47\xb9\x80\x44\x92\xc3\x77\xe3\x6f\xde\xc1\xca\x1c\x3d\xbe\x3b\x02\xf0\xc8\x78\xa9\xfe\xe9\xe2\x9b\x7f\xfb\xdd\xbf\x9a\x86\xb3\x4d\xb1\x52\xf2\x98\x30\x4a\x5b\xdb\x3e\x44\x33\xe1\x08\x5f\xa9\xf5\xf2\x6f\xe3\xaf\xc3\x61\xd0\xa3\x6b\x15\x8b\x54\x1f\x3d\xbe\x9b\xd9\x85\x19\x67\x1d\x15\x33\xfe\x9e\x7a\xf1\x05\x52\x2f\x1e\x92\xe2\xef\xa9\x17\xbf\x6a\xea\x45\x7f\x95\xcb\xc9\x18\xe0\xc8\xf6\xf2\xd1\xfc\xdd\xc9\x48\x1b\x89\xd8\x25\x87\x5a\x2e\x87\x30\x31\x6e\x8f\x2b\x62\x60\x01\xbb\xda\xe7\x3a\x5b\xa6\xc3\xe3\x38\xb4\x4e\x4d\xa7\x75\x31\x88\x07\x04\x80\x8e\x49\x04\xbe\x42\xf4\x90\x66\x3c\x69\x4b\xa8\x08\xea\xf6\xec\x31\x85\x58\x47\xa4\x9d\xf4\xac\x4f\xb9\x2d\xaa\x38\x25\xe2\xd9\x8b\x14\xde\x6a\xed\x03\x31\xa1\x83\xdb\x6f\xe8\xdd\x3d\x54\x63\x42\x49\xef\x35\xa3\xaf\x58\x22\xe5\xa5\x6b\xa3\xd0\xe7\x3e\xb3\x2e\x4a\x8a\x6f\x5b\x4c\xb7\x7a\xb2\xf5\x50\x5e\xa2\x8a\x88\xc7\xab\xf7\xab\x20\x82\x9b\x14\xc6\x62\xc7\xd5\x31\x8c\x15\xd7\xcf\x4b\x0e\x38\x41\x0a\x62\x17\x0b\x46\x64\x75\xa2\x6d\x87\xf6\x36\xb6\xec\x4b\xe6\x72\xb7\x24\x8f\x59\x99\x67\x4a\x0b\x3d\x66\x1f\x54\x8e\xb4\x5e\xc4\xb9\xe3\x13\x1e\x6e\x3e\x9c\xb2\x77\xdf\xfe\xdb\xef\xa6\xf2\x6d\x8b\x32\x04\x97\xa8\xca\x97\x94\x7f\x01\x2a\xd0\x9a\xeb\x42\xe4\x47\xf9\x22\x3a\xc2\xab\xe3\xc8\xbc\x7f\x48\x9d\x1e\xaa\xc5\xa1\x2b\x91\x70\x48\x6c\xf1\xe3\x75\x7c\xd0\x85\x4c\x6c\x57\xb8\x7f\x35\xa3\xe7\xa4\x43\x31\x6f\x5b\xdf\xdd\x82\xb5\x72\x84\x50\x11\x21\x2d\x44\x83\xc6\x82\x54\x8c\x6a\xe1\x8a\xfa\x60\x9e\x2f\xd6\xff\x52\x8b\x96\xff\x78\x9f\xaa\xb9\x3e\x70\x04\xb0\x5c\xdb\x3e\x3c\x23\x63\x9b\xdc\x6e\x9c\xb9\x7d\xac\x6f\x9a\x8a\xd7\x74\xab\x59\x99\x18\x2e\xdb\x90\x89\x6f\x17\x1a\x5e\x17\x44\x3e\x2a\x9e\xab\x52\xda\xaa\x19\x4a\x0a\xb5\x00\x98\x13\x98\x49\x16\xa5\x09\x91\x05\xc0\xfe\x39\xee\xa9\x5c\x64\xa8\x7d\x40\x0c\xac\x7b\xba\xf7\xac\x1c\xb3\x6b\x9e\x5f\xa3\x72\xcc\xbe\xf3\x4e\x82\xf1\x57\x9a\xf0\x7d\x53\x29\xf0\x28\x0d\x41\x20\x99\xe7\x77\xa2\x0d\x9c\x1c\xf0\x95\xab\x7d\x91\x86\x8c\xe7\xa0\xc1\x8b\xc3\x42\x1d\x02\x69\x1f\x50\xc1\x61\x2d\xa7\x2e\x08\x12\xa0\x34\x86\x5c\xf7\xe6\xf9\x1e\xe3\x44\xab\xed\x73\x30\x50\x52\x58\x35\x32\xa0\x13\x24\x3d\x91\x52\xe4\x14\x01\xde\xa9\x19\x0c\xc4\x70\x84\x4b\xb9\x1d\x91\xee\xdd\x14\x61\x9d\x1d\x97\x8f\xc8\x03\x21\x30\x66\x60\x9a\xac\xd4\x5a\x19\x5d\x57\x95\x3a\xf8\x11\x4d\x5b\x50\x26\x3a\x15\xf3\x35\xcf\x50\x5f\xfd\xf5\xbe\xc6\x1c\x2d\xf3\x13\xba\xa0\xc3\x87\x06\x95\x2e\x9b\x57\x8b\x35\xed\x18\xbf\xab\xb2\xb3\x7d\xdf\x00\x42\x68\x0d\x21\xbf\x15\x7f\x14\xb6\x76\x46\xf2\x57\x63\xf4\x9a\x2d\xe5\xcc\x48\xa7\x81\x20\xa0\x0d\xb9\xa8\x43\xf8\xa6\xbd\x75\x3b\xd9\x62\xca\xf5\xc0\x35\x70\x49\x56\x7d\x16\x80\x4b\x4c\x3b\xb2\xf9\x46\x87\xad\x09\x47\x5d\xe7\x12\xfc\x6a\xa5\xb1\x4c\x2c\x5f\xfa\xb0\xa1\xde\xba\x06\x88\x1a\xbd\x39\x6e\x4f\x37\x09\xd9\x69\x38\xc7\x28\x10\xac\x6e\xd1\x05\x72\x1e\x7e\x18\xa1\x70\xdd\x90\xb9\x83\x4e\x70\x73\x36\x66\x30\x38\x0b\x5d\x13\x38\xcc\xff\xba\xcd\x9d\xd9\x86\x6f\x47\x86\x5e\x9f\xbd\x6c\x46\xd9\xf0\x2c\xb8\x17\x1f\x7d\x71\x64\x80\xff\xce\x4b\xf8\xfd\xf2\xea\x2e\x44\x36\x25\xf8\xb5\x87\xd1\x4a\x44\x0f\xe0\x4d\xc3\x2b\x0f\x0f\x03\x25\xe3\x03\xdc\xda\x97\x54\x2d\x94\x05\xca\x6c\x5c\x95\x19\x57\x69\x49\xe5\x2c\x4e\x74\x96\xf2\x0d\x40\x12\x24\xe6\x29\x7a\x38\x83\x4b\xf0\x35\xa2\x60\x57\x30\xa1\xff\x4a\x9b\x55\x39\xf1\xef\x0d\x9d\x4b\x0f\x3c\xf7\x93\xd9\x94\x07\x4c\x8b\x35\x97\x45\x12\x4d\xe5\x5a\x70\x19\x22\x58\x09\x92\x61\x26\x39\x56\x82\xea\x25\x2c\x16\x22\x2a\x3c\xe1\x32\x18\x21\x6e\xa6\x76\x9d\xc1\x61\xdf\xee\x4e\xde\xd6\x4f\xff\xc1\x96\x7d\x4e\xd6\x80\x8f\xa6\x3d\x44\x57\xe3\x33\x43\x8d\x50\x82\x97\xae\x5c\x6b\xd4\xc2\xbf\xec\x9e\x62\x73\x51\x3c\x09\xe0\x13\x22\x02\x84\x36\x1d\x7f\xef\x32\x4c\xfb\x24\x0f\x9e\x38\xfe\x41\xa2\x97\x6f\x10\x08\xd3\x01\x0b\x81\x97\x8e\xf8\x50\xd6\x18\x0c\xdf\x10\x25\x03\xb8\x02\xdf\x90\x53\xf3\x0d\x5c\xd3\xc6\x0a\xce\x1f\x45\x3c\x95\x55\x5a\x49\xd2\x19\xfd\x81\x63\xbe\xc0\xe8\xcb\x48\x1b\x3b\xc7\xbd\x02\x3d\xe7\x40\xa5\xe5\x49\xb4\x1d\xe9\xc0\x96\x82\xa7\xf8\xd1\xaf\x69\x55\xd9\x5a\xcb\x7d\x8d\x61\x5f\x83\x94\x0a\x08\x52\xbd\xe1\x0a\xfa\xc7\x6d\x4a\x47\x9a\x87\x8c\xba\x0e\x2c\x4e\x4e\xeb\x86\x1b\xbc\xad\x8d\xa9\xb4\x6c\x32\x8b\x32\x45\x96\xf4\xae\x9c\x1d\xe2\xd0\xb4\x99\xaf\xbf\x5e\x06\xb4\x73\xba\xb2\xa0\x66\xab\x03\xe9\x04\xc0\x7d\x94\x75\x76\xd7\x0b\xa9\x4b\x50\x29\x6c\xb9\x46\x88\x4a\x2c\x45\x01\xb7\x79\x5c\xa6\x08\xd8\x84\x70\x0a\xf0\x71\xf2\x34\x65\x49\xa1\xa7\xd2\xd1\x87\x62\x62\x0e\x48\x58\x1b\x6f\x89\xc9\xe4\x82\x2e\xa0\x59\xf8\x99\x4b\xd0\xc3\x92\x28\x29\x1a\xe9\x0e\x9b\xb0\xc4\x59\x96\x09\x8e\xb9\xfc\xb8\x6c\x53\x19\xda\x5c\xf5\x45\xa0\xc4\x77\xa8\x8a\xff\x12\x39\xe8\x5b\x1c\xb7\xa6\x8b\x67\xa1\x6c\xf0\xeb\x8c\xc1\x65\xab\x97\xe3\x68\x89\x3f\x88\x50\xc9\xc6\xaa\x29\xb4\x0d\xa0\x78\xbb\x15\x72\x7a\xa2\x32\xe5\x39\x26\x33\x2d\xca\x94\x25\x8b\xa0\x10\x3b\xac\x01\x92\x47\x9a\xe5\x8a\x14\xdc\xd5\x36\x84\xa2\xf9\x5a\x04\xbc\x35\xe4\xde\x49\x03\xc4\x0f\x56\xc4\x40\x28\x89\x69\xeb\x60\xcc\xce\x3c\x3d\x2e\xae\x30\x9c\x89\x80\x74\x3a\xd1\x28\xfe\xdc\x78\x03\xca\x05\xf8\x3a\x33\x44\x25\xcd\x89\x74\xa7\xae\x63\x05\xa1\x78\xcd\x30\x38\x91\x2d\x5d\xb4\x1d\x61\xdf\x4a\xb9\x62\x5e\xad\x81\x8c\xdc\x81\xe8\x18\xa0\xbd\x15\x06\x0e\x32\x24\xec\x7e\xc6\x40\x1d\x21\x7a\xcb\x60\xd7\x5b\xea\xbe\xc3\x3a\x0e\x1c\x6a\x50\x45\x71\xf8\x40\x83\x9d\x13\x82\xc7\xfa\xcc\xec\x92\x17\x43\x91\x64\x2e\x71\x6d\xf8\x40\x5b\x51\x7b\x7d\x86\x09\xd2\x63\xe0\x38\x4f\xcc\x3b\xcf\x1c\xa8\x2e\xe7\x87\x28\xa0\x5d\x3d\x24\x10\x15\x82\x47\xab\x2a\x87\x84\x65\x7a\x76\x5f\x00\x39\x84\x70\x1e\x87\xd3\x5f\x9c\xf8\x3d\x07\x85\x24\x99\x19\xfe\x98\x5d\x49\x81\x38\x4f\xb5\x08\x2e\x15\x1a\x00\x55\x9c\x84\x62\x3b\x4e\xca\xcd\xcd\xc0\xe4\x83\xa5\xd6\x32\x47\x6e\xc4\xb8\x6f\x1d\xa4\x1e\x6e\x1b\x94\x22\x1d\xba\x64\x5b\x69\xaa\x3d\xd4\xcb\x7e\x04\x15\xed\x36\x7f\x00\x97\x1e\x2e\x01\xda\xbe\xa3\xff\xb2\x6c\xcd\x7b\x70\x56\x9c\x4d\x76\xa8\xee\x1b\x86\xe0\xe7\x5d\xf3\x7b\xbd\xaa\x62\x66\x07\x14\x88\xbc\xbf\x3c\x3b\xff\x30\xb9\xac\xd6\x5f\xfc\xc3\xfd\xf9\x7d\xf5\x2f\x37\xf7\x97\x97\x93\xcb\xef\xc3\x3f\xdd\xde\x9f\x9e\x9e\x9f\x9f\x55\x9f\xfb\x70\x32\xb9\xa8\x3d\x67\xfe\x54\x7d\xe8\xe4\xfd\xd5\x4d\xad\x8e\x64\x4b\x11\xc8\xbb\xc9\xc7\xf3\xb3\xd9\xd5\x7d\xa5\x14\xe5\xd9\x7f\x5c\x9e\x7c\x9c\x9c\xce\x5a\xc6\x73\x73\x7e\x7a\xf5\xd3\xf9\xcd\x8e\x82\x91\xfe\x7b\x5b\xa7\xf4\x25\xb0\x85\xcf\xae\x2b\x7a\xc2\x16\x79\x22\x64\x9c\x6e\x30\x53\xc4\x5a\xb6\x35\xe8\x77\x78\xf7\x26\x6b\xa1\xca\x7d\x12\x3e\xee\x56\x82\xa9\x47\x91\x03\x0b\x18\xb6\x46\x94\x21\x9e\x71\xa0\xde\x6b\x2e\x8a\xbc\x19\x15\xd8\x9a\x55\x57\xe4\x1b\x97\xb7\xb9\x6d\x38\x9e\x41\x92\x3a\x61\x99\xc8\xb7\x8d\x05\x34\xa3\xbc\xcc\x8a\x64\xde\x9d\xc2\x33\x38\xf1\xbe\xaf\xed\x8d\x7c\xc7\xed\xe4\x70\x97\xed\x82\xb1\x92\xc9\xb2\x0f\x4c\x1e\x5a\x78\x6e\xb9\x5c\xf7\xb6\x85\x16\x67\xe5\x3c\x4d\x22\x96\xc4\x75\x7f\x0a\xf1\x61\x80\xcb\xb8\x4e\x8b\x9e\x89\x1c\x54\x55\x63\x01\x64\xb9\x38\xe4\x65\xb1\x42\x0a\x4f\x4a\x9c\xa1\x22\x36\x53\xa9\x45\x94\x0b\x8c\x05\x08\x0d\x4e\x5a\x2c\x87\x1a\xf4\x04\x83\x21\x06\x9b\x18\xc8\xf2\xc6\x41\x89\x9a\x8e\x18\x01\xbe\x89\xad\x0f\x70\x92\xe2\xf3\x5b\xa7\x86\x46\x9c\x60\xc1\xd5\x00\x16\x06\x37\x3c\xfe\x68\x8b\xaa\x9a\xef\x36\x92\xda\x15\x15\xc5\x45\xb6\x99\x46\xed\x9f\xb1\x6b\x8f\x85\x1b\xa5\x9a\x7a\x43\xad\xd3\x4f\xa7\xb9\x80\x4b\x84\x20\x0d\xd6\x7f\x01\xb8\x26\xca\x4c\x82\x84\x24\x63\xaa\xcd\xc5\x8a\xa7\x0b\xd4\x38\xcc\xd2\xb4\xb3\x8a\x60\xfb\x77\xea\x41\xc8\x1b\x5c\xb0\x5f\x45\x1c\x4a\xb4\x7c\x3c\xa7\x91\xf3\x08\x79\x17\xa6\x19\xa3\xdd\x55\x36\x2f\x14\x94\xa9\x02\xed\x84\xe0\x67\x4c\x40\xf2\x15\x0b\x6c\x4a\xe9\x62\x91\x7c\x36\x0d\x4e\xa5\x68\xe5\x6c\x07\x30\x99\x65\x97\x74\x72\x19\x80\x73\x48\xd1\xf7\x20\x24\xd4\x53\x05\x72\xc0\xdd\x7b\x76\x98\xff\xbc\xb9\x16\x5b\x1c\xfa\xe0\xf3\x4b\x2a\x65\x66\xc3\x28\x8f\x9d\xa7\x02\x33\xc2\x1c\x07\x07\xec\x9b\xd3\x8b\xc9\xf9\xe5\xdd\xec\xf4\xe6\xfc\xec\xfc\xf2\x6e\x72\x72\x71\xdb\xf7\xf8\xbd\x44\x16\x5f\xed\xf4\xd5\x93\xd9\x9c\x84\x38\xa2\x93\xe7\x53\xd9\xdd\x47\xf9\x63\x07\x4b\xb2\x7b\xf4\x49\x9c\xcd\xe2\x44\x47\xe6\xfa\xdb\xcc\x84\x8c\xa1\xd8\xc5\xb3\xb6\x6a\x7b\x53\xf5\xaf\x70\x4f\x30\xf7\x84\x95\x20\x78\xdb\x3d\xda\x1d\xed\x7e\x07\x48\x26\xb8\x21\x73\x61\x0e\x7f\x5c\xe1\x18\x19\xef\xae\x70\x66\x9a\xdb\xef\xdb\xaa\x4d\xd4\xbf\x09\xc7\x9b\x68\x5d\x02\x95\x89\x7d\x0c\xf0\xa8\x1d\xb3\x42\x0c\xc4\x61\xc5\x8d\x24\xa8\x42\xcf\x12\x3d\x95\x6b\x2e\x63\x5e\xa8\x7c\xd3\xf1\x89\xfd\x84\x67\x78\x6c\xaa\x22\x34\xbc\xb2\xa5\x10\xb1\x5d\x05\x7c\x94\xcb\xfa\x56\xc2\xba\x1c\x77\x57\x3f\x9e\x5f\xde\xce\xce\x2f\x7f\x9a\x5d\xdf\x9c\x7f\x98\xfc\xd1\xc1\x64\x33\xae\xdb\x8a\x4b\x67\xb9\x30\xd2\xc5\xd2\x9c\xb5\xca\x17\x2c\xd9\x6c\xdb\xa1\x32\x9d\xc9\x62\x2a\xad\x64\xc9\x7d\xf3\xab\x5c\x95\xcb\x55\x7b\x43\xf5\x51\x5e\x9f\xdc\xfd\xf0\xac\x61\x02\x09\x25\xd6\x75\xc5\xd3\xd6\x84\x0b\x27\x0b\x92\x7b\x88\x31\xae\x0d\x0f\xa8\x54\xe1\xd1\xb6\x28\x43\x87\x44\x7b\x96\xf5\xd2\x14\x5a\x5b\x95\xff\x96\xc7\xbb\x36\xd0\x5d\x20\x37\x2b\xd7\x08\xc0\xd7\xb1\x38\x78\xa3\xb5\xe3\x96\xbf\x55\x6e\xb0\x6f\x0e\x53\xb1\x5c\x8a\x18\xb7\x57\xbd\x61\xf2\xc1\x91\x08\x8c\xfc\xbd\xde\x36\x8b\x54\xc0\x77\x8f\x8b\xd9\xe1\xbd\xfa\x0b\xf0\x6b\xf7\x4a\xbb\xac\x38\x25\x22\x29\x88\x6f\x16\x5c\x76\x04\x92\x77\xe7\x83\xb5\x37\x7f\x95\x33\x97\xaa\x47\x0e\x13\x1b\x32\xf0\xe7\xa0\x0b\xf0\xb2\x3f\xbe\xd5\x8d\xe3\x46\x64\x29\x8f\x84\x4b\x70\x41\x06\x60\xb0\xeb\x9f\x13\xc0\xa3\x32\xc9\x92\xfc\x2d\x41\xf9\x64\x5f\x19\xae\x6d\x0b\x80\xe7\xf6\xc6\xca\xe3\xd7\x77\xad\x6c\x35\xdc\x88\xf7\x13\x1c\xcd\x58\xa7\x92\xf2\x22\xd0\x17\x05\xc5\x5f\x3b\x31\xeb\x83\xb6\x43\xad\xe7\x9f\x68\xe1\xd1\x66\xae\x3a\xba\xb9\x65\xd6\x75\xdb\xc3\xa9\x8e\xdb\xfc\x85\x45\x91\x6f\x25\xe3\x7e\x89\x70\xc4\x75\xae\xd6\x89\x16\x27\x45\x91\x27\xf3\x32\xac\x46\x3c\x10\x30\x57\x31\x4e\xfc\x07\x67\xb9\x8a\xcb\xc8\xd2\x67\xc1\xd7\x7a\xd8\x0f\x79\xf9\xac\xd6\x11\xb3\x43\xb3\xfb\xc8\x72\x13\xf1\x21\x64\x7b\x20\xbf\x5b\x5b\x8c\xcd\x0a\xc6\x0e\xdf\xdf\xb5\xbd\xca\x5f\x38\x67\xb4\x7b\x32\xed\x1e\xe8\x97\x06\xce\xec\xe3\xa0\x01\x77\xa0\xa6\x68\xbb\xcc\x39\x06\xd0\xab\x3a\x4a\x17\x5b\x8e\xbb\x6a\x86\x81\xbb\xfa\x61\x63\xaa\xe9\x54\xa8\x37\xac\xb8\x46\x75\xbe\x88\x56\xd5\x81\xc3\xd7\x54\x59\x83\xeb\xc3\x75\xea\xf1\x7e\x6e\x93\x5e\x61\xb4\x11\x3a\x1a\x12\x72\x6c\x57\x2a\xc0\xba\x72\xd6\x9d\xfe\x7b\x4c\xb9\x98\xfd\x52\x8a\x21\x55\x9d\x6d\xaa\xc6\x1f\xe0\xb5\x9d\x80\x94\x04\xb1\x5b\xce\xf7\x5a\x24\x6b\xa3\x01\xf1\x3c\x5a\xb1\x39\xd7\x44\x47\x18\xb2\x25\x60\xf9\x79\x96\x98\xb7\x78\x54\x50\x39\x5e\xdb\xad\x2d\xc9\x7b\x67\xa1\x90\x46\xad\xf5\x5e\x8f\xb6\xed\xb6\x6b\x02\x86\x78\xaf\xed\x30\x26\x67\x83\x62\x08\xa1\x1e\xee\xec\x64\xbc\x62\xe1\x76\x4a\x79\x29\xa3\x15\xcb\x52\x8e\x84\x12\x2b\xae\x51\x50\x58\x84\x0e\x9f\x27\x69\x52\x00\x4f\x18\x06\x8e\x6b\xfb\xd6\x18\xcf\x3c\x7f\xb0\xe5\x16\xb8\x27\x85\xdb\x26\x4a\xf6\x44\x42\xbb\xaf\xfa\xa2\x58\x68\x2f\x08\x43\xe1\xde\xef\xb0\x13\x0e\xda\x2f\x87\xb9\xde\xe0\xb0\xfb\x6f\x19\x16\x1d\xa2\x16\xaf\xeb\xaf\xd7\xe6\x1b\x09\x28\xf7\x22\x09\xde\x9a\x97\xf5\x2a\xa0\x73\x9f\x55\xb6\xfd\x1a\x6d\x7e\x70\x8b\x12\x3c\x1c\xf8\x44\x85\x93\x06\x28\x31\xf5\xb2\x4a\xad\xe7\x7e\x91\x2a\x5e\x6c\xcf\x72\xc3\x2a\x49\x5d\x6d\xc7\xaa\x9c\x77\xd5\xe5\xc0\x51\x3d\x3f\x87\xce\x8a\xff\x97\xf2\xb9\x87\xf7\x28\x2f\x84\x91\xbe\xcf\x9b\x50\xf3\xf6\x21\xbc\xde\xde\x38\x65\x31\x0f\x66\xa4\x70\xdb\xc0\xd7\xea\x73\xba\x3f\x40\x52\x5b\x8e\x53\x5d\xc9\xdb\x2b\xe7\x71\xbf\xf5\x4a\xe4\x8e\xad\xb4\xbb\xfc\xd7\xef\xbe\xe9\x93\x8d\xf8\x87\x92\x9b\x0b\xe0\x6a\x71\x8b\x04\x61\xfb\x7c\x74\x91\x34\x8f\x55\xbb\x18\xa8\xf7\x7a\x57\x8d\xd2\x86\x1b\xbf\x37\xdb\x41\xdb\xd7\xdc\x9a\xb7\xfb\x8b\xdd\x49\xc5\x1b\x9b\xe5\x89\x02\xa2\x2c\xb5\xa8\xe8\x1a\x2d\x92\xb8\xb5\xdf\x3d\x66\xf2\x97\x52\x94\xc2\x6c\xa0\x79\x19\x2f\x9b\xc1\x92\x01\x06\x97\xff\xa4\x95\x7a\x62\xeb\x32\x5a\x31\xdb\x38\x8b\x45\xca\x37\x55\x35\xca\xd8\x1a\x85\x02\x0a\xe5\x41\x7c\x81\x01\xf1\x7d\x54\xea\x42\xad\x01\xa7\xee\xdb\xcd\x4b\x09\xa7\x9c\x71\x7b\xba\xda\x2e\xb4\x0a\xa1\xe7\x33\x23\xe4\xb7\xd7\xe7\xa7\x93\x0f\x93\x5a\x78\xfa\xe4\xf6\xc7\xf0\xdf\x3f\x5f\xdd\xfc\xf8\xe1\xe2\xea\xe7\xf0\x6f\x17\x27\xf7\x97\xa7\x3f\xcc\xae\x2f\x4e\x2e\x2b\x41\xec\x93\xbb\x93\xdb\xf3\xbb\x1d\x71\xea\x66\xaf\xdd\x0b\xc1\x03\xbe\x51\x8b\x9c\xb7\xc5\x74\xac\xbb\x8a\x7a\x3d\x66\x27\x96\x7d\xb5\xc2\x0f\x6c\xb1\x06\x00\x4e\x4a\x11\x63\x89\x90\x84\x33\x5e\xf0\x53\x5e\xf0\x54\x2d\xc7\xec\x84\x51\x5e\x01\xe6\x8b\x68\xa3\x12\x12\x35\xa5\x59\x1d\x6c\xc2\xe8\x85\x91\x77\x05\xf9\x6a\xe1\x6a\x41\xa4\xb0\xa9\x08\xeb\x4a\xd9\x24\xcf\xa9\x3c\x7f\x14\xb2\x28\x41\xd1\xe6\x69\xca\xa8\x5b\xfb\x40\xc0\x0a\x62\x47\xa9\x93\x75\x92\xf2\xdc\x17\x76\xbe\xa2\xb6\xc0\xd8\xb5\x63\x75\xac\x74\x4d\xca\x09\xeb\x0f\xb8\x9f\x30\x18\xf7\xe9\xc5\x04\x14\xdd\xa8\xb0\x55\x0b\x6d\xe7\x53\x89\xa4\xa3\xd4\xe3\x9a\x43\x0e\x53\xa1\xc8\x41\x8f\xdd\xd3\xc3\xdd\x1b\x71\x2f\xc5\xca\x86\xb2\x5e\xcb\x31\xe1\x06\x69\xff\xe3\x5c\x16\xf9\xa6\xb7\xf6\x7a\x07\x8c\x0e\x1a\xec\x3a\x82\x44\x56\x8b\x3d\xa3\xff\x94\xd9\xd6\x2f\x41\xa5\xb5\x78\x5d\x0a\xef\xb9\x28\x1e\xc2\xa3\x3a\x4c\xa2\xd4\xdc\xbc\xbf\xd5\x79\x08\x59\xc0\x60\x16\xe6\xaa\x94\xb1\x26\xf0\xe6\x3a\x91\x47\x6b\xfe\xf9\xc0\x7e\x29\x92\xd8\xb8\x92\x6b\xc0\x98\x28\x52\x63\x0f\x6e\x8c\x90\xdb\x3e\x5d\x53\xb9\x65\xbe\x76\xdb\x04\x56\xb2\x82\xcb\xc0\xfb\x77\x10\x86\xfa\x28\x36\x6d\xeb\xd7\x28\x9b\xc9\xc2\xda\x0f\xd0\x48\x96\x0b\xf3\xa0\xc3\xb8\xa6\x08\x5d\x76\xff\x86\x5c\x96\x4a\x69\xef\x76\xd9\x1d\xc2\x46\xf6\x3a\x36\xad\x80\x95\xfe\x8a\x4f\xef\xba\xa7\xd4\x93\x59\x33\x84\xaf\xd8\xc8\x09\xe5\xee\x50\x5c\xde\x2c\xd6\x5f\xd4\x9c\x2d\x20\x91\x8d\xfc\x04\xb9\x80\x48\x19\x2c\x85\x2d\xd4\x03\xbc\x7a\x0d\x4c\x8c\xdd\x02\xa9\xd0\x10\x3f\x92\xc6\xa8\x16\xbf\x94\x04\x01\x78\xf7\xf5\xb0\x7b\xb6\xc0\x6a\x0f\x48\xef\x5d\xaf\x83\xe0\xee\x72\x18\x57\x29\x93\x36\xb2\xcd\x9b\x52\x9a\xab\xf8\x25\xd0\x53\xfd\xc3\xe3\xb5\x4e\xe9\x9f\x3b\x73\xcd\x6c\x64\x27\xc7\xe7\x5f\x8d\xb9\xf9\xa7\x1a\x61\x33\x75\x07\x99\x0d\xd4\x7a\x78\xa1\xcd\x79\xf4\xf0\xc4\xf3\x18\xdd\xff\x00\x67\x1a\xb3\x1f\xd4\x93\x78\x14\xf9\x88\x45\x22\x2f\x38\xf1\x15\x6a\xc0\x73\xc0\x81\xa2\x76\xa6\x12\x12\x7d\x90\xfc\x51\xea\x32\x17\xac\x48\x96\xab\x42\xe4\x21\x1a\x47\xe5\x46\x1c\x15\x48\x55\x9b\x89\x88\x08\xd9\x3a\x26\x60\x91\xf2\xc7\x26\x01\xe3\x73\x98\x64\xd8\xc4\x65\x2b\xdb\x70\xb7\x2d\x7e\xb6\x0d\x3f\x45\x13\x46\x42\x13\x29\xb4\x46\x6c\xa9\x52\x2e\x97\xe3\xf1\x18\x0a\x7d\x1c\x0c\xda\xe8\xd4\x60\x18\x40\x77\x28\xfd\x54\x29\x2d\xd2\x8d\x23\x11\x73\x79\x54\x00\xdc\xfd\x5c\x08\xa9\x13\x74\x6c\xb5\x6c\xff\xdb\x7a\x70\xe9\xcb\xc6\xe2\xda\xcd\xf3\xc1\x59\xba\x1d\xed\x40\x2d\xd5\x01\x2d\xe1\xf3\xed\x96\xd7\xb3\xb2\xce\xdb\xdb\x92\x4a\x0e\x4d\xa5\xfe\x49\x25\x1d\x50\x90\x67\x91\x8d\xb6\xb6\x44\x44\x48\xcf\x4a\x3f\x6d\x9f\xb3\x46\x46\xf0\x1e\xc9\xc0\x5b\xf2\x7a\x07\xa6\xf4\xf6\x71\x04\xdc\xd6\x97\x7b\xf0\xb1\xd8\x5d\xde\xad\xf5\x83\x06\xa6\x4c\x7b\x6e\x83\x21\xaa\x13\x66\x5d\xa6\x1b\xb0\xb8\x5c\x02\x35\x84\x07\xe2\x20\xaa\x54\x09\x9a\x41\x2a\x9f\x8f\xba\x39\x82\xba\x20\xc8\xa6\x0b\x95\xf3\xa5\x60\x6b\x11\x27\xe5\xba\x55\xd8\xb8\xe1\xee\x03\x1f\x55\x69\xb9\xee\xa6\x0a\xdd\x57\x81\xf6\x83\xc4\xff\x3a\x85\xee\xfa\x73\xe8\xb8\xcc\x08\x5b\x65\x93\xc6\x8b\x21\x24\x9a\x6b\x73\x53\xe6\x89\x06\x96\xdd\xe7\x64\xce\xba\x66\xb0\x69\x08\xc0\x6f\x32\x74\xb2\x57\x56\xf7\xd0\x46\x46\xe9\x15\x8d\xab\x0a\x51\xfb\xee\x4b\xa1\x0e\x4a\x1d\x5e\x6b\x2f\x57\x65\x83\x7b\xaa\x17\x50\x02\xd4\xc6\xa0\xf2\x06\xa1\xe6\xa0\x41\x82\xf6\x14\x8a\x2d\x6c\x2e\xe6\x83\x08\xa8\x0f\x63\xa8\xc9\xf1\x84\x94\x4f\x3f\x7e\xab\x2d\x08\x88\x70\x5a\x5e\x63\x29\x7c\x27\x18\x01\x7a\x7c\x67\xe1\x79\xf8\x85\xd8\x04\x10\x14\xc6\x5c\x16\xad\x0d\x78\xf4\x2a\xb4\x85\xaf\xfc\xc4\xcb\xb4\xfd\x71\x6a\x1f\x1e\xc5\x9a\xad\x27\x3f\xdf\x32\x9c\x6a\xaa\x9f\x90\x6f\x1b\x68\xd0\xc8\x6e\x80\x20\x4c\xd7\xec\x19\x9a\x60\x65\x1d\x70\xd2\x6d\xf9\x0e\x33\xed\xa2\x88\x56\x5e\xf3\x00\x82\x46\x47\x2c\x49\x05\xb9\xe9\x3b\xd7\xbe\x22\x04\x62\xaf\x43\x10\x6b\xb2\x94\x2a\x2c\xa5\xa4\xa4\x80\x50\x9c\x11\x40\x2a\x6c\x96\x25\xc5\x6e\xa4\xe0\x40\x56\xc2\x5d\x5b\xad\x50\x88\x00\xa3\xef\xac\xc4\xa9\xc1\xa4\x48\x90\xae\xca\xc2\xac\xd1\x26\xa2\xfa\xce\xf5\x4a\x01\x55\x02\x90\xa9\xac\x76\xd5\x98\x24\x0b\xe5\x4b\x72\x81\x04\xdf\xda\x68\x6f\x45\xf2\x68\x0e\x6a\x73\x5b\xbb\x0d\x0a\x12\xa0\xb9\xf7\x28\x6c\xcb\x02\x96\xf0\x07\xb1\xd1\x61\x31\x69\xda\x51\xac\x6b\x43\x26\xe6\x7b\x68\xbd\x76\x2f\x05\x4c\xdc\x2c\xf7\x25\x21\xfb\xdd\x65\xd8\xe9\x47\xf3\xf2\x16\x8c\x70\xa3\x71\xb3\x07\x7d\xb2\xab\xf7\x29\x92\x98\xf0\xf3\x4c\x6b\xe8\x61\x80\x00\xf2\x0c\x61\x9c\x61\xe6\x12\x18\xbe\xc6\xbe\x9d\x4a\x2a\x24\x10\x5c\x72\x46\xe0\x34\x97\x8d\x32\xf0\x91\xbe\x7c\x53\x61\x0f\x02\x6a\x55\x4b\x33\x5b\xed\xd2\x46\x97\xa1\x2e\x1f\x6c\x0f\xe8\x1a\x73\x94\xad\x0f\xaf\xb5\xc3\x67\x62\x4b\x69\x71\x3b\xf1\xa4\x41\x22\x20\x3e\x49\xec\xa2\x58\x95\x1c\xad\x9f\x48\x98\xe9\x3b\x91\xad\x50\x4e\x0b\xe4\xbc\x3d\x3f\xbd\x39\xbf\xfb\x62\x78\x53\x0b\xf6\x1c\x0c\x38\xb5\xe3\x3c\x3b\xff\x70\x72\x7f\x71\x37\x3b\x9b\xdc\xbc\x06\xe2\x94\x7e\x7a\x06\xe4\xf4\x96\xea\x93\x9c\x2a\x59\x88\xcf\x7b\xdd\xc9\x79\x29\x67\x7c\x40\xea\x93\xab\x50\xb4\x4d\xdd\xc1\x46\x9b\xf5\x55\x5c\xf1\x13\xe2\xb6\x25\xd4\x89\x2d\xa7\xb2\xf0\x4e\xc3\x45\x92\xa6\x90\x09\xee\xdc\xeb\x94\x65\x68\x26\x15\xe4\x8f\xa5\xf3\x25\x99\x3a\x95\xf3\x4a\xf9\x1b\x70\xf9\xad\x8c\x11\x8c\x39\xe0\x99\x99\x80\x3c\x81\x0c\xdb\x6d\x25\x58\x96\x89\x14\x7e\x18\xb0\x6a\x66\x7c\x9d\x34\xf5\xb4\x88\xaf\x89\xac\x23\xc5\xab\xaf\xae\x69\x77\x5c\x65\x7f\x5a\xf5\xd3\xfe\xe8\xbe\x10\x0f\x71\x22\x51\x31\xad\x9c\xe6\xdb\xf6\xad\x7b\xe4\x8f\x00\xcc\xbb\x59\x49\x0e\x31\x08\x5d\xf0\xbc\xf0\x0b\x49\x0b\x81\x85\xe1\x7c\x70\xe2\x21\x41\x04\x9a\x5a\xd4\xe6\xd9\x88\x42\x33\xd7\x09\x44\x2a\x38\x91\xdb\x44\x69\xa9\x0b\x91\x93\xdb\xe4\xe4\xe7\xdb\xa9\x7c\x6f\xae\xaf\x03\xba\x85\xa8\x7c\x17\x76\x81\x48\x1d\x55\xe9\xdf\x6a\x28\xa1\x04\x7b\x8b\x3e\xea\xb5\xe0\x52\x33\x38\x1a\x69\x2a\x72\xbf\x33\x70\x3c\x42\xc4\x54\x44\x1b\xa8\x9e\xfd\xfb\x07\x8c\xc0\xad\x66\x2a\xcc\x78\x5d\x19\xb1\xb5\x2a\x9a\xfb\xa9\x8b\x68\x00\x10\xe7\xaf\xb9\x73\x5a\x12\x9f\xfa\xee\x22\x02\xeb\xb7\x6e\xa2\x6a\x1a\x52\xaf\xbd\x74\x87\xcd\xfd\x7d\x2b\xbd\xe0\x56\xea\x71\xaf\x87\xb7\x04\x5b\x29\x23\x40\x5d\x6d\x2b\x1f\x66\x76\x44\x27\x29\xa0\xdc\xcc\x34\xb6\xde\x3a\xb5\xea\xb2\xfb\x60\x3f\xa0\xa9\xfd\x10\xda\x27\x2d\x8c\x4a\xbe\x8c\xa1\x8d\xed\x6c\x2d\x5c\xfb\x3a\xcc\x85\x27\x16\xab\x2a\x55\x61\x39\x48\x1c\x3c\x94\xb0\xae\xe6\x01\x47\x7e\xb3\x75\x8c\x44\x28\x63\xb5\x94\xd9\x9e\xc5\x1f\xef\x42\x4c\x6d\x25\x2b\x1b\x47\x11\xf2\x39\x58\x0e\x07\xc7\x01\x33\x64\xf3\x3d\xbf\xbc\x70\x75\xcf\x39\x3e\xd1\x67\x81\x1d\x2e\xaf\x2e\xcf\x43\xa8\xc2\xe4\xf2\xee\xfc\xfb\xf3\x9b\x4a\x3e\xff\xc5\xd5\x49\x25\x27\xff\xf6\xee\xa6\x96\x8a\xff\xfe\xea\xea\xe2\xbc\x81\x79\x38\xbf\x9b\x7c\xac\x34\x7e\x76\x7f\x73\x72\x37\xb9\xaa\x3c\xf7\x7e\x72\x79\x72\xf3\x1f\xe1\x5f\xce\x6f\x6e\xae\x6e\x6a\xfd\xdd\x9f\x6e\x47\x4f\x54\x3e\xa3\xdd\xfd\xe3\x83\xb3\x01\xb5\x6a\xeb\x31\xae\x96\x5f\xde\xe3\x14\xf7\x44\x9e\xed\xda\x8e\x36\x5d\x3f\x0e\xcb\x71\xe0\xc1\x30\x43\x1d\xb4\xeb\x5e\xbe\x5e\x74\x65\xea\x32\xbe\x9f\xd8\x33\xb7\xda\xec\x25\x90\x80\x5b\x15\x40\xd7\x4b\xcd\x71\x4b\xe5\xd9\x71\x6a\x33\x88\x60\x2d\x79\x67\xbd\x32\x19\xbf\xfa\x48\x6d\x1f\xbb\xc6\xe9\xa9\xbc\x76\x30\x22\xbd\x14\x1b\xca\xb6\x41\x07\x9d\x59\xb2\x81\x24\xb6\x8a\x82\xfd\x31\x84\xdd\x9b\xcf\x30\x3b\x27\xd8\x8e\x5d\x85\x7d\xdb\xd3\x96\xb6\xb3\xef\x0d\x1d\x3f\x75\xd2\x1c\x7b\x8d\xaa\x65\xc0\xb8\x81\x32\x6b\xc8\xb8\xef\xb8\x7e\x18\x3a\x6e\xea\xa4\x39\x6e\x50\xfb\x9e\x35\x6e\x70\x78\x17\xed\x34\x3a\x03\x84\x58\xd8\x4c\x75\x78\x2e\xc7\xdf\x3d\x12\xd4\xcf\xee\x37\x46\x73\x00\x5e\xd7\xbc\xcc\x78\xff\x40\x06\x8c\xc6\x1d\x57\x5e\x63\x95\xbf\x85\x5f\xe1\x0b\xe7\xb9\xe0\x0f\xb1\x7a\xa2\xf5\xa8\x23\x43\x59\x2f\x69\x5e\x9d\x20\x23\xc3\xed\x15\x51\xe4\x14\x81\x42\x94\x9a\x6f\x1e\x60\x72\x09\xf1\xa2\xa3\x0e\x16\x94\x5e\xae\x13\x11\x01\xf5\x93\xf4\xab\x33\x95\xa8\xcd\xb7\x95\x6f\x36\xab\x6a\x46\x44\xd4\x21\xf0\xa9\x4e\x87\xc6\xe0\xba\x0e\x16\x96\xf2\x80\xca\x1c\xc0\x74\xf3\x1c\x6c\x26\x98\x90\x44\x82\x33\x39\x37\x06\x4f\x2e\xa2\x44\x8b\xa0\x62\x5c\xeb\x8d\xfd\xcb\x7e\xa5\x50\x0a\x5e\xb4\xba\x5d\x7b\xfb\xc3\x79\x54\x94\x3c\x65\x90\xae\x44\x0c\x8c\xe8\xab\xc4\xbf\x44\x5c\x62\x6a\x4c\x21\xd6\x19\x64\xf5\x87\x39\x1d\x53\xf9\x33\x00\x25\x70\x09\xde\x68\xf6\x3d\x40\x1e\xec\xc3\x74\x09\xaf\x79\x01\x77\xf1\x1f\xb0\x0f\xf7\xdb\x78\x2a\x2b\x15\x98\x82\xb7\x2a\xc5\x98\xc6\x53\x69\xab\x75\xc4\x2a\xd2\x63\xb0\xf8\xc6\x2a\x5f\x1e\x51\x2d\x75\xb3\xd9\xd5\xc3\x5c\xa9\x87\x23\x21\x8f\xc0\x27\x55\x1c\xf1\xb2\x50\x47\x00\x97\xc2\xf5\xd7\x47\xb6\xe8\xb1\xad\x1a\xad\x8f\x56\xc9\xa3\x80\xff\x37\x5e\x15\xeb\xf4\x9f\x74\xb6\xfa\x7c\xb8\x4c\xf3\x43\xf3\xee\x61\xf8\xee\xa1\x7d\xf7\xd0\xbe\x7b\x68\x5e\xc3\xff\x97\x6d\x30\xbc\x23\x3e\x73\x73\x97\x8d\xa6\x32\x91\x5a\xe4\x05\x68\x3f\x4f\x79\x52\xf8\x52\x57\x1b\xf6\xe6\x3f\xff\x93\x8d\x73\xfe\x84\x19\xb1\x67\xbc\xe0\xd7\xe8\x5f\xfc\xdb\xdf\xde\x40\x40\x15\xb3\x98\x32\x9e\xff\x52\x8a\x62\x2a\xb5\x30\x87\x90\xfd\xaf\xa9\x84\x08\xec\x7a\x33\x2b\xd0\xef\x8a\x3e\xc8\x58\xb3\xef\xb0\xcd\x09\xb2\x91\xc6\xda\xb4\xd4\x91\x4e\x90\xf0\xb4\xa5\x4a\x7f\x87\x8b\xfe\x97\xf4\x8c\x9e\x1f\x70\xac\x7f\x49\xab\xa7\xda\x16\x5b\xd2\xbf\xa4\x70\x81\xa6\x8a\x5b\xb0\x16\x73\x9b\x17\xec\x64\x1a\x5c\xdb\x19\x69\x40\x03\x5e\x35\x4c\xdf\x7e\x56\x6e\x91\x11\xdd\x7a\xee\x1b\x62\x04\x62\x05\x3e\x0e\x01\xd1\xf3\xc4\x9c\x90\x5b\xf4\x84\x82\xe6\x86\x5f\x0e\x3a\x29\x85\xce\x5d\x7b\xe8\xb8\xd0\xbf\x3b\x3e\x3a\x1a\xb1\xa5\x86\xff\x99\xff\x02\xff\x03\xe8\xa1\x97\x22\xf5\x6d\x4c\xa6\x03\xc2\x35\x57\x79\xf7\x4a\xbc\x04\x8a\xee\x4b\xf0\xc8\xd7\xb6\xe9\xfb\x52\xc6\xa9\xf0\xa9\x8d\x95\x90\x48\xaa\xcc\x4a\xda\x85\x6a\x56\x1e\x82\x35\x9e\x8b\x88\x1b\xc1\xd7\xe8\x1b\xc1\xa5\x6a\x51\x08\x89\xde\xb0\xdc\x57\x7b\xe4\xe8\xb9\x02\xb5\x18\xa0\x90\xbc\x20\xc8\xb9\x80\x3f\x42\x27\x40\xcc\x3e\xaa\xff\xc4\x36\xaa\x24\x8e\x71\x60\xce\x8d\x45\x94\x42\x21\x07\xcb\x1e\xc4\x72\x51\x94\xb9\x64\x9c\x65\x5c\xc6\x5c\xc3\x0e\x5c\xe4\x10\xed\xcc\x19\x6f\x0e\x74\x84\x70\x5c\x55\x16\xc0\x89\x85\xc8\x82\x70\x26\x90\x04\x3e\x18\xf3\x28\x18\x04\xde\x09\xc0\x45\xdd\x78\x71\x3c\x95\xb6\x1e\x21\x61\xe1\xd0\x53\x16\xa9\x6c\x43\x8c\x47\xf5\x49\x4f\xac\xe7\x8c\xa6\x7b\xe4\xf1\x26\xf5\x67\x47\x2c\xa9\x86\xd6\x80\x6f\xbe\x08\x4a\xbc\xdb\x22\xf9\x6f\x85\x8c\x54\x2c\x72\x7d\x60\x8e\x61\xe2\xec\x0e\xd4\x1f\x12\xed\x17\x03\xa4\x94\xb9\xdc\xc8\x5b\x68\x9a\x77\x05\xa6\xcc\xec\x54\x18\xca\xdb\xf4\x9c\xdd\x47\xe5\xb7\x8e\x82\x69\x1b\x2f\xfd\xe7\x17\x45\xc4\x84\xb8\x4e\x6b\x73\x3e\xdf\x05\x81\x47\x36\x94\xb8\xd8\x28\xea\x38\xa4\x9c\xd8\x7a\xda\x49\x01\x15\x32\x73\xa1\x8b\xa9\xa4\x1b\x78\xc4\x16\x82\x1b\x3d\x6f\xc4\x22\xfd\x88\xc2\x18\xaf\xfb\xe2\x49\x79\x0c\x8e\x2d\x6f\x03\x60\xd8\x4a\xe3\xde\x49\x8c\x8f\x71\xca\xc0\x46\x80\x41\x97\x85\xee\x54\x15\x98\xac\x56\x81\xf8\x8c\x79\xb0\xd5\x52\xea\x15\xd6\xc2\x62\x3d\x30\x13\x1b\x0c\x14\xb3\xfa\x38\xf0\x07\x23\x78\xf0\xeb\x10\x06\x12\x08\x47\xd0\xb8\x09\x4b\x8b\xe7\xcc\xc7\x70\x43\xca\x7a\xf0\xcd\x74\x1d\xaa\x2d\x13\x01\x03\x78\x9e\xdf\xc2\xbc\xba\xd3\x61\xa5\x45\x6e\x4b\xb9\xe0\xb7\x22\xc1\xe4\x2a\xc9\xe3\xc3\x8c\xe7\xc5\xc6\x6e\xdf\x34\x99\x43\x05\x88\x34\x79\x10\xec\x24\xcf\xd5\xd3\x4b\xcf\x42\xa7\x68\xe9\xb2\xb0\xf7\x41\xb2\x0f\xb5\xf2\x5b\xe9\x65\xeb\xee\x8e\xe7\x51\xd9\x76\x39\x3e\x5a\xfb\xc9\x45\x91\x6f\x66\x66\x23\xae\xb3\x4e\x49\xd1\x2b\x69\xa2\xbf\x92\x3b\x8c\x25\xb7\xe6\xc2\xe8\x64\xc9\xad\xac\xea\x6f\x87\x25\xb7\x85\x00\xb7\xc9\x92\x3b\xb9\x9c\xdc\x4d\x4e\x2e\x26\xff\xa7\xd6\xe2\xcf\x27\x93\xbb\xc9\xe5\xf7\xb3\x0f\x57\x37\xb3\x9b\xf3\xdb\xab\xfb\x9b\xd3\xf3\xed\xb4\x57\xcd\xd1\x7b\x15\xfc\x90\x85\xfd\x1c\xb3\xbb\x00\xa8\x81\xc9\x06\xa4\x7f\x53\x7d\x5c\xd8\x55\xe6\x30\x27\x72\x39\x82\x83\x7a\xcc\xce\xf3\x7c\xb2\xe6\x4b\x71\x5d\xa6\x29\xc0\xa9\x30\xb3\xe7\x34\x17\x60\x78\x8e\xd8\xb5\x8a\x27\xc1\x7b\x90\x8e\xd8\xfa\x19\xd0\x3f\x8f\xe3\x5c\x68\x8d\xdd\x8f\xa8\xff\x00\x3c\xe4\x52\x1d\x09\x3c\xc7\x1f\x79\x92\x1a\xfb\xed\x98\xbd\xe7\xd1\x83\x5a\x2c\x30\x7d\x66\xe4\x12\xa7\xd8\x2f\xa5\x2a\x38\x13\x9f\x23\xa0\x7a\x6b\xdf\x27\x17\x6a\xf9\x2b\x40\x95\x7b\x84\xa7\x3a\x8c\x14\x28\x75\x37\x6b\xbf\xce\xdb\x05\x01\x7d\xe5\x47\x7c\xf5\x03\xbe\xd9\xee\xa0\x2c\xd2\x17\x48\x8f\xbf\x50\xcb\xf6\xc2\x43\xa0\x5d\x53\xb5\x24\x0a\x24\x44\xc4\x2e\xa2\x96\x4c\x27\xf2\x61\x2a\x7f\x5e\x09\xc9\x54\x99\xe3\x9f\xc0\xcc\x37\x6a\x66\x5a\xea\x95\x80\x32\xd5\x23\xf6\x24\xd8\x9a\x6f\x50\x6d\x06\x9b\xc0\x55\x4b\x81\x2d\x03\xb7\x88\x79\x3b\x4d\xa4\x91\x16\x59\x62\xf3\x12\xea\x4b\xff\x12\x16\x97\x25\x3a\xe4\xfb\xf3\x10\x6f\xbb\x4f\x2b\xf8\x3c\x70\x95\x79\xdc\xa4\x05\x08\x91\xe4\x86\xa2\xb2\x4a\x3d\x94\x99\xa7\x44\x7d\x63\x83\x93\x30\xdd\x8f\x2a\x89\x59\x5c\x66\x69\x12\x39\xb9\xfb\xa4\xf2\x4e\xde\x67\x4c\xa0\xe9\x7f\xeb\xd4\xd3\xc2\xb6\x7d\x58\x4b\x76\x4e\x80\xa4\xdb\xc2\x00\xfd\xca\x1c\xd8\x2c\x91\x51\x5a\x42\x99\xb9\x52\x8b\xfc\xd0\x95\x8e\x76\xb9\x7e\xbf\x7d\x92\x6c\x4f\xc2\xb9\x7f\x5a\x5b\x98\x74\x9e\xaa\x65\x12\xf1\x34\x04\x37\x7b\x54\x84\x63\xe1\xb5\xc7\x9e\x8a\x09\x43\x1e\x84\x1d\x50\x27\x91\x56\x96\x0b\x20\x82\x9e\x81\x28\x9f\x91\xb8\xdb\x67\xdc\x0b\x66\x0c\x74\x1c\x57\xc8\x91\x6b\xc3\x0b\xf6\x86\xf3\x7d\xdb\x4a\x6c\xa0\x62\x62\x09\x7f\xa6\x9e\xa4\xc8\x41\x83\x05\xd8\x87\xf9\x52\xa9\x40\x37\x71\xd5\xd9\x1c\x3e\xd9\x56\x27\x5c\x38\x20\x36\x66\xce\x2e\x93\x47\x21\xbf\x3c\xa9\x79\xd0\x41\xc4\xa3\x95\x98\x59\xbd\xfc\xa5\x45\x96\xbb\x00\x06\x0a\x2b\x5b\x26\x25\x14\xa5\x2e\xbc\x09\xa6\x13\x8e\xb8\x29\xbb\x30\x90\xb8\x25\x23\xcb\x0c\x62\x16\x8b\xe8\xe1\x8b\x8b\x66\x0f\xb2\xb2\x03\x61\x9c\x9d\x89\xe8\x81\xdd\xdf\x4c\x30\x1b\x38\x29\x98\x11\x05\x7a\xe5\xcb\x3e\x75\xda\x6e\x05\x5f\xbe\x02\x85\x55\xdf\xba\x55\xbe\x54\x81\xab\xd6\x67\x06\x44\x80\x28\xc8\x97\x34\x42\x92\x72\x69\x00\x08\xc6\x0b\x5b\xcd\x08\x1c\xf1\x4c\xaf\xa1\x78\x51\x59\x04\x15\xff\x52\x3e\x17\x69\x07\x71\x67\xa6\xe2\x99\x8d\x93\xec\x0b\xe6\x69\xb4\x65\xfd\x18\x14\x75\xb4\x79\x0c\xdc\x68\xac\x77\xf4\x20\x7b\xf8\x56\x07\xf4\x1a\x2a\xe4\x0f\x07\xbb\x9e\x6b\x48\xef\x5e\x24\x4b\x1b\x6d\x4b\x16\x54\x62\x09\x13\xfa\x8d\x1e\x0c\xf2\xd2\xb4\x74\xad\x62\x82\xe9\x39\x2e\x3c\xa3\x05\x09\xf2\x9e\x78\x5c\x45\x38\x04\x8b\x03\x84\x7e\xcd\x89\x10\x3c\x66\x6a\x41\xde\xc4\x2c\x4b\x13\x60\x86\x8e\x91\x84\x1e\xd8\x33\x74\x15\x1d\x1f\xb6\x66\x07\x1b\x90\x7c\x5c\x5b\x20\x5e\x57\x31\x5e\x10\x18\x98\xc1\x30\x03\x36\xb8\xd9\x23\xef\x26\x53\x7b\xf5\x8a\x69\x1d\xe3\x71\xd1\xe4\x2a\x25\x6c\x85\xb4\x8f\x7c\x05\x78\xad\xdb\x84\xfc\x88\xa7\x51\x49\x71\x32\x28\x97\x6f\xab\xe0\x6f\x47\x10\xfa\xa8\x9f\x59\xe8\xaa\xd7\xbf\xae\x64\xee\x5b\x5d\xd1\x25\x68\x3d\xd7\xa7\xb0\xdd\xbd\xb8\x4c\xd5\x1c\x76\x4e\x37\x4a\x70\xcb\x8d\x65\xc4\x75\x9e\xc4\x43\xf4\x1d\x3b\x27\x57\xee\xd5\x6d\x03\xbc\xb2\xae\x1f\xd7\x93\xdd\xf7\x8c\x0a\x19\xd4\x98\x1b\x87\x51\x20\x2c\xa8\xaa\x6a\xd5\x3c\x29\xa8\x8c\x07\x6c\x2b\x77\x3f\x75\xf8\x19\xaa\xdf\xb2\xd7\x42\x37\x99\x62\x76\xcc\xa5\x27\x97\xd9\xbe\xc8\x7b\xd0\x7d\xa0\x28\x73\x9c\x1f\xdd\x9e\x45\x19\x8b\x78\xf6\x8c\x6f\x38\xa7\x77\xfb\x7d\x8b\x9b\x69\x1c\x1e\xf8\x00\xe5\xa1\x51\x15\x62\x9e\xc7\xfe\x3b\x46\x70\xde\x23\x9e\x81\x1b\x1e\xc2\x1a\x8f\xef\xc6\xb6\x8f\x1b\x9f\x5d\x64\xe4\x25\xe6\xfc\x23\x7e\x5b\xb5\xd4\xc0\xd9\xb5\x8f\xdc\x26\x45\x78\xb7\xd9\x39\x7e\xbb\x56\xf2\x6e\x7a\xed\xdd\xfa\x0e\xb3\x02\x7c\x9f\xcd\xf5\x1a\xb2\xa3\x2c\x94\x8f\xf6\xc0\xf7\x4c\x80\x76\x38\xcc\xe8\x03\x01\x39\x89\x3b\x90\x22\x56\xfd\xb6\x42\x68\x00\xfe\x78\x10\x02\x3a\xcb\x85\x8d\x1b\x6e\x44\xe1\x78\x1d\x52\x5b\x57\x10\xc2\x62\xee\xab\xab\xc4\x36\x96\xbb\xc2\x91\x91\x41\x10\x8b\x54\xfd\x48\xad\x33\x25\x01\x96\x84\x59\x6a\x53\x49\x8d\xdb\xea\xf0\x2e\xb2\x56\x49\x75\x1c\x91\x43\x13\x13\x67\x84\x56\xe9\x23\x85\x50\x83\x22\x26\x50\x57\xd2\x0c\xf0\xd4\xd8\x86\x2a\x47\x82\x2d\x7b\xb3\x43\x26\x40\xad\x44\x7a\x2e\x96\x89\x2e\x44\x98\x1d\x1a\xbe\xff\x62\xd5\x6c\x2b\xce\x93\x6d\x53\xdf\x59\xcd\x76\x97\x15\x64\xe4\xd3\x80\xf1\x6c\x32\x11\x4f\xdc\x7b\xdb\x37\x43\x2d\x81\xdf\x8b\xc3\xca\x7d\x87\x7b\x00\xad\x3f\x8d\x54\x5f\xda\x95\x1f\x71\x8b\x44\x24\x4c\xdc\x03\x1a\xcd\x12\x2d\x4b\x9e\x73\x59\x08\xa1\xa7\x92\x02\xcf\x48\x59\x17\xb2\xb2\xd4\x80\x90\xce\xb6\x89\x94\x2e\x90\x01\x0a\x5e\x59\xf0\x24\x2d\xf3\x4e\x77\x03\xee\xca\x67\xd1\x4e\x6c\x9b\xa5\x53\x68\x96\xb5\x2d\x9a\x4b\x60\x0e\x4e\x91\x63\x4d\xa9\x87\x8d\xab\xf9\xbd\x1d\x9f\x60\x2f\x97\xfe\xeb\xed\x7c\xcd\x1d\x39\xcd\xdf\xea\x59\xa6\x06\x48\xbc\x1f\xbf\xd5\xd7\xaa\x23\x1b\x5c\xff\xd2\xf0\x89\x6e\x81\x4f\xfc\xd2\x55\x90\x85\xeb\x07\x88\x3c\xee\x72\xc5\xf4\x62\xe3\xdc\x19\x9f\xec\x94\x5d\xb0\x6b\x57\x5c\xc6\xa9\x51\x79\x79\x51\xe7\xbd\x76\x38\x6f\x63\x12\x15\x56\x38\x76\x27\xf5\x41\x8e\xcc\x2c\x6a\x24\x58\xee\x9a\xa7\x5a\x66\xe6\x56\x2c\x65\xad\x97\x6a\xbe\x64\x5b\x9e\x8e\xd7\x61\xa8\x0c\xb2\x3b\xb0\xbf\xba\xfe\x72\x1e\x8e\xfd\x0b\xa9\x2f\xd5\xb3\xb6\x48\x96\xbf\x01\x47\xc2\xc7\xe6\x95\x10\x91\xcc\xa1\x8b\xda\x65\x37\xec\x29\x75\x20\x91\xcc\x48\xed\x90\x71\x7c\x2a\xa9\x1c\x3c\xa2\x0b\x20\xac\x8c\x7c\x6b\x9a\xbd\x73\xd9\xc5\xef\xfe\xc5\xb2\x6d\x6d\xd8\x02\x36\x15\x50\xda\xa9\x28\x2a\x73\x08\xfd\x93\x7b\x92\x09\xbc\x84\xf5\x20\x22\x19\x50\x3d\x1c\x60\x0b\xf5\xc4\x36\x35\xc9\xf9\xa3\x2b\x1f\x75\x07\x6e\x48\x2c\x6c\xef\x2e\x7d\xaa\x57\x96\xeb\x82\xe9\x42\x64\xad\xe2\xb7\xa2\x5d\x6e\x32\x71\x22\xa5\x2a\xea\xf9\x29\x83\xf5\x4b\xee\x5a\xe9\x79\x74\x06\x5c\x46\x27\x81\xcb\xe8\xf7\xb7\x57\x97\x2c\xe3\x1b\xc0\x3e\x16\x8a\xe1\xa3\x40\x38\x5a\x17\x54\xbb\x56\xa0\xfa\xf1\x55\xa9\x82\x73\x6a\x41\xd4\xed\xf1\x09\xea\xb1\xa9\x2c\xc2\x9e\xa1\x2d\x69\x64\x56\xae\xd2\xc3\x2c\xe5\x32\x80\xb7\xeb\x31\xab\x75\x1f\xe2\x19\x5c\x64\x93\x10\x63\x30\x00\xf0\x57\xd0\x5e\xc8\xcb\x56\x00\x34\xf0\xee\xd8\x0d\xb5\x1f\x84\xa1\x53\x46\x6c\x05\x76\x7e\xc4\x2a\x30\x58\x13\x01\xd9\x33\x2c\x2c\xc3\x21\x7b\xb8\x06\xd0\x6d\x27\x03\x38\x8f\x52\xae\xf5\x56\x94\xce\xab\x50\xc9\x07\x59\x8b\xbb\xc5\x57\x75\x9c\x08\x23\x04\x6e\x13\xb4\x4b\xdd\xcf\xc0\x96\x60\x45\x97\x2f\xfa\x16\xe8\xfb\x41\x35\x08\x82\x3e\x10\x5f\x14\xbc\x8f\x4c\x90\x0f\x62\x63\x3d\x5c\x24\xaa\xf8\x5a\x8c\x9c\xb3\xd5\x79\x13\x03\xd0\x5f\xb3\xe1\xa9\x04\x54\xec\x87\x70\x78\xec\x83\x52\x23\xc4\x67\x52\xe7\x1c\x9b\xe5\x21\xc2\x69\x2a\x3f\x28\x35\xe6\xce\x88\xa5\xf1\x93\xb8\xa9\x77\x48\xa8\x28\xc0\x1c\xd6\x96\xb3\xff\xd9\xfc\x21\x91\x58\x9e\x30\x59\x1b\x03\x8a\xe6\x09\x76\x14\x0c\xc8\x56\xc3\x57\x4f\x9a\xc5\x48\x29\x53\x26\x7a\x05\x61\x17\x8c\x73\x42\xff\x74\xa5\x20\x20\x2b\xe7\x52\x9b\x33\x0c\xa1\x1a\xf1\x28\xc8\x5f\x5b\xc1\x18\x4c\xce\x2e\x1c\x6c\x09\xcf\x25\x95\xee\xe8\x38\x6d\x81\xd1\xb1\x8f\x71\x0e\x70\xf3\x01\x84\x76\xe4\xe0\xfc\xc8\xb3\x6d\xc9\xb0\x7b\xb7\xb8\x6b\x95\x1c\xa1\x56\xdd\xa2\x82\x4a\xe6\x50\xc3\xb0\x92\x11\x1b\xce\xde\xbd\xdc\xf3\xc6\x69\xe5\xb4\xdf\x5d\x72\xa7\xb7\x83\x61\xa0\xa8\xd8\x7d\xdd\x04\xdc\x96\x0e\x32\xe8\x6c\x41\x23\xd8\xa1\x42\x1c\x90\xf2\xe1\x91\x1e\xb3\x5b\x21\xd8\x27\x98\x29\xd3\xd9\x27\xaa\x40\x0a\x28\xe8\x82\x27\xad\x05\xe2\xe0\xe9\x89\x5c\xa8\xfd\xe4\x7f\xbe\x6c\xa0\x6c\xf7\x9a\x95\xf6\x71\xee\x8b\xe3\x05\x4f\xbf\x7c\x5d\x5a\x91\x5e\x17\x43\x6d\xad\xaf\xbd\xbf\x89\x92\x8d\xed\x48\x8d\x4a\x06\x4b\xfc\x1c\xe2\xba\xda\x26\x31\x5f\x39\x42\x32\xf6\x07\xa9\x9e\x24\xca\x63\xea\x89\xbd\x35\xe7\x0f\x74\x16\x8c\x0b\xa1\x26\x58\xa2\x34\x3c\x00\x76\xf8\x13\xf7\x6f\x76\x8b\x21\x70\x1c\x33\x94\x0e\xd3\xa0\xef\x52\xd1\x2f\xb8\xc0\xdf\x9e\x8c\xd8\xfb\x11\x3b\x1d\xb1\xf1\x78\x7c\x30\x62\x82\x47\x2b\x3b\x22\x7c\x05\x45\x7f\xc1\x97\xa6\x6d\x2a\xfb\xb3\x08\x3a\x80\xf2\x80\x46\x3f\xb1\x24\x88\xdc\x3f\x15\x78\xd5\xec\x27\x60\x6a\x36\xe5\x91\x11\x5c\x28\x5a\xa9\xc4\x0f\x0a\x90\xe7\x22\x52\xb9\xc5\xae\xeb\x42\xe5\x16\x87\xfb\xc8\x73\x9e\x48\x60\xac\xe0\xcd\x2c\x04\xea\x39\xe0\xac\x17\x9f\xf9\x1a\xbe\x3f\x91\x8e\xb6\xd7\x4c\xd3\x9d\x1b\x7f\xb1\xc9\x28\xce\xf6\x94\x27\x45\x61\x14\x32\x3d\x95\xb7\xec\xf8\x3b\x76\x92\x65\xa9\x60\x27\xec\xbf\xd8\x7b\x2e\xb9\xe4\xec\x3d\xfb\x2f\x76\xca\x65\xc1\x53\x55\x66\x82\x9d\xb2\xff\x32\xd3\x66\xda\xbb\x54\x46\x03\xda\x8c\x18\x67\xb2\x4c\x51\xd1\x7b\x6b\x31\xae\x07\xee\xbb\xb8\x5f\x9d\xb9\x28\x9e\x84\x90\x4c\xab\x35\x5d\x85\x7f\x74\xb7\xbf\x4e\xe4\x32\x15\x05\xed\x87\x2a\x1a\x19\x3b\x38\x84\x2f\x3d\x9e\x4a\xe7\xa7\xfe\xa3\x19\xf1\x1f\xd9\x7f\xb1\xcb\x32\x4d\xcd\x90\x8c\xa0\x31\x1b\xe9\x98\xd9\xec\x30\x21\xc7\x4f\xc9\x43\x92\x89\x38\xe1\x90\x1f\x66\xfe\x75\x74\x07\xab\x3d\x2b\x3d\x15\x68\x78\xa6\x5d\x39\xb6\x7d\x44\xcf\xab\x70\x4d\xb8\x62\x81\xa1\xb6\xd2\x09\x42\x09\x5f\x1d\xae\x04\x7b\x02\x64\x3a\x0f\x64\xa3\x60\x29\xbd\x30\x40\xd9\xde\xbf\xab\xfa\x95\x99\xff\x6a\xa5\xff\xe8\x55\xfd\x6b\xdb\x7c\xf8\x31\x82\x72\x8a\x8b\xe3\x93\x33\xc1\x90\x81\x5c\x42\x3c\x77\x1b\x28\xf9\x61\xdb\x46\xb3\x13\xc3\xdb\xc6\x94\x46\x6d\xb4\xe0\xcb\x11\xcb\x5c\x1d\x29\x7b\xa8\x5c\x60\x1b\xcf\x31\xd6\x4c\x20\x65\xf3\xad\x05\x10\x99\xbd\x4c\xf9\x87\x47\xb1\x5a\xf3\x44\x1e\x40\x1f\x96\x3a\x6f\xc7\x44\xb5\x98\x2b\xbb\x67\xe8\x8e\x6f\x45\x33\x76\x53\xfb\x57\x95\x9d\x5a\x09\xb7\xb6\xe3\xb0\x67\x0d\x33\x5f\xe1\xf4\x0b\x9a\x43\x3f\x35\xb6\x68\xef\xda\x07\x54\x6f\xac\xc2\x9e\x02\xba\xbc\x67\x90\xeb\x15\x5b\x77\x95\xcb\x7e\xaa\x96\x78\xad\x4c\xb1\x4a\x7a\xd5\xc2\xad\x0d\xf6\x9e\x2c\x31\xcc\x7b\x36\x62\x32\x49\x8f\x8c\xa8\x3c\xba\x54\x52\x30\xae\x75\xb2\x44\xd6\x3b\x70\xa8\x61\x11\x59\xab\x94\xdd\x55\x4d\x86\x40\x04\x81\x7e\x66\x86\x84\x88\xe9\xc2\x48\x61\xb3\x04\xe9\x66\x2a\xcd\x1b\xa4\x11\x40\xf6\x54\xe2\xc8\xd1\xb1\x37\xe2\x1e\xb7\x7d\xd1\x85\x18\x34\xde\xb2\xc1\xb6\x51\x33\xec\xb1\xe1\xe8\x24\xee\x11\x71\xbb\x0c\x88\x41\xa9\x35\xcb\x1a\x85\x70\x9a\xb9\x48\x95\x5c\x9a\x5d\xd1\x25\x84\x41\x0a\xbc\xd0\x10\xb0\xb1\xce\x11\x18\x65\x85\x1e\xa1\x25\x31\x7a\x4a\x12\x7b\x97\x9a\x2e\xe7\x46\x8f\x73\xd1\x1e\xa7\x8d\xd0\xc7\x75\xf1\x54\xec\x07\x5b\xba\x37\x32\x58\xe5\x16\x38\xe7\x22\x89\xa8\xb8\x78\x0e\x27\xfc\xa2\x2e\x84\x46\x3e\xc8\x95\xd3\x11\xd9\x1e\x31\x95\x23\x7d\xa7\x8d\x60\x3b\xfe\xad\x66\xef\xdd\x47\x7a\x6b\xb6\x47\xbb\xef\x93\xe2\xa4\x0d\xbe\x90\x1e\xa7\xe1\xd7\x4c\xfc\xe8\x93\x0e\xf2\xe1\x64\x72\x51\x7b\xae\x99\x0e\xd2\x92\x33\x72\x37\xf9\x78\x7e\x36\xbb\xba\xbf\x6b\x3c\x67\x5a\xa3\x3f\xed\xc8\x08\xe9\x9c\xbd\x97\xc0\xc4\xff\x82\x35\xcc\x66\x6a\x61\xe9\x01\xfa\x5f\xcf\x8d\x2a\x72\xfd\xa0\x97\x45\x60\x5d\x87\xd5\xd6\x9a\x1b\xa7\x93\xe4\x44\xce\x28\xd6\xda\x6f\xb0\xf5\x09\xbb\x92\x1f\xf0\xf5\x6b\x95\x26\xd1\x76\x24\xb7\xbd\x2c\x8d\x56\xd5\x84\xc6\xce\x05\xa4\x36\x90\xc3\x97\x06\x85\xf6\x59\x21\xa2\xc2\x63\x09\x9a\x1f\xf7\xff\x34\x7a\x74\xb7\x07\x06\xfd\xb0\x6e\xda\xa0\x38\xb9\x43\x27\xc0\xcd\x0e\xac\xd1\x50\x2c\x05\xb5\x5c\xf0\xec\x82\xcc\x8b\x38\xc5\x9c\x2a\x33\x0f\xd7\xc3\xd3\x4a\xa5\xe4\x8f\x45\x06\xee\xa9\xcc\x44\x1e\x29\x40\x5d\x22\xb9\x8b\x62\xd1\x2a\x49\x63\x5f\x91\xec\x2d\xa4\xa9\x00\x98\xfc\x80\x8a\xeb\x0a\x87\x9e\xb1\xcd\x6f\xb9\xf3\xed\xb6\x3b\xc3\xd3\xbd\x17\xf2\xec\x25\x71\xe7\xdb\xb6\xfd\xcf\x84\x8f\xc6\xa9\x20\xce\xbc\x1a\x0e\x02\xd4\xfe\x70\x3c\x83\x42\x3a\xe6\xb2\xa7\x62\x53\x91\x37\x9b\x8b\xda\xba\xd2\x36\xab\x4f\x25\x30\xa9\xa3\x1f\x1d\x41\x80\x5a\xc0\x70\xd6\x82\xa3\x26\xe8\x79\x8d\x69\x51\xa7\xd2\x23\x3f\xde\xe8\x50\x2b\x6c\x5d\x67\xf4\xbe\x5b\x64\xfb\x88\xbd\xa9\x7c\xe8\x1b\x60\xda\x96\x0a\xfa\xa3\xe8\x7c\x65\x6a\x60\xbb\x8e\x58\x52\x4c\xa5\xb1\xd9\xcc\xce\xcc\x45\x2a\x1e\xcd\xe8\xc2\xe8\x10\xe1\x15\xad\xe7\xc4\x7e\x36\x24\x47\x71\xcb\xa9\x41\xdb\x86\x0e\x61\x1e\x32\x36\x63\x58\x3a\x16\xda\x68\xad\x50\x6b\x4a\x7c\x36\x07\x20\x81\xe0\x27\x02\xdb\x62\x21\xed\xf8\x00\xef\x06\x63\x1b\x4f\xe5\x64\x01\xc4\x06\x40\xa7\x10\xc7\xe8\x83\xb0\xd5\x87\x1c\x7d\x66\x42\xd1\x20\x45\x1e\x19\xbb\x10\x54\x1b\x1a\x4f\x92\x78\x14\xf9\xa6\x00\x97\x3e\xcc\xab\x14\xbc\x58\xb1\xa4\x18\x01\xef\xa9\x95\x94\x53\xc9\xe3\x98\xf2\xc1\xb1\xb9\xc0\x9c\xed\x5c\x67\xfa\x7d\xae\x1e\xb7\xa9\xd5\xfb\x22\x77\xf1\x54\x67\x29\x97\x33\xbc\x41\x7e\x05\xec\x6e\x50\xb6\xbb\x0b\xc4\x51\xce\x67\x8e\xab\xed\x45\xc6\xe9\xe4\xfd\x8d\x85\x2e\x93\x69\x53\xce\x6d\x47\xa3\x0a\x34\x7b\xee\x69\x3d\x9c\x97\x8e\x70\x53\x39\xb3\xd8\x92\xfe\x52\xc0\xc3\x7a\x79\x0d\x63\x65\x77\xeb\x2e\x5c\xaf\xdd\x01\xbf\x55\xe4\x65\x9f\x95\xaf\xdd\x21\xf5\x65\x1f\x0e\xfa\x6b\x68\x88\xcf\x02\xfe\xed\x18\xd6\xeb\x82\xff\x3a\xbd\x38\x4d\x10\xa0\xfd\xda\x20\xbe\x4f\xa9\x03\xe8\x85\x75\x0e\xb4\xf6\xb2\xec\xa1\x15\xa8\x5a\xf0\xf1\x2f\x19\xa1\x00\x39\xd5\xd7\x4f\xe3\x09\x45\x60\x5c\x63\x36\x91\xcc\xaa\x7b\x23\xf6\x06\x37\x96\x7e\x43\x0e\x68\xaa\xed\x4f\x60\x99\x98\x4e\x0f\x51\x30\xd4\x41\x66\x98\x08\xe7\x8f\x1b\xc6\x01\xb7\xf2\xf5\xbe\xea\xbc\xbc\x4f\x20\x11\xef\x39\x5c\x2b\x18\x43\x9e\x63\x03\x36\x8f\x24\x70\x85\xd2\xe7\x42\x2c\xc3\x7f\xb0\x8d\x76\xb2\xf7\xf6\x45\x33\x45\x59\x49\xf7\xa9\xfd\x9d\xa9\x7c\x2a\x6d\x6b\xe4\x90\xd6\x58\x20\xb0\xde\x54\x90\x17\x44\x3a\x7f\xb0\x53\x01\x0a\x60\x6b\x42\x42\xa9\x51\x4f\x2a\x5e\x97\x02\x80\x82\x9a\x3b\x04\x2a\x54\xa1\xf0\xbd\x19\xc5\xc3\x6c\xf0\x35\x5e\xf3\x75\xe2\xe1\x34\x35\x93\x92\x14\x96\xe7\x38\xc8\xd9\xd3\x25\xb0\x75\x2f\x4a\x23\x8c\x02\x4a\xf3\xa9\x34\x93\xc7\x16\x09\xe4\x6e\xd0\xbc\x4c\xe5\x47\xa5\x2d\x45\x8c\xf6\xf3\x61\x81\x05\x34\x6d\x6f\x5c\x69\x4c\xfa\xc3\x19\x5c\xda\x14\xf1\x41\xb2\x37\x77\xb5\x40\xb2\x26\xf1\x3c\x6d\x54\x99\xfb\x8f\x8a\xb8\x9c\xca\xbf\x98\xe9\x01\x73\x8a\x4b\xbb\xac\x6a\x81\x47\x18\x56\x10\x42\x65\x9f\xb0\xd1\xb7\xff\x72\xf0\xe9\x00\x93\xab\x4a\x0d\xd5\x88\x47\xd5\x0b\xc4\x55\xb7\x28\xd3\x14\x70\x08\xf6\x0b\x1c\xc3\x92\xef\x62\x2b\x0e\x8f\x8c\xba\x99\xac\xaa\x18\x7d\x0e\x7a\x3f\xb7\xfe\x09\x8b\x78\x11\xad\x0e\xad\x2e\x47\x62\xcc\xde\x7e\xb4\x7c\x98\x05\x65\x34\x2d\xd6\x5a\xe0\xc1\x18\x9c\xf9\xda\x51\xce\x56\xf6\x8b\xf9\x04\x70\xff\xdf\xd5\xab\x9d\x39\x46\x6c\xdc\x9c\x88\x03\xaa\xea\x79\xee\x71\x5b\x6b\xd4\x5b\x9c\x14\x23\x91\x7c\x2d\x62\xf6\x06\xd2\x80\xdf\xd8\xc5\x9f\xca\x6c\x3e\x4e\x37\x8b\x82\x78\x0b\xcd\xa4\x8c\xa1\x2a\xdf\x8e\x5b\x6e\x16\x37\xcd\xa4\x1d\x93\xdd\x69\x68\xb5\xeb\x3a\x6e\x6e\x5c\x4f\xfd\x15\x16\xf4\x71\xb9\xd9\xb9\xad\x62\x02\xab\xe5\x41\xb8\x7e\x18\xb1\x79\xce\x25\x14\x54\x8a\x43\xa5\xca\x9f\x4e\x30\x9e\x91\x14\xd0\xe6\x05\x4a\x9e\x6e\x20\xff\x67\x34\x95\xc8\xa0\x08\x54\xfb\x9b\x28\x4d\x22\xb6\xcc\x79\xb6\xaa\xe9\x41\xe2\x51\xc8\x02\xea\x72\xdf\x08\xae\xf7\xc3\x6a\xe4\xf5\x16\x58\xef\x68\xda\x89\x04\xeb\x83\xcb\x1a\xe7\x35\x0c\xaf\xe3\x6a\x01\x7c\xa6\x88\x67\xc3\xf8\xae\x76\xb2\x32\x57\xb8\x3e\x89\x78\x0e\xe2\xcf\xe6\xe3\x98\xed\x75\x17\xf8\x01\xe7\x95\xa8\x98\x2c\xa2\x77\x5f\xc0\x86\xa3\x76\xda\x8b\xa0\x77\x52\xd5\x22\xb9\xe7\xac\xf2\x7e\x73\x0a\xfb\xa1\xa7\xc2\xa6\x41\x38\xc1\x31\xa2\xba\xa9\x40\xde\xc9\xfe\x50\xce\x55\x6a\xd9\x4f\x27\x67\x4c\xe5\x50\x78\xa8\x50\xf4\xa7\x24\xee\xd2\x0e\x12\x19\x8b\xcf\x7b\x51\x10\x6d\xbf\xe8\xad\xda\x6c\xba\x09\xea\xdb\xd4\x3f\x16\xa4\x53\x2e\xcc\x25\x5c\x58\xcb\xb8\xf1\x94\xae\x43\x95\x4f\xd2\x62\x05\xf8\x61\x4c\xd1\xf1\x93\xba\xe6\x1b\x16\xad\xb8\x5c\x06\xae\x09\x80\x73\x8a\x4c\xe5\x58\xa0\xf7\x11\xb8\x3e\x55\x6e\x29\x1e\x88\xb8\x80\xf2\x84\x5c\x18\x03\xe1\xf9\xca\xb2\x13\xf0\xe5\x32\x17\x4b\x48\x63\x9d\xca\x0a\xf5\x0a\xf0\x9c\xda\xda\x40\xd8\xcf\x36\xe6\x8a\x97\xa1\x7f\xea\xb2\x06\x8b\x7c\xe3\xf2\xfe\xa9\xba\xb5\x3f\xcf\xf5\x69\x1d\xb1\x44\x8c\x47\xec\x1b\x9f\x92\x20\x22\x25\x1d\x71\x40\x47\xd6\x78\xcd\xe5\xcf\x76\x98\x0e\x4d\x9e\xa8\xf6\xb1\xc3\x6f\x8d\x1a\xd9\xad\x9b\x66\x2b\xf3\x42\xc1\x8b\x72\xc0\x1d\x74\xca\x0b\x9e\xaa\xe5\xa9\x79\xf9\x16\xdf\xdd\xb6\xaf\x4f\x31\x5f\xc0\x72\xf4\x99\xe7\xcd\xcd\x69\xfa\xf6\x1c\xfe\x6d\x73\xbd\xd3\x81\x9c\xaa\x6e\x07\xf2\x4b\xa8\xea\x96\x88\x69\xb7\x0f\x39\xed\x20\x17\xda\xf2\x4d\x43\x5d\xc4\x16\xd5\x4f\x89\x49\xba\x6e\xc6\xb6\x48\x80\x2c\x57\x71\x19\x89\xd8\x9c\x5c\xb0\x87\x10\x0f\xe5\x38\x8e\x2a\x42\xb2\xed\xa2\xad\x10\xb5\xc1\xad\xfb\xa5\x7c\x0e\xbd\xb8\xf1\xdd\xf4\xdf\x77\xf8\x1b\xac\xc6\xd7\x36\xe9\xe1\xf9\xc4\x79\xca\x07\xde\x53\xae\xfb\x2a\xa3\xbd\xca\x93\x65\x22\x79\xa1\x72\xf6\xd6\x31\x19\x1c\xb8\x32\x78\xdd\x1a\xc2\x40\x31\x51\x99\x22\x14\x13\x5f\x54\xf1\x68\xdb\xa4\xe6\x29\x5d\xf0\x75\x16\x72\x44\x83\x17\x38\x98\x99\x14\x27\xc1\xe9\x26\xe0\x3b\x4d\xb4\xcf\xda\x9d\x4a\x8a\x38\xe0\xba\xa9\x3c\x2c\x72\xd0\x79\x37\x67\x65\x31\x7b\x26\xef\x19\xbe\x3c\xcc\xf1\x44\x20\x88\x8f\x3c\xdb\xce\x24\xc5\xc9\xe5\x80\x69\x8b\xe4\x8e\xf0\x9a\x4a\x75\x7f\x6e\x2f\x23\x34\x90\xd9\xba\x1e\xb8\xbf\xb9\xb0\x81\x22\x6f\x0f\x56\x0c\x2c\x58\x08\xa4\xd4\xc5\x34\x30\x34\xed\x9d\x58\x33\xb7\xb8\xa5\x9f\x3a\x4d\x55\x19\x33\x12\x6a\x04\x02\xc8\xc7\x78\x3b\x02\xc7\xf5\x78\xdc\x95\xd6\x36\xb0\xbc\xb9\x93\x3f\xf0\x5e\xfb\x09\x84\xdf\x3a\x24\xf0\xd6\xa3\x4f\x33\xfb\x6a\x4b\x4f\x33\x0d\x6b\xef\xc4\xf1\xa0\xb5\x77\x5e\x70\x20\xdc\x1c\xe6\x20\x05\x7b\x34\x89\x53\x38\x6f\x61\x00\xa1\x85\x12\xbc\x12\x98\xd5\x0f\x7b\x77\x67\x59\x28\xb6\x77\x95\xf1\x5c\xc8\x62\x06\x3d\x0e\xeb\x0c\x3a\xb9\x86\xd7\x2b\x0a\x53\x2f\x47\xf0\x9f\xee\x14\xfa\xf7\x2d\xbb\xd6\x9f\xd9\x2d\xf9\xb4\x8c\xbc\x4a\x00\x42\xac\x1f\xd8\xdb\x04\x10\x4f\x41\x2c\xd4\x2d\x5c\xc7\x72\xd1\x07\x3d\x63\xf6\x82\x0f\xaa\x88\xf6\x5e\x1f\xe4\x47\x0f\xa1\x6a\x68\x85\xdc\x7b\xc4\x09\x60\x44\xad\xfd\x5b\x50\x71\xe3\xb2\xf2\x6f\x60\x47\x36\xeb\x97\xb2\xbf\x8a\x5c\xf9\xfc\x2f\x74\x56\x85\x0d\x6f\xd5\xd7\x9f\x5f\x2c\x1c\xf5\x71\x2c\x53\x1d\xd6\x69\x85\xbf\x10\x81\x19\x7a\x14\xe6\x1b\x6b\x8e\x74\x84\x90\x32\x11\xcd\x3a\x8a\xf2\xf4\x1a\x4a\x60\x78\x86\x45\x76\x92\xda\x65\x66\x0f\xe8\x11\xf8\x2b\x28\xb1\x6a\xcd\x33\x42\x17\x12\x90\xbc\x1e\xbc\x19\xc3\x47\xfc\xe9\x8f\x7f\x1e\x27\x1d\x29\xde\x30\xf4\xa1\x60\x2d\x37\xf8\x0f\x79\x22\x64\x0c\xc1\x58\x1e\x37\xeb\xc5\xc9\x8a\x77\xbe\x22\x9e\xcd\x36\x7c\x91\x7c\xf0\xf6\xab\x56\xcf\x70\x13\x7d\x81\x88\xbe\x17\xb2\xee\xf8\x56\xe2\x7d\x5d\xaa\x84\x9e\xc5\x1b\xc9\xd7\x49\xf4\x45\xc7\xb8\x49\x44\x1a\xc3\x10\xa9\xf7\x5d\x51\xa9\x58\x44\x0f\x43\x75\x82\x67\x57\xbb\x10\xd1\x03\xfb\xe1\xee\xe3\x05\x16\x37\x4e\xf4\x54\x5e\xf2\x22\x79\x14\xf7\x79\xea\xc2\x01\x04\xd2\xce\x53\x7b\x46\xaa\xec\xeb\x01\xd3\x97\xa5\x6a\xb7\x8a\x43\x58\x1c\x63\xbd\x39\x9c\x97\xd1\x83\x28\x8e\x72\x2e\x63\xb5\xc6\xcf\x38\xd2\xe5\x62\x91\x7c\x1e\x17\x3c\xef\xa8\x94\x81\x7e\x84\x5f\x51\xcf\xf5\xf5\xcf\x0a\xaf\xf3\xa2\xaa\xfb\x04\x69\xde\x54\x55\xbf\xa2\xdc\x62\x56\x22\x5f\x0b\xa0\x3a\x65\xd5\x2a\x33\xd0\x0a\x66\x4e\x43\x31\x56\xad\x29\x7f\x42\x51\xa9\xf7\x4f\x81\x72\xff\x29\x18\x95\x0f\x61\x87\x83\xf2\x05\x4e\xd7\xfc\x01\xed\xc3\x65\x2e\xb4\x1e\x31\xad\x60\xc4\x53\x69\x33\x11\x6c\xb6\x1c\xe0\x5e\x80\x2c\x39\xdd\xb0\x48\x65\x0e\x32\x8f\xdf\xb5\x52\x4f\xe0\xa7\x0f\xf3\x84\xa1\x84\x77\x29\x8b\x24\x65\x7c\x51\x90\x13\x1f\x2a\x43\xd8\x4a\x70\x7a\x3c\x95\x10\x8a\x8d\xe0\xf3\x01\x22\xe1\xc2\x2f\xee\x23\x34\x5b\xf0\x28\x49\x93\x82\xf8\xea\x20\xc5\x8c\x9b\xef\x35\xf7\x81\x99\xcb\x9c\x6f\x78\xea\x0d\x2b\x9e\x96\x3e\x35\xfa\x50\x8b\x2d\x7c\xa8\x89\x9e\xa1\x83\xe0\xf5\x0e\xb8\x47\x01\x26\x61\xf0\x01\xb9\xe3\x4f\x4c\xe7\x97\xb5\x5b\xf4\x1f\xc2\xff\xad\xd8\xe1\xdb\xb4\x82\x3d\x0c\xf2\x7d\x2e\xc7\xa6\xc9\xed\xca\xa7\x7b\x3d\x23\x89\x2d\x3a\xb9\xa2\x8a\xfb\xe4\x63\x77\x3d\x42\xcc\xa4\xc3\xe8\x1f\xdb\xa2\x77\xcd\x1e\x06\xcc\x5e\xbb\x92\xf8\x85\xdc\x19\x5d\x84\xfe\x7d\x86\x6f\xbd\xf1\xd7\x4a\xa5\xfb\x7a\xe4\x89\x92\x23\x51\x72\x06\x75\xa0\xf7\x31\x27\x71\x03\x38\xc7\xd6\xe4\xcc\xc5\xdc\x1d\x43\x7e\xb5\x7a\x1c\xc1\xc1\x68\x08\x20\xc8\x60\x10\x5b\x70\xea\x3a\x6b\x01\x5d\x0c\xc4\xdb\x43\x1b\x88\xd6\xb2\xaa\x7d\x33\x44\x10\xb0\xb3\x70\x3f\x46\x60\x11\xae\x8d\x70\x90\xb3\x0e\xab\x36\xd7\xba\x72\x8e\xbb\x90\x6d\xdc\xcd\x63\xd0\xb7\x9d\xcf\x35\x97\xe4\xf9\x23\x2d\x7e\x2a\x03\x8d\x1d\x19\xf1\x6c\x42\x83\x9b\xb5\x36\x7f\x5e\x65\x1b\xee\xed\xcf\xdb\xa7\xa4\xc4\x56\xc9\x79\x16\x16\x87\x04\x2c\x48\xa4\xd6\xf3\x44\x5a\x4e\x0a\x72\x72\x83\xa9\x71\x62\x19\x7b\x5d\x40\xc2\x9a\x0c\x58\x32\xa8\x36\xf7\x4e\xcd\x09\xc9\x8f\x43\x91\xb5\xcb\x1c\x0f\xed\xbb\x97\xad\x7e\xd1\x11\x69\xac\x7f\x81\xb9\x40\xd2\x27\xbe\xd1\x50\x40\x5d\x18\xa9\xb8\x40\xc7\x6e\x75\xfc\xa3\x40\xfd\xb0\x6c\xd0\x53\x09\x33\x84\x6c\x61\x56\x90\x1a\xc9\x0a\x1b\x30\xb5\xa5\xe2\x3d\xd3\xdb\x1b\xdd\x3e\x39\xbf\x4e\xac\x26\xdf\x1a\xab\xc1\x20\xf4\x7f\x8f\xf0\xcc\x16\x27\xf0\x9e\xbe\xe8\xe0\x9a\x44\x8d\x91\x60\x42\x90\x36\xe6\x42\xd4\x23\xb6\xe6\x89\xa4\x63\x80\xe5\x38\x63\x31\x2f\x97\xcb\x4e\x17\xe9\x6f\x3f\xd6\x52\x3d\x27\xff\xe3\x7d\xe1\x5b\xb9\x0a\x5f\xc2\x5b\x3c\xb1\x3d\xa1\xfb\xda\xd8\x7d\x5f\xc6\x41\xfc\x2b\x7a\xe3\x5b\x43\x62\x8d\x4d\xf4\x32\xde\xf8\x49\x1f\x6f\xbc\xc5\x76\x41\x82\x1f\x99\xd3\x16\x7f\xf3\x77\x37\xfd\x97\x71\xd3\xf7\xda\x14\x48\xea\x33\x4b\xaa\x0a\xfa\x96\x11\x3e\x93\xf7\xd2\x51\x41\xc3\xa8\x90\x5b\xce\x48\xf7\x58\xb3\x39\x8f\x5e\x81\x08\x13\x6e\xc7\xfd\xfd\x81\x3b\xc0\x2f\xb7\x6a\x2d\x18\x74\xa5\xb1\x90\x13\xa3\x2c\xc6\x11\xa0\x55\xcd\x07\x7a\xc4\x08\xe1\x51\xe0\x3a\x45\xe4\x4a\xec\x95\xea\xb7\x52\x3c\x31\x73\x5b\x8d\x42\xf8\x5e\xb0\x3c\x50\xe1\xef\xc0\x68\x87\x15\xac\xbf\x23\xec\xc8\xc5\x92\xe7\x31\x64\x98\xd0\x91\x4c\x79\xf4\x60\xfe\x1b\xc6\x47\x3d\x12\xc4\xd0\x72\x05\x20\xec\xd5\xb7\x96\xc8\x08\xa9\x10\x2d\xa7\xbb\x1b\x1f\xbe\xae\x19\x8f\x72\xa5\xd1\x69\xe4\x0a\x63\x43\x7e\x35\x28\xb0\x8f\x49\x5c\xf2\x14\x7b\xec\xf4\xb4\x0f\x85\xaf\xd5\x01\x47\x41\x0d\xbb\x26\x9a\x8d\x96\x03\x19\xaa\x60\x1a\xc7\x53\x79\xe6\x02\x26\xc7\xec\x5e\x0b\x42\x99\x69\x5b\x05\x60\xeb\x48\x5f\x4d\x7d\x68\x60\x02\x3b\x75\x88\x2d\x13\x60\x41\xd6\xc1\x44\xe8\xee\x99\xd8\x41\xa7\xba\xcf\xa2\x0c\xa6\x85\x9e\x04\x85\xf4\xfd\xb4\xa0\x9d\x90\x0b\x1e\x6f\x42\x2e\xc6\x44\x32\x88\xd2\x31\x1e\xaf\x13\x69\x0e\x81\x2d\xd6\xea\x6e\x1a\x5b\xb7\x01\x21\xc7\x50\xd3\x2c\x4d\x6b\x42\x50\x33\x29\x8c\x72\xc9\xf3\x24\xdd\x80\x3d\x91\xe5\xe2\x30\xe8\x27\x58\x1f\xca\x78\x82\x0a\x14\x44\x22\x53\x6a\xb1\x28\x53\xb4\x3a\xc0\x2e\x77\x1f\x40\x12\xe9\x7e\x32\x32\x0a\x47\x41\x95\x84\x82\x8e\xb1\x3e\xe7\x4b\x64\x8f\x34\xa2\x95\xc3\x22\x6e\x9e\x2b\x34\x07\x90\xfb\x4a\x3d\xd9\x54\xb7\x27\xee\xb1\xcc\x5d\xb7\xeb\x8b\x45\x59\xb6\xeb\xa1\xd6\x02\xb4\x72\x2a\x20\xfc\x73\xa1\x35\xfa\x4d\xc4\x4e\x36\x25\x12\x3e\x87\x4a\x5c\x7b\xcf\x75\xa9\x31\x63\xce\xac\x25\xdc\x5f\xd6\xd1\x51\x75\x5c\x33\xf7\x75\x89\x56\x92\x4d\xcb\xaf\xbf\xfe\x9d\x60\x5f\x43\x0a\x21\xd9\x23\x18\x1f\x03\xb6\x50\x6c\x1d\x44\xb6\xeb\x40\x20\x95\x68\x63\x45\x58\x1b\x44\xd5\xe6\xeb\x03\xc8\x93\x47\x2b\xa6\xcb\x39\x22\x18\x39\x85\x58\xb8\x74\xac\xe3\x17\x0a\xc0\x88\x78\xb3\xdb\xd1\xff\x3f\x12\x50\xc0\xa2\x2f\x53\x99\x29\x24\xc6\x07\xe8\xe7\x5c\xb0\x35\xcf\x1f\xa0\x86\x2f\xba\xe7\xa1\x10\xc0\xdb\x44\x8c\xab\xe1\x85\x83\xca\x78\x28\xa0\x83\x84\xd7\x2c\x2f\xa5\xb4\x45\xc9\x98\x51\x4c\xbd\xaf\x7f\x34\x95\xf3\x32\xb4\x3d\x2b\xc1\x02\xbf\xb5\x20\x60\x00\xc2\x56\x01\x53\x09\x0d\x8a\x6b\x3f\xae\x31\xeb\x11\x35\x98\xca\x17\x0e\x1b\xec\x72\xf8\x5d\x93\x0e\x66\x9d\x79\x41\xbe\x02\x7c\x6e\x58\x37\x1b\x96\x03\xb7\x3d\x28\x39\xd7\x50\x3c\x7b\xc4\x7e\x48\x1e\xc5\x88\xdd\x66\x3c\x7f\x18\xb1\x33\x0c\xff\xfd\x5e\xcd\xdb\x7c\x78\x0d\x42\x89\xbd\xfd\x78\xcf\x73\x63\x6d\xa3\x79\x69\xd7\xfe\x7f\x6e\x10\x03\xb0\xae\xd8\xf7\x7f\x4f\x44\x5e\x07\xd7\xc7\xff\x74\x4f\xc4\x8e\x30\xf5\xdf\xc1\x6b\xff\x23\xad\xe2\xed\x34\x1f\xff\x10\xfe\xaf\x95\x5f\x56\xe3\x02\xdd\x93\xa4\x5c\x2b\x2a\xed\xb7\x95\xd8\x9c\xc4\xf5\x4b\xb9\x99\xdf\xdc\xef\x28\x50\xfa\x78\xec\x52\xdb\x07\x80\xee\xe9\x55\x3b\x5f\xa7\xa9\xd2\x65\xbe\xfd\xf0\xdf\x54\x47\x6d\x7b\x6f\x21\x7a\x85\xcd\xb6\x9e\x0b\x60\x2d\xe8\x0b\x3f\xc1\xc7\x66\x7f\x51\xf3\x19\x60\xad\xf6\x3b\xe1\x6d\xcd\x39\xfa\x68\x15\x55\x86\xea\x6f\xc8\xdb\x4c\x00\xdf\x95\x57\x45\x7d\x40\xa0\xb6\xc3\x9c\x6b\x64\x2a\x2d\xe3\x3e\x66\xcc\xe6\xb9\x00\x6a\xf0\x5c\x40\xa1\x47\x46\x0c\x87\xe9\x26\xd0\x88\x02\xcb\xc7\x83\x62\xc2\x2c\x37\x48\x56\x25\x7b\x6b\x2e\x84\x74\xb3\x3d\x44\x95\x00\x1a\xec\xda\xec\x13\xda\xed\x49\xd8\xc2\x0b\x1d\x45\x69\x1b\xef\x05\xb6\x20\xa8\xdc\x4b\x51\x04\xd2\xbc\xa6\x5a\x54\x8e\x66\x25\x42\xf5\x9b\x42\xfc\xb7\xc6\xa0\x6b\xe4\x5c\x15\x07\x4a\xaf\x98\xde\x4b\xf8\xcb\xaf\x79\xb1\x42\x83\x76\xad\x0a\x81\x32\x13\x59\x82\x70\xbf\xa0\xd7\x79\x9e\xaa\x39\x54\x58\x2c\xb6\x30\x48\x46\x74\xb4\x7b\x4d\x5d\x73\xc1\xfa\x48\x06\x23\x4d\x20\xd3\x36\x17\x1a\x08\x57\x9a\x51\xaa\xbe\xf8\xe4\x61\x46\x77\x73\xb8\x46\xe8\x9f\x35\x8c\xed\x66\x49\x0e\x73\xac\x01\xac\x7a\xfe\x8c\x0c\x9a\x46\x81\x13\xa2\xaa\xa6\x30\x30\xb2\xd5\xd6\xbe\xd7\x16\xf2\x9f\xca\x13\xfc\x25\xb8\x04\xb8\xaf\xb1\xe5\xf0\xa0\x54\xb3\xd9\x9d\x3f\x4c\x5f\x65\x27\x21\x02\x91\x3c\x04\x23\xef\xcb\x04\x63\x60\x04\x59\x8d\xb2\x48\x72\xc1\x24\xa0\x10\xa6\x52\x97\xf3\x43\x4f\x4c\x62\xac\xb8\x47\x20\xd3\xd1\x22\xe3\x60\xca\x00\x5f\xd1\x61\xcb\x35\x8c\x9e\x49\x5f\x2b\xc7\xd2\x07\xf2\x94\x84\x3f\xe4\x4a\x62\x66\xbc\xfb\x76\xd7\x8e\x31\xd6\xc0\x8a\xb6\x70\x25\xbc\xec\xb6\xc9\x0b\xa8\xe6\x05\x19\x98\x37\x88\xa2\xf8\xb5\x2f\xf0\x30\x1a\xda\xf7\xea\x86\x78\xda\x54\xfe\xb3\xbd\x1b\xba\x41\xc5\x03\x76\xba\x99\x19\x73\x45\x75\x82\x9d\x2b\x63\xb3\x26\x64\xa0\x04\x76\x0f\xaa\xb1\xe5\xdb\x5a\xe5\x16\xd7\x12\x96\x74\x51\x94\x2e\x0b\xbf\x3e\x26\x3a\x20\x7b\x87\xde\x6e\x85\x60\xc7\xb9\x58\x1c\x7f\xca\xc5\x62\x66\x57\x7a\x0c\x1f\x34\x36\x5f\xd4\xa4\x7c\xef\xb9\x39\x74\xa6\x64\x3b\xf9\xe1\x0e\x6a\xd4\xda\x27\x61\x3b\xc1\x37\x25\x0b\xe6\xab\xdb\x9a\xef\x01\x06\x08\x11\xd7\xb9\xe8\x1b\x23\xfb\xe2\xd7\x5c\x17\x12\xac\x07\xd4\xaa\xa3\x08\xea\xff\xfc\xeb\xad\x32\x67\x7d\xae\xb7\xbb\x2a\x64\xc6\x0a\x7b\x2e\xdd\x85\xd7\x8d\x0b\xfd\xb2\xe8\x74\x58\x40\x9d\xf1\x27\x49\x3c\x36\x83\x5c\x4f\xfd\xae\xb5\x1a\x80\x28\xb8\xd6\x1a\x18\x38\x7f\xca\xa4\xf5\xf4\x25\xae\x8e\xe6\x88\x79\x0b\x9a\xa7\x69\x58\x51\xc3\x47\xda\xa6\xd2\xe7\xa5\x1a\xad\x35\x4d\xad\x0b\xaf\xa2\x6f\xb8\x82\xc7\xba\xe0\x85\x18\x59\xd2\x15\xa2\x2b\xa4\x78\xd8\xe1\x9c\x43\x69\x6b\x57\x43\x6d\xd7\x69\x7e\x29\x23\xf2\x37\x96\x17\xbd\x23\xf2\x8c\xdd\xce\x1e\x44\x03\xce\xbc\x73\xac\xed\x91\x8e\x80\x52\x02\x0e\xb3\x95\xb2\x11\xcf\x73\x8b\xf2\xa7\x5e\x99\xa5\x3b\x0f\xad\x92\x8e\x71\xae\x44\xf4\x90\xa9\x44\x0e\x96\x45\x15\x8a\x0b\xd8\xec\x05\xf3\xad\x39\xeb\xb0\xd7\xe5\x58\xd1\x27\xf1\x43\x34\xc0\x2b\x2c\x34\xd4\x93\xb1\x71\xe6\x2a\x65\x77\x6f\xbb\x97\xf6\x5f\x08\x7f\x37\xbc\x82\x2f\xb6\x25\x3e\x54\xbb\x55\x78\x8b\x63\xa7\xc2\x04\xca\x1b\xd9\x5f\x3d\x27\x9b\xb3\x0a\x85\x61\xeb\x94\x82\x0b\xf2\xef\x9e\xa1\xbf\x7b\x86\xfe\x9b\x7b\x86\xbe\xa4\x5b\x08\xb0\x31\xaf\xe9\x13\xda\x12\x20\xdf\xe3\x38\xba\x5e\x07\xe7\x38\xb6\x6a\xc7\xa3\xa0\xe8\x77\x90\xe9\xd8\x04\xfa\x5b\x22\x0c\x33\x3f\x73\x1e\x3d\x08\xd9\x19\xa3\xb7\xf4\x45\x9d\xf5\x57\x5f\x16\xc1\xd2\xc6\xbe\x14\xbc\xbd\x1d\xca\xe2\xa1\x4e\x44\x1a\xdc\x46\x08\x62\xce\x09\xe8\x9e\xe6\xc3\x0f\x01\x34\xa6\x72\x47\x6c\xad\x29\x0b\x0f\x83\x91\x48\x93\x84\x60\xa9\x1a\x15\x74\x5f\x4c\x9c\xed\x78\x96\x29\x95\xb6\x42\xe3\x5e\x74\x02\x1b\x89\x32\x7d\x27\x6f\x82\xca\xa8\x0e\x01\x63\x76\x16\x7d\xd2\x85\x4f\xd1\xc0\x7c\x0c\xa8\xc4\x01\xbb\x29\x2e\x21\x97\xd2\x4f\x47\x50\x5e\x91\x3b\x87\x0b\x61\xc4\xe6\x22\xe2\x50\xf8\xd5\x82\xf7\x22\xee\xb2\x4f\x42\x52\xa4\x46\x3a\x88\x6e\xf6\xd3\x11\xb5\x84\x76\x67\x49\x5b\xd9\x8d\xa1\x87\xab\xa6\x21\x58\x68\x39\x8e\xdc\x22\x49\x2c\xed\xe2\xae\x82\xc6\x96\x63\x7a\x06\xd5\x17\xfb\xdd\x70\xad\x72\x67\x42\x0d\x9d\x42\x3b\xfd\x05\xe9\x0f\x90\x8e\xb3\xee\x89\xdc\x99\xca\x13\x57\xe7\xd6\x63\xbf\x1c\x72\x0f\xc3\xa5\x88\x59\x6c\x2c\x0d\x72\x39\x7a\xcb\x65\xc4\x74\x19\xad\x80\xad\xb2\x2a\xa7\x42\xb9\xd5\x3c\xb1\xa3\xa9\x34\x06\x11\xb8\x5a\xd6\x1c\xf2\xe2\x9f\x8c\xb2\xaa\x93\xbf\x0a\x07\xcf\x22\xf2\xae\x10\x91\x85\x86\x93\x92\xad\xe8\x35\x4b\x1c\x8a\x00\x0b\x8f\x29\x29\xb3\x98\x17\x62\x3c\xf5\x68\x9b\x04\x3d\x9d\x16\xe5\x41\x2a\xb3\x0e\x3f\x2c\xc4\x31\xd6\x24\x6d\x9a\x2c\x44\xb4\x89\x1a\x55\x88\xb6\xd3\x44\xfc\xdd\x6c\xfb\x6d\x99\x6d\xc8\xb2\x8b\x39\x83\x43\xa6\x96\x86\x7a\xe3\x5f\xdf\x6f\x72\x05\x0b\x46\xa2\x07\xcc\xf3\x17\x34\x3b\x5b\x74\xe0\x61\xfa\x7c\x6f\x3b\x68\xfb\x75\xe6\x0d\x5b\x7f\x59\x07\x14\x08\x0d\xb5\x30\x0c\x2e\x16\xe1\xd6\x31\x0a\x6d\xef\xb0\x7e\x37\xcb\xcc\x6f\x0a\x9c\xd4\xc7\x70\x35\x1a\xb7\x83\x2b\x5d\x5a\x4d\x5b\x0a\xbc\xef\xb6\x68\xdc\x01\xab\x3b\x2f\xde\x68\x37\xeb\x55\x09\x68\xb1\xff\x27\x72\xb3\x57\x02\xe6\x26\x13\xb3\x32\x4f\xf7\x82\x1b\xdf\xdf\x5c\x1c\x39\x6d\x03\x34\xe7\xce\xba\x47\x45\xad\x34\xb4\xad\x49\x2c\x62\x82\x83\x46\x2a\x65\xf3\x72\xb1\x80\xfa\x25\x04\x0c\xb5\xc2\x08\x2a\xd3\x97\xba\xb0\xf7\x09\x32\xcd\x70\x5d\x4c\xa5\x92\x82\x4d\xbf\x3a\x9a\x7e\x65\xae\xb2\x9c\x47\x85\xc8\x91\x64\x20\xe5\xba\x60\x5a\x2c\x41\xd5\xa2\x4e\xef\x6f\x2e\x20\x2b\xb1\x58\x61\x73\xce\x64\xc5\x7c\x4f\xe4\x7c\x86\x5a\x3f\x40\x50\x2d\x83\x8a\x5b\x30\xf6\xb7\x5c\xb3\x44\x4e\xe5\x27\xd3\xc4\xd1\x52\xa9\x65\x2a\xc6\x76\x41\xc6\x67\xe4\x7a\xfc\x74\x80\x23\x80\xd7\x43\x58\xbf\xb9\x10\xb9\x54\x32\x89\x78\x0a\x09\x39\x53\x09\x5a\xf3\xc8\x7c\x0c\xb8\x46\xa7\x5f\x8d\xa7\x5f\x31\x08\x9f\x16\x8c\x47\x91\xc8\x0a\x11\x63\x69\xd3\x89\x64\x19\xe0\x17\x23\x31\x62\x85\xe0\x6b\x6d\x29\x9d\x59\x66\x6c\x4c\x30\x0d\x59\x22\x09\xe9\x34\x4f\x24\xcf\x37\x08\x66\xc2\x62\xe5\x94\xfc\xb1\x99\x4a\xf1\x19\xe8\x3f\x13\x60\x00\x2d\xb5\xa3\xa5\xa1\xc2\x04\xe6\x93\x4f\xe4\x66\xcc\x7e\x40\x86\x06\xa4\x40\xbd\xbf\xb9\xb0\xf4\x46\x94\x03\x3a\x95\x3a\x5a\x89\xb5\x60\x9f\x56\x45\x91\x7d\x1a\xe1\xff\xea\x4f\x10\x71\x94\x8a\xe1\xaf\x23\x66\x96\xc8\x28\xaa\x16\x2f\x9f\x6e\xa0\x86\x6c\x99\x51\xc1\xf9\xa9\x04\x2e\xf6\x3c\x44\xf7\x9a\xd9\x86\x1e\x03\x13\xbc\x82\x0b\x37\x52\x1c\x8a\x3b\x1e\x9b\xc9\xf9\x5f\x6c\xb2\xf0\x5d\x9a\x09\xb4\xb5\xc5\xdc\xa8\x40\x21\xd1\x90\xb2\x35\x36\x2f\x9c\x48\xf6\xc3\xdd\xdd\x35\xfb\xfe\xfc\xce\x2a\x3b\xf7\x37\x17\xb8\x2f\x80\x4e\x85\x71\xf6\xa7\xfa\x12\xdf\x6d\x32\xf1\xe7\x3f\xfd\x79\x2a\x99\xad\x51\x2e\xed\x4c\xe3\x89\x1e\x21\x25\x2c\xe0\x9d\x20\x30\x0b\x54\xce\xd0\x1f\x96\xdc\xa1\xe1\xe7\xa8\x9d\x3f\x91\xb7\x00\xee\xa8\x54\xa9\x87\x32\x73\x6e\xee\x50\x0f\x33\x1d\xde\xdf\x5c\x40\xeb\x40\xa7\x54\xac\xa0\x7e\x9a\x70\xde\x17\x58\x78\x6e\x07\x63\xfe\xfb\x51\x25\x31\xe3\x72\x63\xde\xc5\xa6\x61\x5b\xe6\x62\xa1\x72\x31\xb2\x4f\x9a\x06\x78\x91\xcc\x93\x34\x29\x36\x20\xa5\x6c\x5d\xfb\xcc\x72\xe4\x9b\x06\x8c\x35\x43\x00\x6f\xb3\xc1\xb0\x8c\xed\xdb\x7b\x1d\x22\xc0\x61\xd1\x5c\x6d\x44\x34\x74\xcc\xbb\xf3\x5c\xf0\x07\xb3\xbb\xa9\x85\xf1\x01\xd5\x8c\x15\xc7\x78\xc7\x2c\x4a\x19\xe1\xd6\x30\x63\xa0\xdd\x4f\x96\x53\xba\x61\xfc\x91\x27\x58\x53\xd6\x86\xcb\x17\x8b\x24\x4a\x78\x4a\x92\x63\x5e\x2e\xa0\x6c\x0c\xd7\x54\xb2\x08\xc1\x87\xa6\x11\xb0\x32\x6c\xc1\x7e\xdc\x50\x73\xb1\x4c\x10\x70\xfc\x94\x14\x2b\xcc\x2b\x18\xe3\x3a\xf3\x2c\xd1\xe3\x48\xad\xe1\xbc\xdd\xc2\x56\xd2\x64\xf4\x02\x0e\xbc\xb6\xcf\xd9\x5b\x0b\xb5\x5b\x67\xc5\x86\xf6\xde\x01\x5b\x27\xcb\x55\x01\x85\x5c\xa0\x77\x80\x44\x24\xeb\x2c\x05\xa3\x8f\x22\x8c\x16\xef\xab\xc5\x9a\xcb\x22\x89\xba\x62\x4a\xad\x25\xc1\xfb\x61\x3c\xe7\x9b\x62\xbb\x1f\xef\x23\xf1\xec\x73\xa4\xd0\x0f\x24\x32\xab\x0b\x64\x92\x81\x50\x5e\x26\x20\xf0\xaf\x97\x9c\xdd\x65\x42\x7d\x3a\x91\x9b\x4f\x9e\x84\x94\xcb\xa0\xf6\xd5\x96\xde\xed\xf9\xe7\xa9\xa2\x55\x63\x7c\x2a\x01\xd5\x69\x04\x06\x15\xa3\xdd\x7a\xc7\xb8\x2b\xc5\xac\xec\xb5\xdd\x34\x69\x32\x87\xbe\x49\x56\x68\xa6\xcb\x0c\xf2\x09\x0a\xc5\x32\x1e\x3d\x1c\x95\xd2\xfc\x8f\x11\x86\x78\xdc\x75\x48\x4e\x34\x95\x6a\xc1\xca\x02\x0f\x8e\xdd\xc2\xe0\x14\x09\x5c\x01\xde\x40\x5b\x8b\x62\xa5\x62\x97\x17\x66\xda\x84\xf9\x33\x23\x3a\x27\x7a\xe9\x77\xc7\xec\xda\x74\x68\x36\x31\xf5\xcd\xdd\xe7\x27\x92\x9d\xfe\xf3\x3f\xc3\xf3\x66\x72\x3f\x28\xc5\x16\x4a\xb1\xef\xd8\x78\x3c\xfe\x77\xfc\x9b\x69\x94\xcb\x0d\xfd\x8b\xcb\xcd\xd8\x34\xf7\x21\x57\xeb\xb7\x0b\xa5\x0e\xe8\xef\x50\xb4\xd9\xfc\x47\xb2\x60\x6f\xcd\x43\xf7\xd0\xd5\x9d\x7a\x3b\x2d\xbf\xfe\xfa\x9b\x7f\x35\x8f\x1e\xb0\xff\xc4\x67\x82\xc7\xff\x16\x0e\xf5\x9b\x1d\x43\xfd\x3d\x7f\xe4\x7d\xc6\xca\xbe\x83\xbb\xc6\x34\xb0\x75\x8c\x89\x7e\xfb\x41\xa9\x31\x58\xff\xe1\xe8\xb0\x59\xf3\x04\x8e\x22\x78\xea\xdf\x83\x61\x33\x3b\xee\xdf\xed\x18\x37\xa2\xea\xdd\xc8\xb1\xf9\x0f\x4a\xbd\x1d\x8f\x8d\xdc\xa2\x79\xc5\x51\xbf\x3d\xa8\x4e\x34\x7c\x40\x73\xfc\xe6\xe7\x09\x0e\xff\xec\xfc\xf6\xf4\x66\x72\x7d\x77\x75\x73\x70\x6c\xbf\xc0\xaf\x40\xf0\x3e\xb3\xa5\xb5\xdd\xc0\xff\xf7\x8e\x81\x7f\xaf\xec\x98\x61\xd0\xc7\xdf\x31\x5c\xcd\x6c\x3e\xfe\xa0\xd4\x7f\x8e\xc7\xe3\xbf\xd1\xcf\x5c\x6e\x46\xe6\x62\x32\xcf\x64\x28\xca\x3f\xf2\x5c\xaf\x78\x6a\xbe\x29\x18\x83\xfb\x88\xd6\x16\x6d\x73\xc9\xa2\xd6\xd8\xbd\x5c\xfb\xe6\xa0\x33\x58\x58\x78\xea\x1f\xbf\x63\x32\x49\xfd\xf2\x05\x7d\xc0\x3a\xdd\x01\xb5\x44\xf4\xe0\x8e\x8b\xab\x11\x3a\xdf\xb0\xac\x7e\x70\x31\xef\x6c\x63\x2b\x14\x18\x71\x3f\x95\x6f\x5a\x24\xfa\x91\x51\xed\xc6\xf0\x83\xb9\xa0\xde\xd8\xea\xf1\xf6\x5a\x70\x95\xb5\x70\x66\x21\x10\x8d\xa7\x55\x52\x8e\x5a\x9b\x7e\xe8\x2e\xbc\x80\xac\x0a\xd4\xce\x37\x47\x6f\x28\x51\xc8\x77\x51\x25\x92\x9f\x7e\xb5\x50\x6a\x3c\xe7\x39\x8c\xee\xf3\xd1\x66\xfc\xd7\xe9\x57\xf8\x3d\xa8\x7c\xa0\x62\x04\x8d\x4f\xbf\x82\x5f\x61\x3b\x4c\xe5\xef\x6f\xaf\x2e\xa7\xf2\xbb\xef\xbe\xfb\x0e\x67\xcb\xfc\xbb\x25\xf6\x62\xae\x2b\x10\xb7\xa8\xa7\x94\xda\x96\x94\x14\xcb\x32\xe5\xf9\x54\xb6\x87\x6b\x62\xe1\x85\xe6\xc8\x07\x6f\x68\x9f\x8d\x6c\x75\x0b\x28\x52\x66\x65\x1c\xfa\x26\x3f\xfd\xff\x66\xc8\x9f\x48\x45\x74\x42\x3e\x9c\x82\xb1\xdd\xcc\xc7\x76\xab\x9a\xc9\x36\xfb\xd7\xeb\x59\x8b\x24\x15\x74\x70\xed\xe6\xbe\x16\xb9\x56\xd2\xef\x19\x32\x08\x80\xdb\x0c\x02\x00\xec\x3b\xf6\xee\xdf\x6b\xbf\x9a\x75\xb0\x3f\x7e\x53\x91\x04\x8c\xf9\xa6\xa6\x5f\xc1\xa8\xa7\x5f\x1d\xb3\xe9\x57\x6d\xfb\xa6\x3a\xb0\x31\x0e\x65\xfa\xd5\xc8\x37\x00\xc3\xb8\xe4\x6b\x6c\xa4\xfc\xfa\xeb\xdf\x45\x38\x04\x4c\x5d\x0b\x9e\x34\x43\xea\x7e\x30\x18\xe2\xa4\x16\x3a\xb3\x13\x61\x53\x20\x9f\x44\x9a\x1e\x3e\x48\xf5\x84\x75\xc6\x21\x4e\x44\x59\xca\x0c\xb7\x47\x75\x71\xa9\x36\x59\x6d\xc5\x6d\xd2\xa6\xeb\xc6\x95\xb7\x83\x05\x9d\xca\x4f\xb0\x75\xec\x8a\x12\x1d\x11\xd0\x81\xba\x9e\xc0\xa8\xa1\x9d\x60\x73\x2c\x68\x23\x4c\x25\x34\xe3\xd6\x9c\xbd\x05\xe0\x17\x7d\x4a\x43\xb3\xb6\xc6\xd3\x9f\xff\xf4\xe7\x83\xe3\x7d\xd6\xa9\xda\x5c\x65\xa9\xe0\x7b\xb0\x8d\x77\xe3\x6f\xde\x7d\xa3\xa7\x5f\xd1\xac\xb7\x9b\xd8\x17\x89\x2e\x7e\xaa\x69\x60\xcf\x28\x76\x6e\x14\x87\xd7\x0a\x5e\xd8\xa1\xe2\x30\xfb\x06\x2d\x6e\xaa\x61\x05\xb5\xb0\x6e\x1d\x30\xce\x6c\x19\x78\x33\xee\x41\xea\x9d\x9b\x2f\x34\xb6\xd8\x53\xce\xb3\x4c\xe4\xd6\x57\xde\x08\x67\x40\x4d\x75\xe8\xc5\x8a\xfe\x36\x61\x66\xb6\x4d\xad\x69\x78\x0c\xa6\x6e\xdc\xbe\x72\x97\x65\x9a\x76\xae\xdc\xee\x62\xc9\x97\xf7\x17\x17\xb3\x9f\x4e\x2e\xee\xcf\xed\xe7\xb7\x16\x1f\x0e\x1e\xeb\x9c\x13\x37\x12\x9a\x13\xc4\x55\x15\x80\xa5\x2a\xd7\x22\xb7\x4c\x61\xfe\xab\x11\x47\x52\xa6\x69\xb5\x2c\xf6\x54\x7e\xa2\x76\x40\x0c\x94\x32\xb1\x6a\xca\xd6\x89\xab\xf6\x0f\x8f\x7d\x32\x8d\x7f\xc2\x77\x0f\x99\xff\x88\x63\x76\xe9\x7a\xed\x98\x57\x22\x9c\xd8\xe3\x38\x60\xbe\x6d\xd7\x71\x78\xe9\xc2\xff\xcf\x3b\x1e\xf7\x12\x8a\x7e\x19\xc9\x8b\xf5\xfa\x5f\xe4\x74\xe0\xdc\x7d\xaa\x42\xc1\x9d\xbb\x34\xc6\xa8\x21\xb4\x3b\xc2\x72\xed\xba\x20\xce\x62\x9c\xb3\xa9\x44\x41\x6c\xc6\x54\xa8\xee\x31\xb1\x09\x45\x90\x52\x2e\x97\x25\x5f\x0a\x3d\x62\xb6\xf3\xa9\xb4\xd6\xa9\xb5\x75\x1c\x30\x07\x18\x59\x6b\x5b\xa8\x96\x02\x9c\xc8\xa9\xa4\x6f\x82\x1b\x96\x9a\xc7\x74\xd4\xdf\xdf\xba\xcf\xa1\xbc\x6f\x6c\x88\x2a\xce\xcb\xa9\xc4\xc5\x45\xdf\x98\x05\x1b\x82\xda\xd1\xbc\x9b\x38\xc0\x83\xd1\xae\x8b\x59\xa1\x96\x00\x7b\x9c\x4a\xc7\x82\x85\xe0\x0c\x6b\xaf\xf9\xda\xa0\x38\xa4\xdd\xf2\xc4\x2e\x86\x3d\x13\x34\xb6\xf6\x5d\xbf\xf7\x1d\x60\x0e\xdc\xac\xd5\x96\xdf\xbe\x6d\xbd\x18\xeb\x09\xc8\xe1\x81\xe0\xe8\xa2\x46\x04\xea\xb3\xf6\xd1\xd8\xef\xc2\x67\x3a\xb3\x47\x55\x39\x4f\x07\x0c\x09\x9f\xdf\x3a\x28\x14\xc9\xdb\x07\xd5\xc3\x23\x7d\x53\x3b\x5a\x66\x9b\x6e\xeb\x76\xae\x54\xc7\xba\xbc\x20\x66\xb7\x32\x28\x7a\x61\xd7\x64\x94\x51\xf1\x9c\xfd\xd2\x83\x0f\xa8\x3e\x45\x56\xfa\x6c\x1b\x50\x9a\xe8\x67\x0d\xc7\xeb\x4f\xbd\x47\xe4\x34\x04\xba\xec\x06\x49\x58\xba\xe7\x2a\x02\xb6\x43\x4c\x5a\x33\x05\xd3\x5b\x44\x82\xe2\xc5\x1c\x9e\x11\x1c\x22\xb3\xff\x47\x6e\x13\x8d\xfc\xca\x8d\x60\x90\x51\x99\x6b\x23\x2e\x49\xde\x91\xd4\x56\x39\xe3\x53\x69\xd9\x60\xac\x38\x3e\xb1\xfe\xe0\xdc\xfd\x15\x39\x96\x32\x2c\x59\x07\x41\xa1\x02\xbc\xe4\x24\x0d\xa7\xf2\x91\xe7\x09\x97\x80\x69\x9e\x6b\xa8\x37\x0c\x26\xdd\x86\xd1\x0f\x8e\x80\x43\x87\x4e\xe6\x1d\x32\xaf\xa6\x06\x54\xee\xf9\x7f\x30\xff\xf7\xb7\x7f\xf8\xbf\x01\x00\x00\xff\xff\x8d\x88\x2a\xee\x6c\xfa\x06\x00") func adminSwaggerJsonBytes() ([]byte, error) { return bindataRead( @@ -93,7 +93,7 @@ func adminSwaggerJson() (*asset, error) { return nil, err } - info := bindataFileInfo{name: "admin.swagger.json", size: 453593, mode: os.FileMode(420), modTime: time.Unix(1562572800, 0)} + info := bindataFileInfo{name: "admin.swagger.json", size: 457324, mode: os.FileMode(420), modTime: time.Unix(1562572800, 0)} a := &asset{bytes: bytes, info: info} return a, nil } @@ -157,11 +157,13 @@ var _bindata = map[string]func() (*asset, error){ // directory embedded in the file by go-bindata. // For example if you run go-bindata on data/... and data contains the // following hierarchy: -// data/ -// foo.txt -// img/ -// a.png -// b.png +// +// data/ +// foo.txt +// img/ +// a.png +// b.png +// // then AssetDir("data") would return []string{"foo.txt", "img"} // AssetDir("data/img") would return []string{"a.png", "b.png"} // AssetDir("foo.txt") and AssetDir("notexist") would return an error diff --git a/flyteidl/gen/pb-java/flyteidl/admin/NodeExecutionOuterClass.java b/flyteidl/gen/pb-java/flyteidl/admin/NodeExecutionOuterClass.java index b1301715d6..da63f51ffc 100644 --- a/flyteidl/gen/pb-java/flyteidl/admin/NodeExecutionOuterClass.java +++ b/flyteidl/gen/pb-java/flyteidl/admin/NodeExecutionOuterClass.java @@ -15710,6 +15710,1218 @@ public flyteidl.admin.NodeExecutionOuterClass.NodeExecutionGetDataResponse getDe } + public interface GetDynamicNodeWorkflowRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:flyteidl.admin.GetDynamicNodeWorkflowRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + boolean hasId(); + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier getId(); + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder getIdOrBuilder(); + } + /** + * Protobuf type {@code flyteidl.admin.GetDynamicNodeWorkflowRequest} + */ + public static final class GetDynamicNodeWorkflowRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:flyteidl.admin.GetDynamicNodeWorkflowRequest) + GetDynamicNodeWorkflowRequestOrBuilder { + private static final long serialVersionUID = 0L; + // Use GetDynamicNodeWorkflowRequest.newBuilder() to construct. + private GetDynamicNodeWorkflowRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetDynamicNodeWorkflowRequest() { + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetDynamicNodeWorkflowRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder subBuilder = null; + if (id_ != null) { + subBuilder = id_.toBuilder(); + } + id_ = input.readMessage(flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.parser(), extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(id_); + id_ = subBuilder.buildPartial(); + } + + break; + } + default: { + if (!parseUnknownField( + input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.class, flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.Builder.class); + } + + public static final int ID_FIELD_NUMBER = 1; + private flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier id_; + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public boolean hasId() { + return id_ != null; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier getId() { + return id_ == null ? flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.getDefaultInstance() : id_; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder getIdOrBuilder() { + return getId(); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (id_ != null) { + output.writeMessage(1, getId()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (id_ != null) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getId()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest)) { + return super.equals(obj); + } + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest other = (flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest) obj; + + if (hasId() != other.hasId()) return false; + if (hasId()) { + if (!getId() + .equals(other.getId())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasId()) { + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + getId().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code flyteidl.admin.GetDynamicNodeWorkflowRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:flyteidl.admin.GetDynamicNodeWorkflowRequest) + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.class, flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.Builder.class); + } + + // Construct using flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (idBuilder_ == null) { + id_ = null; + } else { + id_ = null; + idBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest getDefaultInstanceForType() { + return flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.getDefaultInstance(); + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest build() { + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest buildPartial() { + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest result = new flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest(this); + if (idBuilder_ == null) { + result.id_ = id_; + } else { + result.id_ = idBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest) { + return mergeFrom((flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest other) { + if (other == flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest.getDefaultInstance()) return this; + if (other.hasId()) { + mergeId(other.getId()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier id_; + private com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder> idBuilder_; + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public boolean hasId() { + return idBuilder_ != null || id_ != null; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier getId() { + if (idBuilder_ == null) { + return id_ == null ? flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.getDefaultInstance() : id_; + } else { + return idBuilder_.getMessage(); + } + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public Builder setId(flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier value) { + if (idBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + id_ = value; + onChanged(); + } else { + idBuilder_.setMessage(value); + } + + return this; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public Builder setId( + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder builderForValue) { + if (idBuilder_ == null) { + id_ = builderForValue.build(); + onChanged(); + } else { + idBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public Builder mergeId(flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier value) { + if (idBuilder_ == null) { + if (id_ != null) { + id_ = + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.newBuilder(id_).mergeFrom(value).buildPartial(); + } else { + id_ = value; + } + onChanged(); + } else { + idBuilder_.mergeFrom(value); + } + + return this; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public Builder clearId() { + if (idBuilder_ == null) { + id_ = null; + onChanged(); + } else { + id_ = null; + idBuilder_ = null; + } + + return this; + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder getIdBuilder() { + + onChanged(); + return getIdFieldBuilder().getBuilder(); + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + public flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder getIdOrBuilder() { + if (idBuilder_ != null) { + return idBuilder_.getMessageOrBuilder(); + } else { + return id_ == null ? + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.getDefaultInstance() : id_; + } + } + /** + * .flyteidl.core.NodeExecutionIdentifier id = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder> + getIdFieldBuilder() { + if (idBuilder_ == null) { + idBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.NodeExecutionIdentifierOrBuilder>( + getId(), + getParentForChildren(), + isClean()); + id_ = null; + } + return idBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:flyteidl.admin.GetDynamicNodeWorkflowRequest) + } + + // @@protoc_insertion_point(class_scope:flyteidl.admin.GetDynamicNodeWorkflowRequest) + private static final flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest(); + } + + public static flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GetDynamicNodeWorkflowRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetDynamicNodeWorkflowRequest(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.GetDynamicNodeWorkflowRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface DynamicNodeWorkflowResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:flyteidl.admin.DynamicNodeWorkflowResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + boolean hasCompiledWorkflow(); + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + flyteidl.core.Compiler.CompiledWorkflowClosure getCompiledWorkflow(); + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder getCompiledWorkflowOrBuilder(); + } + /** + * Protobuf type {@code flyteidl.admin.DynamicNodeWorkflowResponse} + */ + public static final class DynamicNodeWorkflowResponse extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:flyteidl.admin.DynamicNodeWorkflowResponse) + DynamicNodeWorkflowResponseOrBuilder { + private static final long serialVersionUID = 0L; + // Use DynamicNodeWorkflowResponse.newBuilder() to construct. + private DynamicNodeWorkflowResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private DynamicNodeWorkflowResponse() { + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private DynamicNodeWorkflowResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + flyteidl.core.Compiler.CompiledWorkflowClosure.Builder subBuilder = null; + if (compiledWorkflow_ != null) { + subBuilder = compiledWorkflow_.toBuilder(); + } + compiledWorkflow_ = input.readMessage(flyteidl.core.Compiler.CompiledWorkflowClosure.parser(), extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(compiledWorkflow_); + compiledWorkflow_ = subBuilder.buildPartial(); + } + + break; + } + default: { + if (!parseUnknownField( + input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.class, flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.Builder.class); + } + + public static final int COMPILED_WORKFLOW_FIELD_NUMBER = 1; + private flyteidl.core.Compiler.CompiledWorkflowClosure compiledWorkflow_; + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public boolean hasCompiledWorkflow() { + return compiledWorkflow_ != null; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public flyteidl.core.Compiler.CompiledWorkflowClosure getCompiledWorkflow() { + return compiledWorkflow_ == null ? flyteidl.core.Compiler.CompiledWorkflowClosure.getDefaultInstance() : compiledWorkflow_; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder getCompiledWorkflowOrBuilder() { + return getCompiledWorkflow(); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (compiledWorkflow_ != null) { + output.writeMessage(1, getCompiledWorkflow()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (compiledWorkflow_ != null) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getCompiledWorkflow()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse)) { + return super.equals(obj); + } + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse other = (flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse) obj; + + if (hasCompiledWorkflow() != other.hasCompiledWorkflow()) return false; + if (hasCompiledWorkflow()) { + if (!getCompiledWorkflow() + .equals(other.getCompiledWorkflow())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasCompiledWorkflow()) { + hash = (37 * hash) + COMPILED_WORKFLOW_FIELD_NUMBER; + hash = (53 * hash) + getCompiledWorkflow().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code flyteidl.admin.DynamicNodeWorkflowResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:flyteidl.admin.DynamicNodeWorkflowResponse) + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.class, flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.Builder.class); + } + + // Construct using flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (compiledWorkflowBuilder_ == null) { + compiledWorkflow_ = null; + } else { + compiledWorkflow_ = null; + compiledWorkflowBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return flyteidl.admin.NodeExecutionOuterClass.internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse getDefaultInstanceForType() { + return flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.getDefaultInstance(); + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse build() { + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse buildPartial() { + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse result = new flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse(this); + if (compiledWorkflowBuilder_ == null) { + result.compiledWorkflow_ = compiledWorkflow_; + } else { + result.compiledWorkflow_ = compiledWorkflowBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse) { + return mergeFrom((flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse other) { + if (other == flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse.getDefaultInstance()) return this; + if (other.hasCompiledWorkflow()) { + mergeCompiledWorkflow(other.getCompiledWorkflow()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private flyteidl.core.Compiler.CompiledWorkflowClosure compiledWorkflow_; + private com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.Compiler.CompiledWorkflowClosure, flyteidl.core.Compiler.CompiledWorkflowClosure.Builder, flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder> compiledWorkflowBuilder_; + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public boolean hasCompiledWorkflow() { + return compiledWorkflowBuilder_ != null || compiledWorkflow_ != null; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public flyteidl.core.Compiler.CompiledWorkflowClosure getCompiledWorkflow() { + if (compiledWorkflowBuilder_ == null) { + return compiledWorkflow_ == null ? flyteidl.core.Compiler.CompiledWorkflowClosure.getDefaultInstance() : compiledWorkflow_; + } else { + return compiledWorkflowBuilder_.getMessage(); + } + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public Builder setCompiledWorkflow(flyteidl.core.Compiler.CompiledWorkflowClosure value) { + if (compiledWorkflowBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + compiledWorkflow_ = value; + onChanged(); + } else { + compiledWorkflowBuilder_.setMessage(value); + } + + return this; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public Builder setCompiledWorkflow( + flyteidl.core.Compiler.CompiledWorkflowClosure.Builder builderForValue) { + if (compiledWorkflowBuilder_ == null) { + compiledWorkflow_ = builderForValue.build(); + onChanged(); + } else { + compiledWorkflowBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public Builder mergeCompiledWorkflow(flyteidl.core.Compiler.CompiledWorkflowClosure value) { + if (compiledWorkflowBuilder_ == null) { + if (compiledWorkflow_ != null) { + compiledWorkflow_ = + flyteidl.core.Compiler.CompiledWorkflowClosure.newBuilder(compiledWorkflow_).mergeFrom(value).buildPartial(); + } else { + compiledWorkflow_ = value; + } + onChanged(); + } else { + compiledWorkflowBuilder_.mergeFrom(value); + } + + return this; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public Builder clearCompiledWorkflow() { + if (compiledWorkflowBuilder_ == null) { + compiledWorkflow_ = null; + onChanged(); + } else { + compiledWorkflow_ = null; + compiledWorkflowBuilder_ = null; + } + + return this; + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public flyteidl.core.Compiler.CompiledWorkflowClosure.Builder getCompiledWorkflowBuilder() { + + onChanged(); + return getCompiledWorkflowFieldBuilder().getBuilder(); + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + public flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder getCompiledWorkflowOrBuilder() { + if (compiledWorkflowBuilder_ != null) { + return compiledWorkflowBuilder_.getMessageOrBuilder(); + } else { + return compiledWorkflow_ == null ? + flyteidl.core.Compiler.CompiledWorkflowClosure.getDefaultInstance() : compiledWorkflow_; + } + } + /** + * .flyteidl.core.CompiledWorkflowClosure compiled_workflow = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.Compiler.CompiledWorkflowClosure, flyteidl.core.Compiler.CompiledWorkflowClosure.Builder, flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder> + getCompiledWorkflowFieldBuilder() { + if (compiledWorkflowBuilder_ == null) { + compiledWorkflowBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + flyteidl.core.Compiler.CompiledWorkflowClosure, flyteidl.core.Compiler.CompiledWorkflowClosure.Builder, flyteidl.core.Compiler.CompiledWorkflowClosureOrBuilder>( + getCompiledWorkflow(), + getParentForChildren(), + isClean()); + compiledWorkflow_ = null; + } + return compiledWorkflowBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:flyteidl.admin.DynamicNodeWorkflowResponse) + } + + // @@protoc_insertion_point(class_scope:flyteidl.admin.DynamicNodeWorkflowResponse) + private static final flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse(); + } + + public static flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public DynamicNodeWorkflowResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new DynamicNodeWorkflowResponse(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public flyteidl.admin.NodeExecutionOuterClass.DynamicNodeWorkflowResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + private static final com.google.protobuf.Descriptors.Descriptor internal_static_flyteidl_admin_NodeExecutionGetRequest_descriptor; private static final @@ -15770,6 +16982,16 @@ public flyteidl.admin.NodeExecutionOuterClass.NodeExecutionGetDataResponse getDe private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_flyteidl_admin_NodeExecutionGetDataResponse_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -15845,9 +17067,13 @@ public flyteidl.admin.NodeExecutionOuterClass.NodeExecutionGetDataResponse getDe ".core.LiteralMap\022E\n\020dynamic_workflow\030\020 \001" + "(\0132+.flyteidl.admin.DynamicWorkflowNodeM" + "etadata\022-\n\nflyte_urls\030\021 \001(\0132\031.flyteidl.a" + - "dmin.FlyteURLsB=Z;github.com/flyteorg/fl" + - "yte/flyteidl/gen/pb-go/flyteidl/adminb\006p" + - "roto3" + "dmin.FlyteURLs\"S\n\035GetDynamicNodeWorkflow" + + "Request\0222\n\002id\030\001 \001(\0132&.flyteidl.core.Node" + + "ExecutionIdentifier\"`\n\033DynamicNodeWorkfl" + + "owResponse\022A\n\021compiled_workflow\030\001 \001(\0132&." + + "flyteidl.core.CompiledWorkflowClosureB=Z" + + ";github.com/flyteorg/flyte/flyteidl/gen/" + + "pb-go/flyteidl/adminb\006proto3" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -15941,6 +17167,18 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_flyteidl_admin_NodeExecutionGetDataResponse_descriptor, new java.lang.String[] { "Inputs", "Outputs", "FullInputs", "FullOutputs", "DynamicWorkflow", "FlyteUrls", }); + internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor = + getDescriptor().getMessageTypes().get(12); + internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_flyteidl_admin_GetDynamicNodeWorkflowRequest_descriptor, + new java.lang.String[] { "Id", }); + internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor = + getDescriptor().getMessageTypes().get(13); + internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_flyteidl_admin_DynamicNodeWorkflowResponse_descriptor, + new java.lang.String[] { "CompiledWorkflow", }); flyteidl.admin.Common.getDescriptor(); flyteidl.core.Execution.getDescriptor(); flyteidl.core.Catalog.getDescriptor(); diff --git a/flyteidl/gen/pb-java/flyteidl/service/Admin.java b/flyteidl/gen/pb-java/flyteidl/service/Admin.java index 9de7851811..a209b21ae6 100644 --- a/flyteidl/gen/pb-java/flyteidl/service/Admin.java +++ b/flyteidl/gen/pb-java/flyteidl/service/Admin.java @@ -38,7 +38,7 @@ public static void registerAllExtensions( "admin/task_execution.proto\032\034flyteidl/adm" + "in/version.proto\032\033flyteidl/admin/common." + "proto\032\'flyteidl/admin/description_entity" + - ".proto2\227v\n\014AdminService\022\216\001\n\nCreateTask\022!" + + ".proto2\270y\n\014AdminService\022\216\001\n\nCreateTask\022!" + ".flyteidl.admin.TaskCreateRequest\032\".flyt" + "eidl.admin.TaskCreateResponse\"9\202\323\344\223\0023\"\r/" + "api/v1/tasks:\001*Z\037\"\032/api/v1/tasks/org/{id" + @@ -178,247 +178,257 @@ public static void registerAllExtensions( "e_id}Z\213\001\022\210\001/api/v1/node_executions/org/{" + "id.execution_id.org}/{id.execution_id.pr" + "oject}/{id.execution_id.domain}/{id.exec" + - "ution_id.name}/{id.node_id}\022\371\002\n\022ListNode" + - "Executions\022(.flyteidl.admin.NodeExecutio" + - "nListRequest\032!.flyteidl.admin.NodeExecut" + - "ionList\"\225\002\202\323\344\223\002\216\002\022s/api/v1/node_executio" + - "ns/{workflow_execution_id.project}/{work" + - "flow_execution_id.domain}/{workflow_exec" + - "ution_id.name}Z\226\001\022\223\001/api/v1/node_executi" + - "ons/org/{workflow_execution_id.org}/{wor" + - "kflow_execution_id.project}/{workflow_ex" + - "ecution_id.domain}/{workflow_execution_i" + - "d.name}\022\217\010\n\031ListNodeExecutionsForTask\022/." + - "flyteidl.admin.NodeExecutionForTaskListR" + - "equest\032!.flyteidl.admin.NodeExecutionLis" + - "t\"\235\007\202\323\344\223\002\226\007\022\251\003/api/v1/children/task_exec" + - "utions/{task_execution_id.node_execution" + - "_id.execution_id.project}/{task_executio" + - "n_id.node_execution_id.execution_id.doma" + - "in}/{task_execution_id.node_execution_id" + - ".execution_id.name}/{task_execution_id.n" + - "ode_execution_id.node_id}/{task_executio" + - "n_id.task_id.project}/{task_execution_id" + - ".task_id.domain}/{task_execution_id.task" + - "_id.name}/{task_execution_id.task_id.ver" + - "sion}/{task_execution_id.retry_attempt}Z" + - "\347\003\022\344\003/api/v1/children/task_executions/or" + - "g/{task_execution_id.node_execution_id.e" + - "xecution_id.org}/{task_execution_id.node" + - "_execution_id.execution_id.project}/{tas" + - "k_execution_id.node_execution_id.executi" + - "on_id.domain}/{task_execution_id.node_ex" + - "ecution_id.execution_id.name}/{task_exec" + - "ution_id.node_execution_id.node_id}/{tas" + - "k_execution_id.task_id.project}/{task_ex" + - "ecution_id.task_id.domain}/{task_executi" + - "on_id.task_id.name}/{task_execution_id.t" + - "ask_id.version}/{task_execution_id.retry" + - "_attempt}\022\203\003\n\024GetNodeExecutionData\022+.fly" + - "teidl.admin.NodeExecutionGetDataRequest\032" + - ",.flyteidl.admin.NodeExecutionGetDataRes" + - "ponse\"\217\002\202\323\344\223\002\210\002\022s/api/v1/data/node_execu" + - "tions/{id.execution_id.project}/{id.exec" + - "ution_id.domain}/{id.execution_id.name}/" + - "{id.node_id}Z\220\001\022\215\001/api/v1/data/node_exec" + - "utions/org/{id.execution_id.org}/{id.exe" + - "cution_id.project}/{id.execution_id.doma" + - "in}/{id.execution_id.name}/{id.node_id}\022" + - "\250\001\n\017RegisterProject\022&.flyteidl.admin.Pro" + - "jectRegisterRequest\032\'.flyteidl.admin.Pro" + - "jectRegisterResponse\"D\202\323\344\223\002>\"\020/api/v1/pr" + - "ojects:\001*Z\'\"\"/api/v1/projects/org/{proje" + - "ct.org}:\001*\022\227\001\n\rUpdateProject\022\027.flyteidl." + - "admin.Project\032%.flyteidl.admin.ProjectUp" + - "dateResponse\"F\202\323\344\223\002@\032\025/api/v1/projects/{" + - "id}:\001*Z$\032\037/api/v1/projects/org/{org}/{id" + - "}:\001*\022\204\001\n\014ListProjects\022\".flyteidl.admin.P" + - "rojectListRequest\032\030.flyteidl.admin.Proje" + - "cts\"6\202\323\344\223\0020\022\020/api/v1/projectsZ\034\022\032/api/v1" + - "/projects/org/{org}\022\325\001\n\023CreateWorkflowEv" + - "ent\022-.flyteidl.admin.WorkflowExecutionEv" + - "entRequest\032..flyteidl.admin.WorkflowExec" + - "utionEventResponse\"_\202\323\344\223\002Y\"\030/api/v1/even" + - "ts/workflows:\001*Z:\"5/api/v1/events/org/{e" + - "vent.execution_id.org}/workflows:\001*\022\304\001\n\017" + - "CreateNodeEvent\022).flyteidl.admin.NodeExe" + - "cutionEventRequest\032*.flyteidl.admin.Node" + - "ExecutionEventResponse\"Z\202\323\344\223\002T\"\024/api/v1/" + - "events/nodes:\001*Z9\"4/api/v1/events/org/{e" + - "vent.id.execution_id.org}/nodes:\001*\022\332\001\n\017C" + - "reateTaskEvent\022).flyteidl.admin.TaskExec" + - "utionEventRequest\032*.flyteidl.admin.TaskE" + - "xecutionEventResponse\"p\202\323\344\223\002j\"\024/api/v1/e" + - "vents/tasks:\001*ZO\"J/api/v1/events/org/{ev" + - "ent.parent_node_execution_id.execution_i" + - "d.org}/tasks:\001*\022\313\005\n\020GetTaskExecution\022\'.f" + - "lyteidl.admin.TaskExecutionGetRequest\032\035." + - "flyteidl.admin.TaskExecution\"\356\004\202\323\344\223\002\347\004\022\231" + - "\002/api/v1/task_executions/{id.node_execut" + - "ion_id.execution_id.project}/{id.node_ex" + - "ecution_id.execution_id.domain}/{id.node" + - "_execution_id.execution_id.name}/{id.nod" + - "e_execution_id.node_id}/{id.task_id.proj" + - "ect}/{id.task_id.domain}/{id.task_id.nam" + - "e}/{id.task_id.version}/{id.retry_attemp" + - "t}Z\310\002\022\305\002/api/v1/task_executions/org/{id." + - "node_execution_id.execution_id.org}/{id." + - "node_execution_id.execution_id.project}/" + - "{id.node_execution_id.execution_id.domai" + - "n}/{id.node_execution_id.execution_id.na" + - "me}/{id.node_execution_id.node_id}/{id.t" + - "ask_id.project}/{id.task_id.domain}/{id." + - "task_id.name}/{id.task_id.version}/{id.r" + - "etry_attempt}\022\361\003\n\022ListTaskExecutions\022(.f" + - "lyteidl.admin.TaskExecutionListRequest\032!" + - ".flyteidl.admin.TaskExecutionList\"\215\003\202\323\344\223" + - "\002\206\003\022\252\001/api/v1/task_executions/{node_exec" + - "ution_id.execution_id.project}/{node_exe" + - "cution_id.execution_id.domain}/{node_exe" + - "cution_id.execution_id.name}/{node_execu" + - "tion_id.node_id}Z\326\001\022\323\001/api/v1/task_execu" + - "tions/org/{node_execution_id.execution_i" + - "d.org}/{node_execution_id.execution_id.p" + + "ution_id.name}/{id.node_id}\022\236\003\n\026GetDynam" + + "icNodeWorkflow\022-.flyteidl.admin.GetDynam" + + "icNodeWorkflowRequest\032+.flyteidl.admin.D" + + "ynamicNodeWorkflowResponse\"\247\002\202\323\344\223\002\240\002\022\177/a" + + "pi/v1/node_executions/{id.execution_id.p" + + "roject}/{id.execution_id.domain}/{id.exe" + + "cution_id.name}/{id.node_id}/dynamic_wor" + + "kflowZ\234\001\022\231\001/api/v1/node_executions/org/{" + + "id.execution_id.org}/{id.execution_id.pr" + + "oject}/{id.execution_id.domain}/{id.exec" + + "ution_id.name}/{id.node_id}/dynamic_work" + + "flow\022\371\002\n\022ListNodeExecutions\022(.flyteidl.a" + + "dmin.NodeExecutionListRequest\032!.flyteidl" + + ".admin.NodeExecutionList\"\225\002\202\323\344\223\002\216\002\022s/api" + + "/v1/node_executions/{workflow_execution_" + + "id.project}/{workflow_execution_id.domai" + + "n}/{workflow_execution_id.name}Z\226\001\022\223\001/ap" + + "i/v1/node_executions/org/{workflow_execu" + + "tion_id.org}/{workflow_execution_id.proj" + + "ect}/{workflow_execution_id.domain}/{wor" + + "kflow_execution_id.name}\022\217\010\n\031ListNodeExe" + + "cutionsForTask\022/.flyteidl.admin.NodeExec" + + "utionForTaskListRequest\032!.flyteidl.admin" + + ".NodeExecutionList\"\235\007\202\323\344\223\002\226\007\022\251\003/api/v1/c" + + "hildren/task_executions/{task_execution_" + + "id.node_execution_id.execution_id.projec" + + "t}/{task_execution_id.node_execution_id." + + "execution_id.domain}/{task_execution_id." + + "node_execution_id.execution_id.name}/{ta" + + "sk_execution_id.node_execution_id.node_i" + + "d}/{task_execution_id.task_id.project}/{" + + "task_execution_id.task_id.domain}/{task_" + + "execution_id.task_id.name}/{task_executi" + + "on_id.task_id.version}/{task_execution_i" + + "d.retry_attempt}Z\347\003\022\344\003/api/v1/children/t" + + "ask_executions/org/{task_execution_id.no" + + "de_execution_id.execution_id.org}/{task_" + + "execution_id.node_execution_id.execution" + + "_id.project}/{task_execution_id.node_exe" + + "cution_id.execution_id.domain}/{task_exe" + + "cution_id.node_execution_id.execution_id" + + ".name}/{task_execution_id.node_execution" + + "_id.node_id}/{task_execution_id.task_id." + + "project}/{task_execution_id.task_id.doma" + + "in}/{task_execution_id.task_id.name}/{ta" + + "sk_execution_id.task_id.version}/{task_e" + + "xecution_id.retry_attempt}\022\203\003\n\024GetNodeEx" + + "ecutionData\022+.flyteidl.admin.NodeExecuti" + + "onGetDataRequest\032,.flyteidl.admin.NodeEx" + + "ecutionGetDataResponse\"\217\002\202\323\344\223\002\210\002\022s/api/v" + + "1/data/node_executions/{id.execution_id." + + "project}/{id.execution_id.domain}/{id.ex" + + "ecution_id.name}/{id.node_id}Z\220\001\022\215\001/api/" + + "v1/data/node_executions/org/{id.executio" + + "n_id.org}/{id.execution_id.project}/{id." + + "execution_id.domain}/{id.execution_id.na" + + "me}/{id.node_id}\022\250\001\n\017RegisterProject\022&.f" + + "lyteidl.admin.ProjectRegisterRequest\032\'.f" + + "lyteidl.admin.ProjectRegisterResponse\"D\202" + + "\323\344\223\002>\"\020/api/v1/projects:\001*Z\'\"\"/api/v1/pr" + + "ojects/org/{project.org}:\001*\022\227\001\n\rUpdatePr" + + "oject\022\027.flyteidl.admin.Project\032%.flyteid" + + "l.admin.ProjectUpdateResponse\"F\202\323\344\223\002@\032\025/" + + "api/v1/projects/{id}:\001*Z$\032\037/api/v1/proje" + + "cts/org/{org}/{id}:\001*\022\204\001\n\014ListProjects\022\"" + + ".flyteidl.admin.ProjectListRequest\032\030.fly" + + "teidl.admin.Projects\"6\202\323\344\223\0020\022\020/api/v1/pr" + + "ojectsZ\034\022\032/api/v1/projects/org/{org}\022\325\001\n" + + "\023CreateWorkflowEvent\022-.flyteidl.admin.Wo" + + "rkflowExecutionEventRequest\032..flyteidl.a" + + "dmin.WorkflowExecutionEventResponse\"_\202\323\344" + + "\223\002Y\"\030/api/v1/events/workflows:\001*Z:\"5/api" + + "/v1/events/org/{event.execution_id.org}/" + + "workflows:\001*\022\304\001\n\017CreateNodeEvent\022).flyte" + + "idl.admin.NodeExecutionEventRequest\032*.fl" + + "yteidl.admin.NodeExecutionEventResponse\"" + + "Z\202\323\344\223\002T\"\024/api/v1/events/nodes:\001*Z9\"4/api" + + "/v1/events/org/{event.id.execution_id.or" + + "g}/nodes:\001*\022\332\001\n\017CreateTaskEvent\022).flytei" + + "dl.admin.TaskExecutionEventRequest\032*.fly" + + "teidl.admin.TaskExecutionEventResponse\"p" + + "\202\323\344\223\002j\"\024/api/v1/events/tasks:\001*ZO\"J/api/" + + "v1/events/org/{event.parent_node_executi" + + "on_id.execution_id.org}/tasks:\001*\022\313\005\n\020Get" + + "TaskExecution\022\'.flyteidl.admin.TaskExecu" + + "tionGetRequest\032\035.flyteidl.admin.TaskExec" + + "ution\"\356\004\202\323\344\223\002\347\004\022\231\002/api/v1/task_execution" + + "s/{id.node_execution_id.execution_id.pro" + + "ject}/{id.node_execution_id.execution_id" + + ".domain}/{id.node_execution_id.execution" + + "_id.name}/{id.node_execution_id.node_id}" + + "/{id.task_id.project}/{id.task_id.domain" + + "}/{id.task_id.name}/{id.task_id.version}" + + "/{id.retry_attempt}Z\310\002\022\305\002/api/v1/task_ex" + + "ecutions/org/{id.node_execution_id.execu" + + "tion_id.org}/{id.node_execution_id.execu" + + "tion_id.project}/{id.node_execution_id.e" + + "xecution_id.domain}/{id.node_execution_i" + + "d.execution_id.name}/{id.node_execution_" + + "id.node_id}/{id.task_id.project}/{id.tas" + + "k_id.domain}/{id.task_id.name}/{id.task_" + + "id.version}/{id.retry_attempt}\022\361\003\n\022ListT" + + "askExecutions\022(.flyteidl.admin.TaskExecu" + + "tionListRequest\032!.flyteidl.admin.TaskExe" + + "cutionList\"\215\003\202\323\344\223\002\206\003\022\252\001/api/v1/task_exec" + + "utions/{node_execution_id.execution_id.p" + "roject}/{node_execution_id.execution_id." + "domain}/{node_execution_id.execution_id." + - "name}/{node_execution_id.node_id}\022\354\005\n\024Ge" + - "tTaskExecutionData\022+.flyteidl.admin.Task" + - "ExecutionGetDataRequest\032,.flyteidl.admin" + - ".TaskExecutionGetDataResponse\"\370\004\202\323\344\223\002\361\004\022" + - "\236\002/api/v1/data/task_executions/{id.node_" + - "execution_id.execution_id.project}/{id.n" + - "ode_execution_id.execution_id.domain}/{i" + - "d.node_execution_id.execution_id.name}/{" + - "id.node_execution_id.node_id}/{id.task_i" + - "d.project}/{id.task_id.domain}/{id.task_" + - "id.name}/{id.task_id.version}/{id.retry_" + - "attempt}Z\315\002\022\312\002/api/v1/data/task_executio" + - "ns/org/{id.node_execution_id.execution_i" + - "d.org}/{id.node_execution_id.execution_i" + - "d.project}/{id.node_execution_id.executi" + - "on_id.domain}/{id.node_execution_id.exec" + - "ution_id.name}/{id.node_execution_id.nod" + - "e_id}/{id.task_id.project}/{id.task_id.d" + - "omain}/{id.task_id.name}/{id.task_id.ver" + - "sion}/{id.retry_attempt}\022\313\002\n\035UpdateProje" + - "ctDomainAttributes\0224.flyteidl.admin.Proj" + - "ectDomainAttributesUpdateRequest\0325.flyte" + - "idl.admin.ProjectDomainAttributesUpdateR" + - "esponse\"\274\001\202\323\344\223\002\265\001\032J/api/v1/project_domai" + - "n_attributes/{attributes.project}/{attri" + - "butes.domain}:\001*Zd\032_/api/v1/project_doma" + - "in_attributes/org/{attributes.org}/{attr" + - "ibutes.project}/{attributes.domain}:\001*\022\203" + - "\002\n\032GetProjectDomainAttributes\0221.flyteidl" + - ".admin.ProjectDomainAttributesGetRequest" + - "\0322.flyteidl.admin.ProjectDomainAttribute" + - "sGetResponse\"~\202\323\344\223\002x\0224/api/v1/project_do" + - "main_attributes/{project}/{domain}Z@\022>/a" + - "pi/v1/project_domain_attributes/org/{org" + - "}/{project}/{domain}\022\223\002\n\035DeleteProjectDo" + - "mainAttributes\0224.flyteidl.admin.ProjectD" + - "omainAttributesDeleteRequest\0325.flyteidl." + - "admin.ProjectDomainAttributesDeleteRespo" + - "nse\"\204\001\202\323\344\223\002~*4/api/v1/project_domain_att" + - "ributes/{project}/{domain}:\001*ZC*>/api/v1" + - "/project_domain_attributes/org/{org}/{pr" + - "oject}/{domain}:\001*\022\212\002\n\027UpdateProjectAttr" + - "ibutes\022..flyteidl.admin.ProjectAttribute" + - "sUpdateRequest\032/.flyteidl.admin.ProjectA" + - "ttributesUpdateResponse\"\215\001\202\323\344\223\002\206\001\032//api/" + - "v1/project_attributes/{attributes.projec" + - "t}:\001*ZP\032K/api/v1/project_domain_attribut" + - "es/org/{attributes.org}/{attributes.proj" + - "ect}:\001*\022\330\001\n\024GetProjectAttributes\022+.flyte" + - "idl.admin.ProjectAttributesGetRequest\032,." + - "flyteidl.admin.ProjectAttributesGetRespo" + - "nse\"e\202\323\344\223\002_\022$/api/v1/project_attributes/" + - "{project}Z7\0225/api/v1/project_domain_attr" + - "ibutes/org/{org}/{project}\022\347\001\n\027DeletePro" + - "jectAttributes\022..flyteidl.admin.ProjectA" + - "ttributesDeleteRequest\032/.flyteidl.admin." + - "ProjectAttributesDeleteResponse\"k\202\323\344\223\002e*" + - "$/api/v1/project_attributes/{project}:\001*" + - "Z:*5/api/v1/project_domain_attributes/or" + - "g/{org}/{project}:\001*\022\334\002\n\030UpdateWorkflowA" + - "ttributes\022/.flyteidl.admin.WorkflowAttri" + - "butesUpdateRequest\0320.flyteidl.admin.Work" + - "flowAttributesUpdateResponse\"\334\001\202\323\344\223\002\325\001\032Z" + - "/api/v1/workflow_attributes/{attributes." + - "project}/{attributes.domain}/{attributes" + - ".workflow}:\001*Zt\032o/api/v1/workflow_attrib" + - "utes/org/{attributes.org}/{attributes.pr" + - "oject}/{attributes.domain}/{attributes.w" + - "orkflow}:\001*\022\200\002\n\025GetWorkflowAttributes\022,." + - "flyteidl.admin.WorkflowAttributesGetRequ" + - "est\032-.flyteidl.admin.WorkflowAttributesG" + - "etResponse\"\211\001\202\323\344\223\002\202\001\0229/api/v1/workflow_a" + - "ttributes/{project}/{domain}/{workflow}Z" + - "E\022C/api/v1/workflow_attributes/org/{org}" + - "/{project}/{domain}/{workflow}\022\217\002\n\030Delet" + - "eWorkflowAttributes\022/.flyteidl.admin.Wor" + - "kflowAttributesDeleteRequest\0320.flyteidl." + - "admin.WorkflowAttributesDeleteResponse\"\217" + - "\001\202\323\344\223\002\210\001*9/api/v1/workflow_attributes/{p" + - "roject}/{domain}/{workflow}:\001*ZH*C/api/v" + - "1/workflow_attributes/org/{org}/{project" + - "}/{domain}/{workflow}:\001*\022\312\001\n\027ListMatchab" + - "leAttributes\022..flyteidl.admin.ListMatcha" + - "bleAttributesRequest\032/.flyteidl.admin.Li" + - "stMatchableAttributesResponse\"N\202\323\344\223\002H\022\034/" + - "api/v1/matchable_attributesZ(\022&/api/v1/m" + - "atchable_attributes/org/{org}\022\350\001\n\021ListNa" + - "medEntities\022&.flyteidl.admin.NamedEntity" + - "ListRequest\032\037.flyteidl.admin.NamedEntity" + - "List\"\211\001\202\323\344\223\002\202\001\0229/api/v1/named_entities/{" + - "resource_type}/{project}/{domain}ZE\022C/ap" + - "i/v1/named_entities/{resource_type}/org/" + - "{org}/{project}/{domain}\022\203\002\n\016GetNamedEnt" + - "ity\022%.flyteidl.admin.NamedEntityGetReque" + - "st\032\033.flyteidl.admin.NamedEntity\"\254\001\202\323\344\223\002\245" + - "\001\022I/api/v1/named_entities/{resource_type" + - "}/{id.project}/{id.domain}/{id.name}ZX\022V" + - "/api/v1/named_entities/{resource_type}/o" + - "rg/{id.org}/{id.project}/{id.domain}/{id" + - ".name}\022\235\002\n\021UpdateNamedEntity\022(.flyteidl." + - "admin.NamedEntityUpdateRequest\032).flyteid" + - "l.admin.NamedEntityUpdateResponse\"\262\001\202\323\344\223" + - "\002\253\001\032I/api/v1/named_entities/{resource_ty" + - "pe}/{id.project}/{id.domain}/{id.name}:\001" + - "*Z[\032V/api/v1/named_entities/{resource_ty" + - "pe}/org/{id.org}/{id.project}/{id.domain" + - "}/{id.name}:\001*\022l\n\nGetVersion\022!.flyteidl." + - "admin.GetVersionRequest\032\".flyteidl.admin" + - ".GetVersionResponse\"\027\202\323\344\223\002\021\022\017/api/v1/ver" + - "sion\022\266\002\n\024GetDescriptionEntity\022 .flyteidl" + - ".admin.ObjectGetRequest\032!.flyteidl.admin" + - ".DescriptionEntity\"\330\001\202\323\344\223\002\321\001\022_/api/v1/de" + - "scription_entities/{id.resource_type}/{i" + - "d.project}/{id.domain}/{id.name}/{id.ver" + - "sion}Zn\022l/api/v1/description_entities/or" + - "g/{id.org}/{id.resource_type}/{id.projec" + - "t}/{id.domain}/{id.name}/{id.version}\022\310\003" + - "\n\027ListDescriptionEntities\022,.flyteidl.adm" + - "in.DescriptionEntityListRequest\032%.flytei" + - "dl.admin.DescriptionEntityList\"\327\002\202\323\344\223\002\320\002" + - "\022O/api/v1/description_entities/{resource" + + "name}/{node_execution_id.node_id}Z\326\001\022\323\001/" + + "api/v1/task_executions/org/{node_executi" + + "on_id.execution_id.org}/{node_execution_" + + "id.execution_id.project}/{node_execution" + + "_id.execution_id.domain}/{node_execution" + + "_id.execution_id.name}/{node_execution_i" + + "d.node_id}\022\354\005\n\024GetTaskExecutionData\022+.fl" + + "yteidl.admin.TaskExecutionGetDataRequest" + + "\032,.flyteidl.admin.TaskExecutionGetDataRe" + + "sponse\"\370\004\202\323\344\223\002\361\004\022\236\002/api/v1/data/task_exe" + + "cutions/{id.node_execution_id.execution_" + + "id.project}/{id.node_execution_id.execut" + + "ion_id.domain}/{id.node_execution_id.exe" + + "cution_id.name}/{id.node_execution_id.no" + + "de_id}/{id.task_id.project}/{id.task_id." + + "domain}/{id.task_id.name}/{id.task_id.ve" + + "rsion}/{id.retry_attempt}Z\315\002\022\312\002/api/v1/d" + + "ata/task_executions/org/{id.node_executi" + + "on_id.execution_id.org}/{id.node_executi" + + "on_id.execution_id.project}/{id.node_exe" + + "cution_id.execution_id.domain}/{id.node_" + + "execution_id.execution_id.name}/{id.node" + + "_execution_id.node_id}/{id.task_id.proje" + + "ct}/{id.task_id.domain}/{id.task_id.name" + + "}/{id.task_id.version}/{id.retry_attempt" + + "}\022\313\002\n\035UpdateProjectDomainAttributes\0224.fl" + + "yteidl.admin.ProjectDomainAttributesUpda" + + "teRequest\0325.flyteidl.admin.ProjectDomain" + + "AttributesUpdateResponse\"\274\001\202\323\344\223\002\265\001\032J/api" + + "/v1/project_domain_attributes/{attribute" + + "s.project}/{attributes.domain}:\001*Zd\032_/ap" + + "i/v1/project_domain_attributes/org/{attr" + + "ibutes.org}/{attributes.project}/{attrib" + + "utes.domain}:\001*\022\203\002\n\032GetProjectDomainAttr" + + "ibutes\0221.flyteidl.admin.ProjectDomainAtt" + + "ributesGetRequest\0322.flyteidl.admin.Proje" + + "ctDomainAttributesGetResponse\"~\202\323\344\223\002x\0224/" + + "api/v1/project_domain_attributes/{projec" + + "t}/{domain}Z@\022>/api/v1/project_domain_at" + + "tributes/org/{org}/{project}/{domain}\022\223\002" + + "\n\035DeleteProjectDomainAttributes\0224.flytei" + + "dl.admin.ProjectDomainAttributesDeleteRe" + + "quest\0325.flyteidl.admin.ProjectDomainAttr" + + "ibutesDeleteResponse\"\204\001\202\323\344\223\002~*4/api/v1/p" + + "roject_domain_attributes/{project}/{doma" + + "in}:\001*ZC*>/api/v1/project_domain_attribu" + + "tes/org/{org}/{project}/{domain}:\001*\022\212\002\n\027" + + "UpdateProjectAttributes\022..flyteidl.admin" + + ".ProjectAttributesUpdateRequest\032/.flytei" + + "dl.admin.ProjectAttributesUpdateResponse" + + "\"\215\001\202\323\344\223\002\206\001\032//api/v1/project_attributes/{" + + "attributes.project}:\001*ZP\032K/api/v1/projec" + + "t_domain_attributes/org/{attributes.org}" + + "/{attributes.project}:\001*\022\330\001\n\024GetProjectA" + + "ttributes\022+.flyteidl.admin.ProjectAttrib" + + "utesGetRequest\032,.flyteidl.admin.ProjectA" + + "ttributesGetResponse\"e\202\323\344\223\002_\022$/api/v1/pr" + + "oject_attributes/{project}Z7\0225/api/v1/pr" + + "oject_domain_attributes/org/{org}/{proje" + + "ct}\022\347\001\n\027DeleteProjectAttributes\022..flytei" + + "dl.admin.ProjectAttributesDeleteRequest\032" + + "/.flyteidl.admin.ProjectAttributesDelete" + + "Response\"k\202\323\344\223\002e*$/api/v1/project_attrib" + + "utes/{project}:\001*Z:*5/api/v1/project_dom" + + "ain_attributes/org/{org}/{project}:\001*\022\334\002" + + "\n\030UpdateWorkflowAttributes\022/.flyteidl.ad" + + "min.WorkflowAttributesUpdateRequest\0320.fl" + + "yteidl.admin.WorkflowAttributesUpdateRes" + + "ponse\"\334\001\202\323\344\223\002\325\001\032Z/api/v1/workflow_attrib" + + "utes/{attributes.project}/{attributes.do" + + "main}/{attributes.workflow}:\001*Zt\032o/api/v" + + "1/workflow_attributes/org/{attributes.or" + + "g}/{attributes.project}/{attributes.doma" + + "in}/{attributes.workflow}:\001*\022\200\002\n\025GetWork" + + "flowAttributes\022,.flyteidl.admin.Workflow" + + "AttributesGetRequest\032-.flyteidl.admin.Wo" + + "rkflowAttributesGetResponse\"\211\001\202\323\344\223\002\202\001\0229/" + + "api/v1/workflow_attributes/{project}/{do" + + "main}/{workflow}ZE\022C/api/v1/workflow_att" + + "ributes/org/{org}/{project}/{domain}/{wo" + + "rkflow}\022\217\002\n\030DeleteWorkflowAttributes\022/.f" + + "lyteidl.admin.WorkflowAttributesDeleteRe" + + "quest\0320.flyteidl.admin.WorkflowAttribute" + + "sDeleteResponse\"\217\001\202\323\344\223\002\210\001*9/api/v1/workf" + + "low_attributes/{project}/{domain}/{workf" + + "low}:\001*ZH*C/api/v1/workflow_attributes/o" + + "rg/{org}/{project}/{domain}/{workflow}:\001" + + "*\022\312\001\n\027ListMatchableAttributes\022..flyteidl" + + ".admin.ListMatchableAttributesRequest\032/." + + "flyteidl.admin.ListMatchableAttributesRe" + + "sponse\"N\202\323\344\223\002H\022\034/api/v1/matchable_attrib" + + "utesZ(\022&/api/v1/matchable_attributes/org" + + "/{org}\022\350\001\n\021ListNamedEntities\022&.flyteidl." + + "admin.NamedEntityListRequest\032\037.flyteidl." + + "admin.NamedEntityList\"\211\001\202\323\344\223\002\202\001\0229/api/v1" + + "/named_entities/{resource_type}/{project" + + "}/{domain}ZE\022C/api/v1/named_entities/{re" + + "source_type}/org/{org}/{project}/{domain" + + "}\022\203\002\n\016GetNamedEntity\022%.flyteidl.admin.Na" + + "medEntityGetRequest\032\033.flyteidl.admin.Nam" + + "edEntity\"\254\001\202\323\344\223\002\245\001\022I/api/v1/named_entiti" + + "es/{resource_type}/{id.project}/{id.doma" + + "in}/{id.name}ZX\022V/api/v1/named_entities/" + + "{resource_type}/org/{id.org}/{id.project" + + "}/{id.domain}/{id.name}\022\235\002\n\021UpdateNamedE" + + "ntity\022(.flyteidl.admin.NamedEntityUpdate" + + "Request\032).flyteidl.admin.NamedEntityUpda" + + "teResponse\"\262\001\202\323\344\223\002\253\001\032I/api/v1/named_enti" + + "ties/{resource_type}/{id.project}/{id.do" + + "main}/{id.name}:\001*Z[\032V/api/v1/named_enti" + + "ties/{resource_type}/org/{id.org}/{id.pr" + + "oject}/{id.domain}/{id.name}:\001*\022l\n\nGetVe" + + "rsion\022!.flyteidl.admin.GetVersionRequest" + + "\032\".flyteidl.admin.GetVersionResponse\"\027\202\323" + + "\344\223\002\021\022\017/api/v1/version\022\266\002\n\024GetDescription" + + "Entity\022 .flyteidl.admin.ObjectGetRequest" + + "\032!.flyteidl.admin.DescriptionEntity\"\330\001\202\323" + + "\344\223\002\321\001\022_/api/v1/description_entities/{id." + + "resource_type}/{id.project}/{id.domain}/" + + "{id.name}/{id.version}Zn\022l/api/v1/descri" + + "ption_entities/org/{id.org}/{id.resource" + "_type}/{id.project}/{id.domain}/{id.name" + - "}Z^\022\\/api/v1/description_entities/{resou" + - "rce_type}/org/{id.org}/{id.project}/{id." + - "domain}/{id.name}ZG\022E/api/v1/description" + - "_entities/{resource_type}/{id.project}/{" + - "id.domain}ZT\022R/api/v1/description_entiti" + - "es/{resource_type}/org/{id.org}/{id.proj" + - "ect}/{id.domain}\022\225\002\n\023GetExecutionMetrics" + - "\0222.flyteidl.admin.WorkflowExecutionGetMe" + - "tricsRequest\0323.flyteidl.admin.WorkflowEx" + - "ecutionGetMetricsResponse\"\224\001\202\323\344\223\002\215\001\022=/ap" + - "i/v1/metrics/executions/{id.project}/{id" + - ".domain}/{id.name}ZL\022J/api/v1/metrics/ex" + - "ecutions/org/{id.org}/{id.project}/{id.d" + - "omain}/{id.name}B?Z=github.com/flyteorg/" + - "flyte/flyteidl/gen/pb-go/flyteidl/servic" + - "eb\006proto3" + "}/{id.version}\022\310\003\n\027ListDescriptionEntiti" + + "es\022,.flyteidl.admin.DescriptionEntityLis" + + "tRequest\032%.flyteidl.admin.DescriptionEnt" + + "ityList\"\327\002\202\323\344\223\002\320\002\022O/api/v1/description_e" + + "ntities/{resource_type}/{id.project}/{id" + + ".domain}/{id.name}Z^\022\\/api/v1/descriptio" + + "n_entities/{resource_type}/org/{id.org}/" + + "{id.project}/{id.domain}/{id.name}ZG\022E/a" + + "pi/v1/description_entities/{resource_typ" + + "e}/{id.project}/{id.domain}ZT\022R/api/v1/d" + + "escription_entities/{resource_type}/org/" + + "{id.org}/{id.project}/{id.domain}\022\225\002\n\023Ge" + + "tExecutionMetrics\0222.flyteidl.admin.Workf" + + "lowExecutionGetMetricsRequest\0323.flyteidl" + + ".admin.WorkflowExecutionGetMetricsRespon", + "se\"\224\001\202\323\344\223\002\215\001\022=/api/v1/metrics/executions" + + "/{id.project}/{id.domain}/{id.name}ZL\022J/" + + "api/v1/metrics/executions/org/{id.org}/{" + + "id.project}/{id.domain}/{id.name}B?Z=git" + + "hub.com/flyteorg/flyte/flyteidl/gen/pb-g" + + "o/flyteidl/serviceb\006proto3" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { diff --git a/flyteidl/gen/pb-js/flyteidl.d.ts b/flyteidl/gen/pb-js/flyteidl.d.ts index 8d9f8ed346..5379ab1f3b 100644 --- a/flyteidl/gen/pb-js/flyteidl.d.ts +++ b/flyteidl/gen/pb-js/flyteidl.d.ts @@ -16439,6 +16439,110 @@ export namespace flyteidl { public static verify(message: { [k: string]: any }): (string|null); } + /** Properties of a GetDynamicNodeWorkflowRequest. */ + interface IGetDynamicNodeWorkflowRequest { + + /** GetDynamicNodeWorkflowRequest id */ + id?: (flyteidl.core.INodeExecutionIdentifier|null); + } + + /** Represents a GetDynamicNodeWorkflowRequest. */ + class GetDynamicNodeWorkflowRequest implements IGetDynamicNodeWorkflowRequest { + + /** + * Constructs a new GetDynamicNodeWorkflowRequest. + * @param [properties] Properties to set + */ + constructor(properties?: flyteidl.admin.IGetDynamicNodeWorkflowRequest); + + /** GetDynamicNodeWorkflowRequest id. */ + public id?: (flyteidl.core.INodeExecutionIdentifier|null); + + /** + * Creates a new GetDynamicNodeWorkflowRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns GetDynamicNodeWorkflowRequest instance + */ + public static create(properties?: flyteidl.admin.IGetDynamicNodeWorkflowRequest): flyteidl.admin.GetDynamicNodeWorkflowRequest; + + /** + * Encodes the specified GetDynamicNodeWorkflowRequest message. Does not implicitly {@link flyteidl.admin.GetDynamicNodeWorkflowRequest.verify|verify} messages. + * @param message GetDynamicNodeWorkflowRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: flyteidl.admin.IGetDynamicNodeWorkflowRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a GetDynamicNodeWorkflowRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns GetDynamicNodeWorkflowRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): flyteidl.admin.GetDynamicNodeWorkflowRequest; + + /** + * Verifies a GetDynamicNodeWorkflowRequest message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + } + + /** Properties of a DynamicNodeWorkflowResponse. */ + interface IDynamicNodeWorkflowResponse { + + /** DynamicNodeWorkflowResponse compiledWorkflow */ + compiledWorkflow?: (flyteidl.core.ICompiledWorkflowClosure|null); + } + + /** Represents a DynamicNodeWorkflowResponse. */ + class DynamicNodeWorkflowResponse implements IDynamicNodeWorkflowResponse { + + /** + * Constructs a new DynamicNodeWorkflowResponse. + * @param [properties] Properties to set + */ + constructor(properties?: flyteidl.admin.IDynamicNodeWorkflowResponse); + + /** DynamicNodeWorkflowResponse compiledWorkflow. */ + public compiledWorkflow?: (flyteidl.core.ICompiledWorkflowClosure|null); + + /** + * Creates a new DynamicNodeWorkflowResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns DynamicNodeWorkflowResponse instance + */ + public static create(properties?: flyteidl.admin.IDynamicNodeWorkflowResponse): flyteidl.admin.DynamicNodeWorkflowResponse; + + /** + * Encodes the specified DynamicNodeWorkflowResponse message. Does not implicitly {@link flyteidl.admin.DynamicNodeWorkflowResponse.verify|verify} messages. + * @param message DynamicNodeWorkflowResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: flyteidl.admin.IDynamicNodeWorkflowResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a DynamicNodeWorkflowResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns DynamicNodeWorkflowResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): flyteidl.admin.DynamicNodeWorkflowResponse; + + /** + * Verifies a DynamicNodeWorkflowResponse message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + } + /** Properties of an EmailMessage. */ interface IEmailMessage { @@ -20479,6 +20583,20 @@ export namespace flyteidl { */ public getNodeExecution(request: flyteidl.admin.INodeExecutionGetRequest): Promise; + /** + * Calls GetDynamicNodeWorkflow. + * @param request GetDynamicNodeWorkflowRequest message or plain object + * @param callback Node-style callback called with the error, if any, and DynamicNodeWorkflowResponse + */ + public getDynamicNodeWorkflow(request: flyteidl.admin.IGetDynamicNodeWorkflowRequest, callback: flyteidl.service.AdminService.GetDynamicNodeWorkflowCallback): void; + + /** + * Calls GetDynamicNodeWorkflow. + * @param request GetDynamicNodeWorkflowRequest message or plain object + * @returns Promise + */ + public getDynamicNodeWorkflow(request: flyteidl.admin.IGetDynamicNodeWorkflowRequest): Promise; + /** * Calls ListNodeExecutions. * @param request NodeExecutionListRequest message or plain object @@ -21056,6 +21174,13 @@ export namespace flyteidl { */ type GetNodeExecutionCallback = (error: (Error|null), response?: flyteidl.admin.NodeExecution) => void; + /** + * Callback as used by {@link flyteidl.service.AdminService#getDynamicNodeWorkflow}. + * @param error Error, if any + * @param [response] DynamicNodeWorkflowResponse + */ + type GetDynamicNodeWorkflowCallback = (error: (Error|null), response?: flyteidl.admin.DynamicNodeWorkflowResponse) => void; + /** * Callback as used by {@link flyteidl.service.AdminService#listNodeExecutions}. * @param error Error, if any diff --git a/flyteidl/gen/pb-js/flyteidl.js b/flyteidl/gen/pb-js/flyteidl.js index 3de32e71c4..3cd10753fe 100644 --- a/flyteidl/gen/pb-js/flyteidl.js +++ b/flyteidl/gen/pb-js/flyteidl.js @@ -39758,6 +39758,230 @@ return NodeExecutionGetDataResponse; })(); + admin.GetDynamicNodeWorkflowRequest = (function() { + + /** + * Properties of a GetDynamicNodeWorkflowRequest. + * @memberof flyteidl.admin + * @interface IGetDynamicNodeWorkflowRequest + * @property {flyteidl.core.INodeExecutionIdentifier|null} [id] GetDynamicNodeWorkflowRequest id + */ + + /** + * Constructs a new GetDynamicNodeWorkflowRequest. + * @memberof flyteidl.admin + * @classdesc Represents a GetDynamicNodeWorkflowRequest. + * @implements IGetDynamicNodeWorkflowRequest + * @constructor + * @param {flyteidl.admin.IGetDynamicNodeWorkflowRequest=} [properties] Properties to set + */ + function GetDynamicNodeWorkflowRequest(properties) { + if (properties) + for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * GetDynamicNodeWorkflowRequest id. + * @member {flyteidl.core.INodeExecutionIdentifier|null|undefined} id + * @memberof flyteidl.admin.GetDynamicNodeWorkflowRequest + * @instance + */ + GetDynamicNodeWorkflowRequest.prototype.id = null; + + /** + * Creates a new GetDynamicNodeWorkflowRequest instance using the specified properties. + * @function create + * @memberof flyteidl.admin.GetDynamicNodeWorkflowRequest + * @static + * @param {flyteidl.admin.IGetDynamicNodeWorkflowRequest=} [properties] Properties to set + * @returns {flyteidl.admin.GetDynamicNodeWorkflowRequest} GetDynamicNodeWorkflowRequest instance + */ + GetDynamicNodeWorkflowRequest.create = function create(properties) { + return new GetDynamicNodeWorkflowRequest(properties); + }; + + /** + * Encodes the specified GetDynamicNodeWorkflowRequest message. Does not implicitly {@link flyteidl.admin.GetDynamicNodeWorkflowRequest.verify|verify} messages. + * @function encode + * @memberof flyteidl.admin.GetDynamicNodeWorkflowRequest + * @static + * @param {flyteidl.admin.IGetDynamicNodeWorkflowRequest} message GetDynamicNodeWorkflowRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + GetDynamicNodeWorkflowRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && message.hasOwnProperty("id")) + $root.flyteidl.core.NodeExecutionIdentifier.encode(message.id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Decodes a GetDynamicNodeWorkflowRequest message from the specified reader or buffer. + * @function decode + * @memberof flyteidl.admin.GetDynamicNodeWorkflowRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {flyteidl.admin.GetDynamicNodeWorkflowRequest} GetDynamicNodeWorkflowRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + GetDynamicNodeWorkflowRequest.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + var end = length === undefined ? reader.len : reader.pos + length, message = new $root.flyteidl.admin.GetDynamicNodeWorkflowRequest(); + while (reader.pos < end) { + var tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + message.id = $root.flyteidl.core.NodeExecutionIdentifier.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Verifies a GetDynamicNodeWorkflowRequest message. + * @function verify + * @memberof flyteidl.admin.GetDynamicNodeWorkflowRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + GetDynamicNodeWorkflowRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) { + var error = $root.flyteidl.core.NodeExecutionIdentifier.verify(message.id); + if (error) + return "id." + error; + } + return null; + }; + + return GetDynamicNodeWorkflowRequest; + })(); + + admin.DynamicNodeWorkflowResponse = (function() { + + /** + * Properties of a DynamicNodeWorkflowResponse. + * @memberof flyteidl.admin + * @interface IDynamicNodeWorkflowResponse + * @property {flyteidl.core.ICompiledWorkflowClosure|null} [compiledWorkflow] DynamicNodeWorkflowResponse compiledWorkflow + */ + + /** + * Constructs a new DynamicNodeWorkflowResponse. + * @memberof flyteidl.admin + * @classdesc Represents a DynamicNodeWorkflowResponse. + * @implements IDynamicNodeWorkflowResponse + * @constructor + * @param {flyteidl.admin.IDynamicNodeWorkflowResponse=} [properties] Properties to set + */ + function DynamicNodeWorkflowResponse(properties) { + if (properties) + for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * DynamicNodeWorkflowResponse compiledWorkflow. + * @member {flyteidl.core.ICompiledWorkflowClosure|null|undefined} compiledWorkflow + * @memberof flyteidl.admin.DynamicNodeWorkflowResponse + * @instance + */ + DynamicNodeWorkflowResponse.prototype.compiledWorkflow = null; + + /** + * Creates a new DynamicNodeWorkflowResponse instance using the specified properties. + * @function create + * @memberof flyteidl.admin.DynamicNodeWorkflowResponse + * @static + * @param {flyteidl.admin.IDynamicNodeWorkflowResponse=} [properties] Properties to set + * @returns {flyteidl.admin.DynamicNodeWorkflowResponse} DynamicNodeWorkflowResponse instance + */ + DynamicNodeWorkflowResponse.create = function create(properties) { + return new DynamicNodeWorkflowResponse(properties); + }; + + /** + * Encodes the specified DynamicNodeWorkflowResponse message. Does not implicitly {@link flyteidl.admin.DynamicNodeWorkflowResponse.verify|verify} messages. + * @function encode + * @memberof flyteidl.admin.DynamicNodeWorkflowResponse + * @static + * @param {flyteidl.admin.IDynamicNodeWorkflowResponse} message DynamicNodeWorkflowResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + DynamicNodeWorkflowResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.compiledWorkflow != null && message.hasOwnProperty("compiledWorkflow")) + $root.flyteidl.core.CompiledWorkflowClosure.encode(message.compiledWorkflow, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Decodes a DynamicNodeWorkflowResponse message from the specified reader or buffer. + * @function decode + * @memberof flyteidl.admin.DynamicNodeWorkflowResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {flyteidl.admin.DynamicNodeWorkflowResponse} DynamicNodeWorkflowResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + DynamicNodeWorkflowResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + var end = length === undefined ? reader.len : reader.pos + length, message = new $root.flyteidl.admin.DynamicNodeWorkflowResponse(); + while (reader.pos < end) { + var tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + message.compiledWorkflow = $root.flyteidl.core.CompiledWorkflowClosure.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Verifies a DynamicNodeWorkflowResponse message. + * @function verify + * @memberof flyteidl.admin.DynamicNodeWorkflowResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + DynamicNodeWorkflowResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.compiledWorkflow != null && message.hasOwnProperty("compiledWorkflow")) { + var error = $root.flyteidl.core.CompiledWorkflowClosure.verify(message.compiledWorkflow); + if (error) + return "compiledWorkflow." + error; + } + return null; + }; + + return DynamicNodeWorkflowResponse; + })(); + admin.EmailMessage = (function() { /** @@ -49069,6 +49293,39 @@ * @variation 2 */ + /** + * Callback as used by {@link flyteidl.service.AdminService#getDynamicNodeWorkflow}. + * @memberof flyteidl.service.AdminService + * @typedef GetDynamicNodeWorkflowCallback + * @type {function} + * @param {Error|null} error Error, if any + * @param {flyteidl.admin.DynamicNodeWorkflowResponse} [response] DynamicNodeWorkflowResponse + */ + + /** + * Calls GetDynamicNodeWorkflow. + * @function getDynamicNodeWorkflow + * @memberof flyteidl.service.AdminService + * @instance + * @param {flyteidl.admin.IGetDynamicNodeWorkflowRequest} request GetDynamicNodeWorkflowRequest message or plain object + * @param {flyteidl.service.AdminService.GetDynamicNodeWorkflowCallback} callback Node-style callback called with the error, if any, and DynamicNodeWorkflowResponse + * @returns {undefined} + * @variation 1 + */ + Object.defineProperty(AdminService.prototype.getDynamicNodeWorkflow = function getDynamicNodeWorkflow(request, callback) { + return this.rpcCall(getDynamicNodeWorkflow, $root.flyteidl.admin.GetDynamicNodeWorkflowRequest, $root.flyteidl.admin.DynamicNodeWorkflowResponse, request, callback); + }, "name", { value: "GetDynamicNodeWorkflow" }); + + /** + * Calls GetDynamicNodeWorkflow. + * @function getDynamicNodeWorkflow + * @memberof flyteidl.service.AdminService + * @instance + * @param {flyteidl.admin.IGetDynamicNodeWorkflowRequest} request GetDynamicNodeWorkflowRequest message or plain object + * @returns {Promise} Promise + * @variation 2 + */ + /** * Callback as used by {@link flyteidl.service.AdminService#listNodeExecutions}. * @memberof flyteidl.service.AdminService diff --git a/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.py index 17e5cbd652..93a29df4d6 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.py @@ -21,7 +21,7 @@ from google.protobuf import duration_pb2 as google_dot_protobuf_dot_duration__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n#flyteidl/admin/node_execution.proto\x12\x0e\x66lyteidl.admin\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\x1b\x66lyteidl/core/catalog.proto\x1a\x1c\x66lyteidl/core/compiler.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/duration.proto\"Q\n\x17NodeExecutionGetRequest\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\"\x99\x02\n\x18NodeExecutionListRequest\x12^\n\x15workflow_execution_id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x13workflowExecutionId\x12\x14\n\x05limit\x18\x02 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x03 \x01(\tR\x05token\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\x12(\n\x10unique_parent_id\x18\x06 \x01(\tR\x0euniqueParentId\"\xea\x01\n\x1fNodeExecutionForTaskListRequest\x12R\n\x11task_execution_id\x18\x01 \x01(\x0b\x32&.flyteidl.core.TaskExecutionIdentifierR\x0ftaskExecutionId\x12\x14\n\x05limit\x18\x02 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x03 \x01(\tR\x05token\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\"\xe7\x01\n\rNodeExecution\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\x12\x1b\n\tinput_uri\x18\x02 \x01(\tR\x08inputUri\x12>\n\x07\x63losure\x18\x03 \x01(\x0b\x32$.flyteidl.admin.NodeExecutionClosureR\x07\x63losure\x12\x41\n\x08metadata\x18\x04 \x01(\x0b\x32%.flyteidl.admin.NodeExecutionMetaDataR\x08metadata\"\xba\x01\n\x15NodeExecutionMetaData\x12\x1f\n\x0bretry_group\x18\x01 \x01(\tR\nretryGroup\x12$\n\x0eis_parent_node\x18\x02 \x01(\x08R\x0cisParentNode\x12 \n\x0cspec_node_id\x18\x03 \x01(\tR\nspecNodeId\x12\x1d\n\nis_dynamic\x18\x04 \x01(\x08R\tisDynamic\x12\x19\n\x08is_array\x18\x05 \x01(\x08R\x07isArray\"q\n\x11NodeExecutionList\x12\x46\n\x0fnode_executions\x18\x01 \x03(\x0b\x32\x1d.flyteidl.admin.NodeExecutionR\x0enodeExecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"\xf6\x05\n\x14NodeExecutionClosure\x12#\n\noutput_uri\x18\x01 \x01(\tB\x02\x18\x01H\x00R\toutputUri\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12@\n\x0boutput_data\x18\n \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x38\n\x05phase\x18\x03 \x01(\x0e\x32\".flyteidl.core.NodeExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x05 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\\\n\x16workflow_node_metadata\x18\x08 \x01(\x0b\x32$.flyteidl.admin.WorkflowNodeMetadataH\x01R\x14workflowNodeMetadata\x12P\n\x12task_node_metadata\x18\t \x01(\x0b\x32 .flyteidl.admin.TaskNodeMetadataH\x01R\x10taskNodeMetadata\x12\x19\n\x08\x64\x65\x63k_uri\x18\x0b \x01(\tR\x07\x64\x65\x63kUri\x12/\n\x14\x64ynamic_job_spec_uri\x18\x0c \x01(\tR\x11\x64ynamicJobSpecUriB\x0f\n\routput_resultB\x11\n\x0ftarget_metadata\"d\n\x14WorkflowNodeMetadata\x12L\n\x0b\x65xecutionId\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x0b\x65xecutionId\"\xc0\x01\n\x10TaskNodeMetadata\x12\x44\n\x0c\x63\x61\x63he_status\x18\x01 \x01(\x0e\x32!.flyteidl.core.CatalogCacheStatusR\x0b\x63\x61\x63heStatus\x12?\n\x0b\x63\x61talog_key\x18\x02 \x01(\x0b\x32\x1e.flyteidl.core.CatalogMetadataR\ncatalogKey\x12%\n\x0e\x63heckpoint_uri\x18\x04 \x01(\tR\rcheckpointUri\"\xce\x01\n\x1b\x44ynamicWorkflowNodeMetadata\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12S\n\x11\x63ompiled_workflow\x18\x02 \x01(\x0b\x32&.flyteidl.core.CompiledWorkflowClosureR\x10\x63ompiledWorkflow\x12/\n\x14\x64ynamic_job_spec_uri\x18\x03 \x01(\tR\x11\x64ynamicJobSpecUri\"U\n\x1bNodeExecutionGetDataRequest\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\"\x96\x03\n\x1cNodeExecutionGetDataResponse\x12\x33\n\x06inputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12\x35\n\x07outputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\x12V\n\x10\x64ynamic_workflow\x18\x10 \x01(\x0b\x32+.flyteidl.admin.DynamicWorkflowNodeMetadataR\x0f\x64ynamicWorkflow\x12\x38\n\nflyte_urls\x18\x11 \x01(\x0b\x32\x19.flyteidl.admin.FlyteURLsR\tflyteUrlsB\xbe\x01\n\x12\x63om.flyteidl.adminB\x12NodeExecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n#flyteidl/admin/node_execution.proto\x12\x0e\x66lyteidl.admin\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\x1b\x66lyteidl/core/catalog.proto\x1a\x1c\x66lyteidl/core/compiler.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/duration.proto\"Q\n\x17NodeExecutionGetRequest\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\"\x99\x02\n\x18NodeExecutionListRequest\x12^\n\x15workflow_execution_id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x13workflowExecutionId\x12\x14\n\x05limit\x18\x02 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x03 \x01(\tR\x05token\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\x12(\n\x10unique_parent_id\x18\x06 \x01(\tR\x0euniqueParentId\"\xea\x01\n\x1fNodeExecutionForTaskListRequest\x12R\n\x11task_execution_id\x18\x01 \x01(\x0b\x32&.flyteidl.core.TaskExecutionIdentifierR\x0ftaskExecutionId\x12\x14\n\x05limit\x18\x02 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x03 \x01(\tR\x05token\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\"\xe7\x01\n\rNodeExecution\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\x12\x1b\n\tinput_uri\x18\x02 \x01(\tR\x08inputUri\x12>\n\x07\x63losure\x18\x03 \x01(\x0b\x32$.flyteidl.admin.NodeExecutionClosureR\x07\x63losure\x12\x41\n\x08metadata\x18\x04 \x01(\x0b\x32%.flyteidl.admin.NodeExecutionMetaDataR\x08metadata\"\xba\x01\n\x15NodeExecutionMetaData\x12\x1f\n\x0bretry_group\x18\x01 \x01(\tR\nretryGroup\x12$\n\x0eis_parent_node\x18\x02 \x01(\x08R\x0cisParentNode\x12 \n\x0cspec_node_id\x18\x03 \x01(\tR\nspecNodeId\x12\x1d\n\nis_dynamic\x18\x04 \x01(\x08R\tisDynamic\x12\x19\n\x08is_array\x18\x05 \x01(\x08R\x07isArray\"q\n\x11NodeExecutionList\x12\x46\n\x0fnode_executions\x18\x01 \x03(\x0b\x32\x1d.flyteidl.admin.NodeExecutionR\x0enodeExecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"\xf6\x05\n\x14NodeExecutionClosure\x12#\n\noutput_uri\x18\x01 \x01(\tB\x02\x18\x01H\x00R\toutputUri\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12@\n\x0boutput_data\x18\n \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x38\n\x05phase\x18\x03 \x01(\x0e\x32\".flyteidl.core.NodeExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x05 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\\\n\x16workflow_node_metadata\x18\x08 \x01(\x0b\x32$.flyteidl.admin.WorkflowNodeMetadataH\x01R\x14workflowNodeMetadata\x12P\n\x12task_node_metadata\x18\t \x01(\x0b\x32 .flyteidl.admin.TaskNodeMetadataH\x01R\x10taskNodeMetadata\x12\x19\n\x08\x64\x65\x63k_uri\x18\x0b \x01(\tR\x07\x64\x65\x63kUri\x12/\n\x14\x64ynamic_job_spec_uri\x18\x0c \x01(\tR\x11\x64ynamicJobSpecUriB\x0f\n\routput_resultB\x11\n\x0ftarget_metadata\"d\n\x14WorkflowNodeMetadata\x12L\n\x0b\x65xecutionId\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x0b\x65xecutionId\"\xc0\x01\n\x10TaskNodeMetadata\x12\x44\n\x0c\x63\x61\x63he_status\x18\x01 \x01(\x0e\x32!.flyteidl.core.CatalogCacheStatusR\x0b\x63\x61\x63heStatus\x12?\n\x0b\x63\x61talog_key\x18\x02 \x01(\x0b\x32\x1e.flyteidl.core.CatalogMetadataR\ncatalogKey\x12%\n\x0e\x63heckpoint_uri\x18\x04 \x01(\tR\rcheckpointUri\"\xce\x01\n\x1b\x44ynamicWorkflowNodeMetadata\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12S\n\x11\x63ompiled_workflow\x18\x02 \x01(\x0b\x32&.flyteidl.core.CompiledWorkflowClosureR\x10\x63ompiledWorkflow\x12/\n\x14\x64ynamic_job_spec_uri\x18\x03 \x01(\tR\x11\x64ynamicJobSpecUri\"U\n\x1bNodeExecutionGetDataRequest\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\"\x96\x03\n\x1cNodeExecutionGetDataResponse\x12\x33\n\x06inputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12\x35\n\x07outputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\x12V\n\x10\x64ynamic_workflow\x18\x10 \x01(\x0b\x32+.flyteidl.admin.DynamicWorkflowNodeMetadataR\x0f\x64ynamicWorkflow\x12\x38\n\nflyte_urls\x18\x11 \x01(\x0b\x32\x19.flyteidl.admin.FlyteURLsR\tflyteUrls\"W\n\x1dGetDynamicNodeWorkflowRequest\x12\x36\n\x02id\x18\x01 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x02id\"r\n\x1b\x44ynamicNodeWorkflowResponse\x12S\n\x11\x63ompiled_workflow\x18\x01 \x01(\x0b\x32&.flyteidl.core.CompiledWorkflowClosureR\x10\x63ompiledWorkflowB\xbe\x01\n\x12\x63om.flyteidl.adminB\x12NodeExecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -62,4 +62,8 @@ _globals['_NODEEXECUTIONGETDATAREQUEST']._serialized_end=2795 _globals['_NODEEXECUTIONGETDATARESPONSE']._serialized_start=2798 _globals['_NODEEXECUTIONGETDATARESPONSE']._serialized_end=3204 + _globals['_GETDYNAMICNODEWORKFLOWREQUEST']._serialized_start=3206 + _globals['_GETDYNAMICNODEWORKFLOWREQUEST']._serialized_end=3293 + _globals['_DYNAMICNODEWORKFLOWRESPONSE']._serialized_start=3295 + _globals['_DYNAMICNODEWORKFLOWRESPONSE']._serialized_end=3409 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.pyi index 8cbb708f01..9bf601847d 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/node_execution_pb2.pyi @@ -158,3 +158,15 @@ class NodeExecutionGetDataResponse(_message.Message): dynamic_workflow: DynamicWorkflowNodeMetadata flyte_urls: _common_pb2.FlyteURLs def __init__(self, inputs: _Optional[_Union[_common_pb2.UrlBlob, _Mapping]] = ..., outputs: _Optional[_Union[_common_pb2.UrlBlob, _Mapping]] = ..., full_inputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., full_outputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., dynamic_workflow: _Optional[_Union[DynamicWorkflowNodeMetadata, _Mapping]] = ..., flyte_urls: _Optional[_Union[_common_pb2.FlyteURLs, _Mapping]] = ...) -> None: ... + +class GetDynamicNodeWorkflowRequest(_message.Message): + __slots__ = ["id"] + ID_FIELD_NUMBER: _ClassVar[int] + id: _identifier_pb2.NodeExecutionIdentifier + def __init__(self, id: _Optional[_Union[_identifier_pb2.NodeExecutionIdentifier, _Mapping]] = ...) -> None: ... + +class DynamicNodeWorkflowResponse(_message.Message): + __slots__ = ["compiled_workflow"] + COMPILED_WORKFLOW_FIELD_NUMBER: _ClassVar[int] + compiled_workflow: _compiler_pb2.CompiledWorkflowClosure + def __init__(self, compiled_workflow: _Optional[_Union[_compiler_pb2.CompiledWorkflowClosure, _Mapping]] = ...) -> None: ... diff --git a/flyteidl/gen/pb_python/flyteidl/service/admin_pb2.py b/flyteidl/gen/pb_python/flyteidl/service/admin_pb2.py index c7b34e85c6..7e71e9e9e5 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/admin_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/service/admin_pb2.py @@ -29,7 +29,7 @@ from flyteidl.admin import description_entity_pb2 as flyteidl_dot_admin_dot_description__entity__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1c\x66lyteidl/service/admin.proto\x12\x10\x66lyteidl.service\x1a\x1cgoogle/api/annotations.proto\x1a\x1c\x66lyteidl/admin/project.proto\x1a.flyteidl/admin/project_domain_attributes.proto\x1a\'flyteidl/admin/project_attributes.proto\x1a\x19\x66lyteidl/admin/task.proto\x1a\x1d\x66lyteidl/admin/workflow.proto\x1a(flyteidl/admin/workflow_attributes.proto\x1a flyteidl/admin/launch_plan.proto\x1a\x1a\x66lyteidl/admin/event.proto\x1a\x1e\x66lyteidl/admin/execution.proto\x1a\'flyteidl/admin/matchable_resource.proto\x1a#flyteidl/admin/node_execution.proto\x1a#flyteidl/admin/task_execution.proto\x1a\x1c\x66lyteidl/admin/version.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\'flyteidl/admin/description_entity.proto2\x97v\n\x0c\x41\x64minService\x12\x8e\x01\n\nCreateTask\x12!.flyteidl.admin.TaskCreateRequest\x1a\".flyteidl.admin.TaskCreateResponse\"9\x82\xd3\xe4\x93\x02\x33:\x01*Z\x1f:\x01*\"\x1a/api/v1/tasks/org/{id.org}\"\r/api/v1/tasks\x12\xd8\x01\n\x07GetTask\x12 .flyteidl.admin.ObjectGetRequest\x1a\x14.flyteidl.admin.Task\"\x94\x01\x82\xd3\xe4\x93\x02\x8d\x01ZL\x12J/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12=/api/v1/tasks/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xc5\x01\n\x0bListTaskIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"Y\x82\xd3\xe4\x93\x02SZ,\x12*/api/v1/tasks/org/{org}/{project}/{domain}\x12#/api/v1/task_ids/{project}/{domain}\x12\xa8\x02\n\tListTasks\x12#.flyteidl.admin.ResourceListRequest\x1a\x18.flyteidl.admin.TaskList\"\xdb\x01\x82\xd3\xe4\x93\x02\xd4\x01Z?\x12=/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}/{id.name}Z(\x12&/api/v1/tasks/{id.project}/{id.domain}Z5\x12\x33/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}\x12\x30/api/v1/tasks/{id.project}/{id.domain}/{id.name}\x12\xa2\x01\n\x0e\x43reateWorkflow\x12%.flyteidl.admin.WorkflowCreateRequest\x1a&.flyteidl.admin.WorkflowCreateResponse\"A\x82\xd3\xe4\x93\x02;:\x01*Z#:\x01*\"\x1e/api/v1/workflows/org/{id.org}\"\x11/api/v1/workflows\x12\xe8\x01\n\x0bGetWorkflow\x12 .flyteidl.admin.ObjectGetRequest\x1a\x18.flyteidl.admin.Workflow\"\x9c\x01\x82\xd3\xe4\x93\x02\x95\x01ZP\x12N/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\x41/api/v1/workflows/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xd1\x01\n\x0fListWorkflowIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"a\x82\xd3\xe4\x93\x02[Z0\x12./api/v1/workflows/org/{org}/{project}/{domain}\x12\'/api/v1/workflow_ids/{project}/{domain}\x12\xc0\x02\n\rListWorkflows\x12#.flyteidl.admin.ResourceListRequest\x1a\x1c.flyteidl.admin.WorkflowList\"\xeb\x01\x82\xd3\xe4\x93\x02\xe4\x01ZC\x12\x41/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}/{id.name}Z,\x12*/api/v1/workflows/{id.project}/{id.domain}Z9\x12\x37/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}\x12\x34/api/v1/workflows/{id.project}/{id.domain}/{id.name}\x12\xae\x01\n\x10\x43reateLaunchPlan\x12\'.flyteidl.admin.LaunchPlanCreateRequest\x1a(.flyteidl.admin.LaunchPlanCreateResponse\"G\x82\xd3\xe4\x93\x02\x41:\x01*Z&:\x01*\"!/api/v1/launch_plans/org/{id.org}\"\x14/api/v1/launch_plans\x12\xf2\x01\n\rGetLaunchPlan\x12 .flyteidl.admin.ObjectGetRequest\x1a\x1a.flyteidl.admin.LaunchPlan\"\xa2\x01\x82\xd3\xe4\x93\x02\x9b\x01ZS\x12Q/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\x44/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xf3\x01\n\x13GetActiveLaunchPlan\x12\'.flyteidl.admin.ActiveLaunchPlanRequest\x1a\x1a.flyteidl.admin.LaunchPlan\"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01ZM\x12K/api/v1/active_launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12>/api/v1/active_launch_plans/{id.project}/{id.domain}/{id.name}\x12\xd8\x01\n\x15ListActiveLaunchPlans\x12+.flyteidl.admin.ActiveLaunchPlanListRequest\x1a\x1e.flyteidl.admin.LaunchPlanList\"r\x82\xd3\xe4\x93\x02lZ:\x12\x38/api/v1/active_launch_plans/org/{org}/{project}/{domain}\x12./api/v1/active_launch_plans/{project}/{domain}\x12\xdc\x01\n\x11ListLaunchPlanIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"j\x82\xd3\xe4\x93\x02\x64Z6\x12\x34/api/v1/launch_plan_ids/org/{org}/{project}/{domain}\x12*/api/v1/launch_plan_ids/{project}/{domain}\x12\xd0\x02\n\x0fListLaunchPlans\x12#.flyteidl.admin.ResourceListRequest\x1a\x1e.flyteidl.admin.LaunchPlanList\"\xf7\x01\x82\xd3\xe4\x93\x02\xf0\x01ZF\x12\x44/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}Z/\x12-/api/v1/launch_plans/{id.project}/{id.domain}Z<\x12:/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}\x12\x37/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}\x12\x8d\x02\n\x10UpdateLaunchPlan\x12\'.flyteidl.admin.LaunchPlanUpdateRequest\x1a(.flyteidl.admin.LaunchPlanUpdateResponse\"\xa5\x01\x82\xd3\xe4\x93\x02\x9e\x01:\x01*ZS\x1aQ/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x1a\x44/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xa4\x01\n\x0f\x43reateExecution\x12&.flyteidl.admin.ExecutionCreateRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"@\x82\xd3\xe4\x93\x02::\x01*Z!:\x01*\x1a\x1c/api/v1/executions/org/{org}\"\x12/api/v1/executions\x12\xbd\x01\n\x11RelaunchExecution\x12(.flyteidl.admin.ExecutionRelaunchRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"U\x82\xd3\xe4\x93\x02O:\x01*Z-:\x01*\"(/api/v1/executions/org/{id.org}/relaunch\"\x1b/api/v1/executions/relaunch\x12\xb9\x01\n\x10RecoverExecution\x12\'.flyteidl.admin.ExecutionRecoverRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"S\x82\xd3\xe4\x93\x02M:\x01*Z,:\x01*\"\'/api/v1/executions/org/{id.org}/recover\"\x1a/api/v1/executions/recover\x12\xdc\x01\n\x0cGetExecution\x12+.flyteidl.admin.WorkflowExecutionGetRequest\x1a\x19.flyteidl.admin.Execution\"\x83\x01\x82\xd3\xe4\x93\x02}ZD\x12\x42/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12\x35/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\xef\x01\n\x0fUpdateExecution\x12&.flyteidl.admin.ExecutionUpdateRequest\x1a\'.flyteidl.admin.ExecutionUpdateResponse\"\x8a\x01\x82\xd3\xe4\x93\x02\x83\x01:\x01*ZG:\x01*\x1a\x42/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x1a\x35/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\x86\x02\n\x10GetExecutionData\x12/.flyteidl.admin.WorkflowExecutionGetDataRequest\x1a\x30.flyteidl.admin.WorkflowExecutionGetDataResponse\"\x8e\x01\x82\xd3\xe4\x93\x02\x87\x01ZI\x12G/api/v1/data/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12:/api/v1/data/executions/{id.project}/{id.domain}/{id.name}\x12\xc5\x01\n\x0eListExecutions\x12#.flyteidl.admin.ResourceListRequest\x1a\x1d.flyteidl.admin.ExecutionList\"o\x82\xd3\xe4\x93\x02iZ:\x12\x38/api/v1/executions/org/{id.org}/{id.project}/{id.domain}\x12+/api/v1/executions/{id.project}/{id.domain}\x12\xf8\x01\n\x12TerminateExecution\x12).flyteidl.admin.ExecutionTerminateRequest\x1a*.flyteidl.admin.ExecutionTerminateResponse\"\x8a\x01\x82\xd3\xe4\x93\x02\x83\x01:\x01*ZG:\x01**B/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}*5/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\xe2\x02\n\x10GetNodeExecution\x12\'.flyteidl.admin.NodeExecutionGetRequest\x1a\x1d.flyteidl.admin.NodeExecution\"\x85\x02\x82\xd3\xe4\x93\x02\xfe\x01Z\x8b\x01\x12\x88\x01/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12n/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12\xf9\x02\n\x12ListNodeExecutions\x12(.flyteidl.admin.NodeExecutionListRequest\x1a!.flyteidl.admin.NodeExecutionList\"\x95\x02\x82\xd3\xe4\x93\x02\x8e\x02Z\x96\x01\x12\x93\x01/api/v1/node_executions/org/{workflow_execution_id.org}/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}\x12s/api/v1/node_executions/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}\x12\x8f\x08\n\x19ListNodeExecutionsForTask\x12/.flyteidl.admin.NodeExecutionForTaskListRequest\x1a!.flyteidl.admin.NodeExecutionList\"\x9d\x07\x82\xd3\xe4\x93\x02\x96\x07Z\xe7\x03\x12\xe4\x03/api/v1/children/task_executions/org/{task_execution_id.node_execution_id.execution_id.org}/{task_execution_id.node_execution_id.execution_id.project}/{task_execution_id.node_execution_id.execution_id.domain}/{task_execution_id.node_execution_id.execution_id.name}/{task_execution_id.node_execution_id.node_id}/{task_execution_id.task_id.project}/{task_execution_id.task_id.domain}/{task_execution_id.task_id.name}/{task_execution_id.task_id.version}/{task_execution_id.retry_attempt}\x12\xa9\x03/api/v1/children/task_executions/{task_execution_id.node_execution_id.execution_id.project}/{task_execution_id.node_execution_id.execution_id.domain}/{task_execution_id.node_execution_id.execution_id.name}/{task_execution_id.node_execution_id.node_id}/{task_execution_id.task_id.project}/{task_execution_id.task_id.domain}/{task_execution_id.task_id.name}/{task_execution_id.task_id.version}/{task_execution_id.retry_attempt}\x12\x83\x03\n\x14GetNodeExecutionData\x12+.flyteidl.admin.NodeExecutionGetDataRequest\x1a,.flyteidl.admin.NodeExecutionGetDataResponse\"\x8f\x02\x82\xd3\xe4\x93\x02\x88\x02Z\x90\x01\x12\x8d\x01/api/v1/data/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12s/api/v1/data/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12\xa8\x01\n\x0fRegisterProject\x12&.flyteidl.admin.ProjectRegisterRequest\x1a\'.flyteidl.admin.ProjectRegisterResponse\"D\x82\xd3\xe4\x93\x02>:\x01*Z\':\x01*\"\"/api/v1/projects/org/{project.org}\"\x10/api/v1/projects\x12\x97\x01\n\rUpdateProject\x12\x17.flyteidl.admin.Project\x1a%.flyteidl.admin.ProjectUpdateResponse\"F\x82\xd3\xe4\x93\x02@:\x01*Z$:\x01*\x1a\x1f/api/v1/projects/org/{org}/{id}\x1a\x15/api/v1/projects/{id}\x12\x84\x01\n\x0cListProjects\x12\".flyteidl.admin.ProjectListRequest\x1a\x18.flyteidl.admin.Projects\"6\x82\xd3\xe4\x93\x02\x30Z\x1c\x12\x1a/api/v1/projects/org/{org}\x12\x10/api/v1/projects\x12\xd5\x01\n\x13\x43reateWorkflowEvent\x12-.flyteidl.admin.WorkflowExecutionEventRequest\x1a..flyteidl.admin.WorkflowExecutionEventResponse\"_\x82\xd3\xe4\x93\x02Y:\x01*Z::\x01*\"5/api/v1/events/org/{event.execution_id.org}/workflows\"\x18/api/v1/events/workflows\x12\xc4\x01\n\x0f\x43reateNodeEvent\x12).flyteidl.admin.NodeExecutionEventRequest\x1a*.flyteidl.admin.NodeExecutionEventResponse\"Z\x82\xd3\xe4\x93\x02T:\x01*Z9:\x01*\"4/api/v1/events/org/{event.id.execution_id.org}/nodes\"\x14/api/v1/events/nodes\x12\xda\x01\n\x0f\x43reateTaskEvent\x12).flyteidl.admin.TaskExecutionEventRequest\x1a*.flyteidl.admin.TaskExecutionEventResponse\"p\x82\xd3\xe4\x93\x02j:\x01*ZO:\x01*\"J/api/v1/events/org/{event.parent_node_execution_id.execution_id.org}/tasks\"\x14/api/v1/events/tasks\x12\xcb\x05\n\x10GetTaskExecution\x12\'.flyteidl.admin.TaskExecutionGetRequest\x1a\x1d.flyteidl.admin.TaskExecution\"\xee\x04\x82\xd3\xe4\x93\x02\xe7\x04Z\xc8\x02\x12\xc5\x02/api/v1/task_executions/org/{id.node_execution_id.execution_id.org}/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\x99\x02/api/v1/task_executions/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\xf1\x03\n\x12ListTaskExecutions\x12(.flyteidl.admin.TaskExecutionListRequest\x1a!.flyteidl.admin.TaskExecutionList\"\x8d\x03\x82\xd3\xe4\x93\x02\x86\x03Z\xd6\x01\x12\xd3\x01/api/v1/task_executions/org/{node_execution_id.execution_id.org}/{node_execution_id.execution_id.project}/{node_execution_id.execution_id.domain}/{node_execution_id.execution_id.name}/{node_execution_id.node_id}\x12\xaa\x01/api/v1/task_executions/{node_execution_id.execution_id.project}/{node_execution_id.execution_id.domain}/{node_execution_id.execution_id.name}/{node_execution_id.node_id}\x12\xec\x05\n\x14GetTaskExecutionData\x12+.flyteidl.admin.TaskExecutionGetDataRequest\x1a,.flyteidl.admin.TaskExecutionGetDataResponse\"\xf8\x04\x82\xd3\xe4\x93\x02\xf1\x04Z\xcd\x02\x12\xca\x02/api/v1/data/task_executions/org/{id.node_execution_id.execution_id.org}/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\x9e\x02/api/v1/data/task_executions/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\xcb\x02\n\x1dUpdateProjectDomainAttributes\x12\x34.flyteidl.admin.ProjectDomainAttributesUpdateRequest\x1a\x35.flyteidl.admin.ProjectDomainAttributesUpdateResponse\"\xbc\x01\x82\xd3\xe4\x93\x02\xb5\x01:\x01*Zd:\x01*\x1a_/api/v1/project_domain_attributes/org/{attributes.org}/{attributes.project}/{attributes.domain}\x1aJ/api/v1/project_domain_attributes/{attributes.project}/{attributes.domain}\x12\x83\x02\n\x1aGetProjectDomainAttributes\x12\x31.flyteidl.admin.ProjectDomainAttributesGetRequest\x1a\x32.flyteidl.admin.ProjectDomainAttributesGetResponse\"~\x82\xd3\xe4\x93\x02xZ@\x12>/api/v1/project_domain_attributes/org/{org}/{project}/{domain}\x12\x34/api/v1/project_domain_attributes/{project}/{domain}\x12\x93\x02\n\x1d\x44\x65leteProjectDomainAttributes\x12\x34.flyteidl.admin.ProjectDomainAttributesDeleteRequest\x1a\x35.flyteidl.admin.ProjectDomainAttributesDeleteResponse\"\x84\x01\x82\xd3\xe4\x93\x02~:\x01*ZC:\x01**>/api/v1/project_domain_attributes/org/{org}/{project}/{domain}*4/api/v1/project_domain_attributes/{project}/{domain}\x12\x8a\x02\n\x17UpdateProjectAttributes\x12..flyteidl.admin.ProjectAttributesUpdateRequest\x1a/.flyteidl.admin.ProjectAttributesUpdateResponse\"\x8d\x01\x82\xd3\xe4\x93\x02\x86\x01:\x01*ZP:\x01*\x1aK/api/v1/project_domain_attributes/org/{attributes.org}/{attributes.project}\x1a//api/v1/project_attributes/{attributes.project}\x12\xd8\x01\n\x14GetProjectAttributes\x12+.flyteidl.admin.ProjectAttributesGetRequest\x1a,.flyteidl.admin.ProjectAttributesGetResponse\"e\x82\xd3\xe4\x93\x02_Z7\x12\x35/api/v1/project_domain_attributes/org/{org}/{project}\x12$/api/v1/project_attributes/{project}\x12\xe7\x01\n\x17\x44\x65leteProjectAttributes\x12..flyteidl.admin.ProjectAttributesDeleteRequest\x1a/.flyteidl.admin.ProjectAttributesDeleteResponse\"k\x82\xd3\xe4\x93\x02\x65:\x01*Z::\x01**5/api/v1/project_domain_attributes/org/{org}/{project}*$/api/v1/project_attributes/{project}\x12\xdc\x02\n\x18UpdateWorkflowAttributes\x12/.flyteidl.admin.WorkflowAttributesUpdateRequest\x1a\x30.flyteidl.admin.WorkflowAttributesUpdateResponse\"\xdc\x01\x82\xd3\xe4\x93\x02\xd5\x01:\x01*Zt:\x01*\x1ao/api/v1/workflow_attributes/org/{attributes.org}/{attributes.project}/{attributes.domain}/{attributes.workflow}\x1aZ/api/v1/workflow_attributes/{attributes.project}/{attributes.domain}/{attributes.workflow}\x12\x80\x02\n\x15GetWorkflowAttributes\x12,.flyteidl.admin.WorkflowAttributesGetRequest\x1a-.flyteidl.admin.WorkflowAttributesGetResponse\"\x89\x01\x82\xd3\xe4\x93\x02\x82\x01ZE\x12\x43/api/v1/workflow_attributes/org/{org}/{project}/{domain}/{workflow}\x12\x39/api/v1/workflow_attributes/{project}/{domain}/{workflow}\x12\x8f\x02\n\x18\x44\x65leteWorkflowAttributes\x12/.flyteidl.admin.WorkflowAttributesDeleteRequest\x1a\x30.flyteidl.admin.WorkflowAttributesDeleteResponse\"\x8f\x01\x82\xd3\xe4\x93\x02\x88\x01:\x01*ZH:\x01**C/api/v1/workflow_attributes/org/{org}/{project}/{domain}/{workflow}*9/api/v1/workflow_attributes/{project}/{domain}/{workflow}\x12\xca\x01\n\x17ListMatchableAttributes\x12..flyteidl.admin.ListMatchableAttributesRequest\x1a/.flyteidl.admin.ListMatchableAttributesResponse\"N\x82\xd3\xe4\x93\x02HZ(\x12&/api/v1/matchable_attributes/org/{org}\x12\x1c/api/v1/matchable_attributes\x12\xe8\x01\n\x11ListNamedEntities\x12&.flyteidl.admin.NamedEntityListRequest\x1a\x1f.flyteidl.admin.NamedEntityList\"\x89\x01\x82\xd3\xe4\x93\x02\x82\x01ZE\x12\x43/api/v1/named_entities/{resource_type}/org/{org}/{project}/{domain}\x12\x39/api/v1/named_entities/{resource_type}/{project}/{domain}\x12\x83\x02\n\x0eGetNamedEntity\x12%.flyteidl.admin.NamedEntityGetRequest\x1a\x1b.flyteidl.admin.NamedEntity\"\xac\x01\x82\xd3\xe4\x93\x02\xa5\x01ZX\x12V/api/v1/named_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12I/api/v1/named_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12\x9d\x02\n\x11UpdateNamedEntity\x12(.flyteidl.admin.NamedEntityUpdateRequest\x1a).flyteidl.admin.NamedEntityUpdateResponse\"\xb2\x01\x82\xd3\xe4\x93\x02\xab\x01:\x01*Z[:\x01*\x1aV/api/v1/named_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}\x1aI/api/v1/named_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12l\n\nGetVersion\x12!.flyteidl.admin.GetVersionRequest\x1a\".flyteidl.admin.GetVersionResponse\"\x17\x82\xd3\xe4\x93\x02\x11\x12\x0f/api/v1/version\x12\xb6\x02\n\x14GetDescriptionEntity\x12 .flyteidl.admin.ObjectGetRequest\x1a!.flyteidl.admin.DescriptionEntity\"\xd8\x01\x82\xd3\xe4\x93\x02\xd1\x01Zn\x12l/api/v1/description_entities/org/{id.org}/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version}\x12_/api/v1/description_entities/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xc8\x03\n\x17ListDescriptionEntities\x12,.flyteidl.admin.DescriptionEntityListRequest\x1a%.flyteidl.admin.DescriptionEntityList\"\xd7\x02\x82\xd3\xe4\x93\x02\xd0\x02Z^\x12\\/api/v1/description_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}ZG\x12\x45/api/v1/description_entities/{resource_type}/{id.project}/{id.domain}ZT\x12R/api/v1/description_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}\x12O/api/v1/description_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12\x95\x02\n\x13GetExecutionMetrics\x12\x32.flyteidl.admin.WorkflowExecutionGetMetricsRequest\x1a\x33.flyteidl.admin.WorkflowExecutionGetMetricsResponse\"\x94\x01\x82\xd3\xe4\x93\x02\x8d\x01ZL\x12J/api/v1/metrics/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12=/api/v1/metrics/executions/{id.project}/{id.domain}/{id.name}B\xc2\x01\n\x14\x63om.flyteidl.serviceB\nAdminProtoP\x01Z=github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service\xa2\x02\x03\x46SX\xaa\x02\x10\x46lyteidl.Service\xca\x02\x10\x46lyteidl\\Service\xe2\x02\x1c\x46lyteidl\\Service\\GPBMetadata\xea\x02\x11\x46lyteidl::Serviceb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1c\x66lyteidl/service/admin.proto\x12\x10\x66lyteidl.service\x1a\x1cgoogle/api/annotations.proto\x1a\x1c\x66lyteidl/admin/project.proto\x1a.flyteidl/admin/project_domain_attributes.proto\x1a\'flyteidl/admin/project_attributes.proto\x1a\x19\x66lyteidl/admin/task.proto\x1a\x1d\x66lyteidl/admin/workflow.proto\x1a(flyteidl/admin/workflow_attributes.proto\x1a flyteidl/admin/launch_plan.proto\x1a\x1a\x66lyteidl/admin/event.proto\x1a\x1e\x66lyteidl/admin/execution.proto\x1a\'flyteidl/admin/matchable_resource.proto\x1a#flyteidl/admin/node_execution.proto\x1a#flyteidl/admin/task_execution.proto\x1a\x1c\x66lyteidl/admin/version.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\'flyteidl/admin/description_entity.proto2\xb8y\n\x0c\x41\x64minService\x12\x8e\x01\n\nCreateTask\x12!.flyteidl.admin.TaskCreateRequest\x1a\".flyteidl.admin.TaskCreateResponse\"9\x82\xd3\xe4\x93\x02\x33:\x01*Z\x1f:\x01*\"\x1a/api/v1/tasks/org/{id.org}\"\r/api/v1/tasks\x12\xd8\x01\n\x07GetTask\x12 .flyteidl.admin.ObjectGetRequest\x1a\x14.flyteidl.admin.Task\"\x94\x01\x82\xd3\xe4\x93\x02\x8d\x01ZL\x12J/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12=/api/v1/tasks/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xc5\x01\n\x0bListTaskIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"Y\x82\xd3\xe4\x93\x02SZ,\x12*/api/v1/tasks/org/{org}/{project}/{domain}\x12#/api/v1/task_ids/{project}/{domain}\x12\xa8\x02\n\tListTasks\x12#.flyteidl.admin.ResourceListRequest\x1a\x18.flyteidl.admin.TaskList\"\xdb\x01\x82\xd3\xe4\x93\x02\xd4\x01Z?\x12=/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}/{id.name}Z(\x12&/api/v1/tasks/{id.project}/{id.domain}Z5\x12\x33/api/v1/tasks/org/{id.org}/{id.project}/{id.domain}\x12\x30/api/v1/tasks/{id.project}/{id.domain}/{id.name}\x12\xa2\x01\n\x0e\x43reateWorkflow\x12%.flyteidl.admin.WorkflowCreateRequest\x1a&.flyteidl.admin.WorkflowCreateResponse\"A\x82\xd3\xe4\x93\x02;:\x01*Z#:\x01*\"\x1e/api/v1/workflows/org/{id.org}\"\x11/api/v1/workflows\x12\xe8\x01\n\x0bGetWorkflow\x12 .flyteidl.admin.ObjectGetRequest\x1a\x18.flyteidl.admin.Workflow\"\x9c\x01\x82\xd3\xe4\x93\x02\x95\x01ZP\x12N/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\x41/api/v1/workflows/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xd1\x01\n\x0fListWorkflowIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"a\x82\xd3\xe4\x93\x02[Z0\x12./api/v1/workflows/org/{org}/{project}/{domain}\x12\'/api/v1/workflow_ids/{project}/{domain}\x12\xc0\x02\n\rListWorkflows\x12#.flyteidl.admin.ResourceListRequest\x1a\x1c.flyteidl.admin.WorkflowList\"\xeb\x01\x82\xd3\xe4\x93\x02\xe4\x01ZC\x12\x41/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}/{id.name}Z,\x12*/api/v1/workflows/{id.project}/{id.domain}Z9\x12\x37/api/v1/workflows/org/{id.org}/{id.project}/{id.domain}\x12\x34/api/v1/workflows/{id.project}/{id.domain}/{id.name}\x12\xae\x01\n\x10\x43reateLaunchPlan\x12\'.flyteidl.admin.LaunchPlanCreateRequest\x1a(.flyteidl.admin.LaunchPlanCreateResponse\"G\x82\xd3\xe4\x93\x02\x41:\x01*Z&:\x01*\"!/api/v1/launch_plans/org/{id.org}\"\x14/api/v1/launch_plans\x12\xf2\x01\n\rGetLaunchPlan\x12 .flyteidl.admin.ObjectGetRequest\x1a\x1a.flyteidl.admin.LaunchPlan\"\xa2\x01\x82\xd3\xe4\x93\x02\x9b\x01ZS\x12Q/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\x44/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xf3\x01\n\x13GetActiveLaunchPlan\x12\'.flyteidl.admin.ActiveLaunchPlanRequest\x1a\x1a.flyteidl.admin.LaunchPlan\"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01ZM\x12K/api/v1/active_launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12>/api/v1/active_launch_plans/{id.project}/{id.domain}/{id.name}\x12\xd8\x01\n\x15ListActiveLaunchPlans\x12+.flyteidl.admin.ActiveLaunchPlanListRequest\x1a\x1e.flyteidl.admin.LaunchPlanList\"r\x82\xd3\xe4\x93\x02lZ:\x12\x38/api/v1/active_launch_plans/org/{org}/{project}/{domain}\x12./api/v1/active_launch_plans/{project}/{domain}\x12\xdc\x01\n\x11ListLaunchPlanIds\x12\x30.flyteidl.admin.NamedEntityIdentifierListRequest\x1a).flyteidl.admin.NamedEntityIdentifierList\"j\x82\xd3\xe4\x93\x02\x64Z6\x12\x34/api/v1/launch_plan_ids/org/{org}/{project}/{domain}\x12*/api/v1/launch_plan_ids/{project}/{domain}\x12\xd0\x02\n\x0fListLaunchPlans\x12#.flyteidl.admin.ResourceListRequest\x1a\x1e.flyteidl.admin.LaunchPlanList\"\xf7\x01\x82\xd3\xe4\x93\x02\xf0\x01ZF\x12\x44/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}Z/\x12-/api/v1/launch_plans/{id.project}/{id.domain}Z<\x12:/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}\x12\x37/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}\x12\x8d\x02\n\x10UpdateLaunchPlan\x12\'.flyteidl.admin.LaunchPlanUpdateRequest\x1a(.flyteidl.admin.LaunchPlanUpdateResponse\"\xa5\x01\x82\xd3\xe4\x93\x02\x9e\x01:\x01*ZS\x1aQ/api/v1/launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name}/{id.version}\x1a\x44/api/v1/launch_plans/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xa4\x01\n\x0f\x43reateExecution\x12&.flyteidl.admin.ExecutionCreateRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"@\x82\xd3\xe4\x93\x02::\x01*Z!:\x01*\x1a\x1c/api/v1/executions/org/{org}\"\x12/api/v1/executions\x12\xbd\x01\n\x11RelaunchExecution\x12(.flyteidl.admin.ExecutionRelaunchRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"U\x82\xd3\xe4\x93\x02O:\x01*Z-:\x01*\"(/api/v1/executions/org/{id.org}/relaunch\"\x1b/api/v1/executions/relaunch\x12\xb9\x01\n\x10RecoverExecution\x12\'.flyteidl.admin.ExecutionRecoverRequest\x1a\'.flyteidl.admin.ExecutionCreateResponse\"S\x82\xd3\xe4\x93\x02M:\x01*Z,:\x01*\"\'/api/v1/executions/org/{id.org}/recover\"\x1a/api/v1/executions/recover\x12\xdc\x01\n\x0cGetExecution\x12+.flyteidl.admin.WorkflowExecutionGetRequest\x1a\x19.flyteidl.admin.Execution\"\x83\x01\x82\xd3\xe4\x93\x02}ZD\x12\x42/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12\x35/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\xef\x01\n\x0fUpdateExecution\x12&.flyteidl.admin.ExecutionUpdateRequest\x1a\'.flyteidl.admin.ExecutionUpdateResponse\"\x8a\x01\x82\xd3\xe4\x93\x02\x83\x01:\x01*ZG:\x01*\x1a\x42/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x1a\x35/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\x86\x02\n\x10GetExecutionData\x12/.flyteidl.admin.WorkflowExecutionGetDataRequest\x1a\x30.flyteidl.admin.WorkflowExecutionGetDataResponse\"\x8e\x01\x82\xd3\xe4\x93\x02\x87\x01ZI\x12G/api/v1/data/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12:/api/v1/data/executions/{id.project}/{id.domain}/{id.name}\x12\xc5\x01\n\x0eListExecutions\x12#.flyteidl.admin.ResourceListRequest\x1a\x1d.flyteidl.admin.ExecutionList\"o\x82\xd3\xe4\x93\x02iZ:\x12\x38/api/v1/executions/org/{id.org}/{id.project}/{id.domain}\x12+/api/v1/executions/{id.project}/{id.domain}\x12\xf8\x01\n\x12TerminateExecution\x12).flyteidl.admin.ExecutionTerminateRequest\x1a*.flyteidl.admin.ExecutionTerminateResponse\"\x8a\x01\x82\xd3\xe4\x93\x02\x83\x01:\x01*ZG:\x01**B/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}*5/api/v1/executions/{id.project}/{id.domain}/{id.name}\x12\xe2\x02\n\x10GetNodeExecution\x12\'.flyteidl.admin.NodeExecutionGetRequest\x1a\x1d.flyteidl.admin.NodeExecution\"\x85\x02\x82\xd3\xe4\x93\x02\xfe\x01Z\x8b\x01\x12\x88\x01/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12n/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12\x9e\x03\n\x16GetDynamicNodeWorkflow\x12-.flyteidl.admin.GetDynamicNodeWorkflowRequest\x1a+.flyteidl.admin.DynamicNodeWorkflowResponse\"\xa7\x02\x82\xd3\xe4\x93\x02\xa0\x02Z\x9c\x01\x12\x99\x01/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow\x12\x7f/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow\x12\xf9\x02\n\x12ListNodeExecutions\x12(.flyteidl.admin.NodeExecutionListRequest\x1a!.flyteidl.admin.NodeExecutionList\"\x95\x02\x82\xd3\xe4\x93\x02\x8e\x02Z\x96\x01\x12\x93\x01/api/v1/node_executions/org/{workflow_execution_id.org}/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}\x12s/api/v1/node_executions/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}\x12\x8f\x08\n\x19ListNodeExecutionsForTask\x12/.flyteidl.admin.NodeExecutionForTaskListRequest\x1a!.flyteidl.admin.NodeExecutionList\"\x9d\x07\x82\xd3\xe4\x93\x02\x96\x07Z\xe7\x03\x12\xe4\x03/api/v1/children/task_executions/org/{task_execution_id.node_execution_id.execution_id.org}/{task_execution_id.node_execution_id.execution_id.project}/{task_execution_id.node_execution_id.execution_id.domain}/{task_execution_id.node_execution_id.execution_id.name}/{task_execution_id.node_execution_id.node_id}/{task_execution_id.task_id.project}/{task_execution_id.task_id.domain}/{task_execution_id.task_id.name}/{task_execution_id.task_id.version}/{task_execution_id.retry_attempt}\x12\xa9\x03/api/v1/children/task_executions/{task_execution_id.node_execution_id.execution_id.project}/{task_execution_id.node_execution_id.execution_id.domain}/{task_execution_id.node_execution_id.execution_id.name}/{task_execution_id.node_execution_id.node_id}/{task_execution_id.task_id.project}/{task_execution_id.task_id.domain}/{task_execution_id.task_id.name}/{task_execution_id.task_id.version}/{task_execution_id.retry_attempt}\x12\x83\x03\n\x14GetNodeExecutionData\x12+.flyteidl.admin.NodeExecutionGetDataRequest\x1a,.flyteidl.admin.NodeExecutionGetDataResponse\"\x8f\x02\x82\xd3\xe4\x93\x02\x88\x02Z\x90\x01\x12\x8d\x01/api/v1/data/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12s/api/v1/data/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\x12\xa8\x01\n\x0fRegisterProject\x12&.flyteidl.admin.ProjectRegisterRequest\x1a\'.flyteidl.admin.ProjectRegisterResponse\"D\x82\xd3\xe4\x93\x02>:\x01*Z\':\x01*\"\"/api/v1/projects/org/{project.org}\"\x10/api/v1/projects\x12\x97\x01\n\rUpdateProject\x12\x17.flyteidl.admin.Project\x1a%.flyteidl.admin.ProjectUpdateResponse\"F\x82\xd3\xe4\x93\x02@:\x01*Z$:\x01*\x1a\x1f/api/v1/projects/org/{org}/{id}\x1a\x15/api/v1/projects/{id}\x12\x84\x01\n\x0cListProjects\x12\".flyteidl.admin.ProjectListRequest\x1a\x18.flyteidl.admin.Projects\"6\x82\xd3\xe4\x93\x02\x30Z\x1c\x12\x1a/api/v1/projects/org/{org}\x12\x10/api/v1/projects\x12\xd5\x01\n\x13\x43reateWorkflowEvent\x12-.flyteidl.admin.WorkflowExecutionEventRequest\x1a..flyteidl.admin.WorkflowExecutionEventResponse\"_\x82\xd3\xe4\x93\x02Y:\x01*Z::\x01*\"5/api/v1/events/org/{event.execution_id.org}/workflows\"\x18/api/v1/events/workflows\x12\xc4\x01\n\x0f\x43reateNodeEvent\x12).flyteidl.admin.NodeExecutionEventRequest\x1a*.flyteidl.admin.NodeExecutionEventResponse\"Z\x82\xd3\xe4\x93\x02T:\x01*Z9:\x01*\"4/api/v1/events/org/{event.id.execution_id.org}/nodes\"\x14/api/v1/events/nodes\x12\xda\x01\n\x0f\x43reateTaskEvent\x12).flyteidl.admin.TaskExecutionEventRequest\x1a*.flyteidl.admin.TaskExecutionEventResponse\"p\x82\xd3\xe4\x93\x02j:\x01*ZO:\x01*\"J/api/v1/events/org/{event.parent_node_execution_id.execution_id.org}/tasks\"\x14/api/v1/events/tasks\x12\xcb\x05\n\x10GetTaskExecution\x12\'.flyteidl.admin.TaskExecutionGetRequest\x1a\x1d.flyteidl.admin.TaskExecution\"\xee\x04\x82\xd3\xe4\x93\x02\xe7\x04Z\xc8\x02\x12\xc5\x02/api/v1/task_executions/org/{id.node_execution_id.execution_id.org}/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\x99\x02/api/v1/task_executions/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\xf1\x03\n\x12ListTaskExecutions\x12(.flyteidl.admin.TaskExecutionListRequest\x1a!.flyteidl.admin.TaskExecutionList\"\x8d\x03\x82\xd3\xe4\x93\x02\x86\x03Z\xd6\x01\x12\xd3\x01/api/v1/task_executions/org/{node_execution_id.execution_id.org}/{node_execution_id.execution_id.project}/{node_execution_id.execution_id.domain}/{node_execution_id.execution_id.name}/{node_execution_id.node_id}\x12\xaa\x01/api/v1/task_executions/{node_execution_id.execution_id.project}/{node_execution_id.execution_id.domain}/{node_execution_id.execution_id.name}/{node_execution_id.node_id}\x12\xec\x05\n\x14GetTaskExecutionData\x12+.flyteidl.admin.TaskExecutionGetDataRequest\x1a,.flyteidl.admin.TaskExecutionGetDataResponse\"\xf8\x04\x82\xd3\xe4\x93\x02\xf1\x04Z\xcd\x02\x12\xca\x02/api/v1/data/task_executions/org/{id.node_execution_id.execution_id.org}/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\x9e\x02/api/v1/data/task_executions/{id.node_execution_id.execution_id.project}/{id.node_execution_id.execution_id.domain}/{id.node_execution_id.execution_id.name}/{id.node_execution_id.node_id}/{id.task_id.project}/{id.task_id.domain}/{id.task_id.name}/{id.task_id.version}/{id.retry_attempt}\x12\xcb\x02\n\x1dUpdateProjectDomainAttributes\x12\x34.flyteidl.admin.ProjectDomainAttributesUpdateRequest\x1a\x35.flyteidl.admin.ProjectDomainAttributesUpdateResponse\"\xbc\x01\x82\xd3\xe4\x93\x02\xb5\x01:\x01*Zd:\x01*\x1a_/api/v1/project_domain_attributes/org/{attributes.org}/{attributes.project}/{attributes.domain}\x1aJ/api/v1/project_domain_attributes/{attributes.project}/{attributes.domain}\x12\x83\x02\n\x1aGetProjectDomainAttributes\x12\x31.flyteidl.admin.ProjectDomainAttributesGetRequest\x1a\x32.flyteidl.admin.ProjectDomainAttributesGetResponse\"~\x82\xd3\xe4\x93\x02xZ@\x12>/api/v1/project_domain_attributes/org/{org}/{project}/{domain}\x12\x34/api/v1/project_domain_attributes/{project}/{domain}\x12\x93\x02\n\x1d\x44\x65leteProjectDomainAttributes\x12\x34.flyteidl.admin.ProjectDomainAttributesDeleteRequest\x1a\x35.flyteidl.admin.ProjectDomainAttributesDeleteResponse\"\x84\x01\x82\xd3\xe4\x93\x02~:\x01*ZC:\x01**>/api/v1/project_domain_attributes/org/{org}/{project}/{domain}*4/api/v1/project_domain_attributes/{project}/{domain}\x12\x8a\x02\n\x17UpdateProjectAttributes\x12..flyteidl.admin.ProjectAttributesUpdateRequest\x1a/.flyteidl.admin.ProjectAttributesUpdateResponse\"\x8d\x01\x82\xd3\xe4\x93\x02\x86\x01:\x01*ZP:\x01*\x1aK/api/v1/project_domain_attributes/org/{attributes.org}/{attributes.project}\x1a//api/v1/project_attributes/{attributes.project}\x12\xd8\x01\n\x14GetProjectAttributes\x12+.flyteidl.admin.ProjectAttributesGetRequest\x1a,.flyteidl.admin.ProjectAttributesGetResponse\"e\x82\xd3\xe4\x93\x02_Z7\x12\x35/api/v1/project_domain_attributes/org/{org}/{project}\x12$/api/v1/project_attributes/{project}\x12\xe7\x01\n\x17\x44\x65leteProjectAttributes\x12..flyteidl.admin.ProjectAttributesDeleteRequest\x1a/.flyteidl.admin.ProjectAttributesDeleteResponse\"k\x82\xd3\xe4\x93\x02\x65:\x01*Z::\x01**5/api/v1/project_domain_attributes/org/{org}/{project}*$/api/v1/project_attributes/{project}\x12\xdc\x02\n\x18UpdateWorkflowAttributes\x12/.flyteidl.admin.WorkflowAttributesUpdateRequest\x1a\x30.flyteidl.admin.WorkflowAttributesUpdateResponse\"\xdc\x01\x82\xd3\xe4\x93\x02\xd5\x01:\x01*Zt:\x01*\x1ao/api/v1/workflow_attributes/org/{attributes.org}/{attributes.project}/{attributes.domain}/{attributes.workflow}\x1aZ/api/v1/workflow_attributes/{attributes.project}/{attributes.domain}/{attributes.workflow}\x12\x80\x02\n\x15GetWorkflowAttributes\x12,.flyteidl.admin.WorkflowAttributesGetRequest\x1a-.flyteidl.admin.WorkflowAttributesGetResponse\"\x89\x01\x82\xd3\xe4\x93\x02\x82\x01ZE\x12\x43/api/v1/workflow_attributes/org/{org}/{project}/{domain}/{workflow}\x12\x39/api/v1/workflow_attributes/{project}/{domain}/{workflow}\x12\x8f\x02\n\x18\x44\x65leteWorkflowAttributes\x12/.flyteidl.admin.WorkflowAttributesDeleteRequest\x1a\x30.flyteidl.admin.WorkflowAttributesDeleteResponse\"\x8f\x01\x82\xd3\xe4\x93\x02\x88\x01:\x01*ZH:\x01**C/api/v1/workflow_attributes/org/{org}/{project}/{domain}/{workflow}*9/api/v1/workflow_attributes/{project}/{domain}/{workflow}\x12\xca\x01\n\x17ListMatchableAttributes\x12..flyteidl.admin.ListMatchableAttributesRequest\x1a/.flyteidl.admin.ListMatchableAttributesResponse\"N\x82\xd3\xe4\x93\x02HZ(\x12&/api/v1/matchable_attributes/org/{org}\x12\x1c/api/v1/matchable_attributes\x12\xe8\x01\n\x11ListNamedEntities\x12&.flyteidl.admin.NamedEntityListRequest\x1a\x1f.flyteidl.admin.NamedEntityList\"\x89\x01\x82\xd3\xe4\x93\x02\x82\x01ZE\x12\x43/api/v1/named_entities/{resource_type}/org/{org}/{project}/{domain}\x12\x39/api/v1/named_entities/{resource_type}/{project}/{domain}\x12\x83\x02\n\x0eGetNamedEntity\x12%.flyteidl.admin.NamedEntityGetRequest\x1a\x1b.flyteidl.admin.NamedEntity\"\xac\x01\x82\xd3\xe4\x93\x02\xa5\x01ZX\x12V/api/v1/named_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12I/api/v1/named_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12\x9d\x02\n\x11UpdateNamedEntity\x12(.flyteidl.admin.NamedEntityUpdateRequest\x1a).flyteidl.admin.NamedEntityUpdateResponse\"\xb2\x01\x82\xd3\xe4\x93\x02\xab\x01:\x01*Z[:\x01*\x1aV/api/v1/named_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}\x1aI/api/v1/named_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12l\n\nGetVersion\x12!.flyteidl.admin.GetVersionRequest\x1a\".flyteidl.admin.GetVersionResponse\"\x17\x82\xd3\xe4\x93\x02\x11\x12\x0f/api/v1/version\x12\xb6\x02\n\x14GetDescriptionEntity\x12 .flyteidl.admin.ObjectGetRequest\x1a!.flyteidl.admin.DescriptionEntity\"\xd8\x01\x82\xd3\xe4\x93\x02\xd1\x01Zn\x12l/api/v1/description_entities/org/{id.org}/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version}\x12_/api/v1/description_entities/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version}\x12\xc8\x03\n\x17ListDescriptionEntities\x12,.flyteidl.admin.DescriptionEntityListRequest\x1a%.flyteidl.admin.DescriptionEntityList\"\xd7\x02\x82\xd3\xe4\x93\x02\xd0\x02Z^\x12\\/api/v1/description_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}/{id.name}ZG\x12\x45/api/v1/description_entities/{resource_type}/{id.project}/{id.domain}ZT\x12R/api/v1/description_entities/{resource_type}/org/{id.org}/{id.project}/{id.domain}\x12O/api/v1/description_entities/{resource_type}/{id.project}/{id.domain}/{id.name}\x12\x95\x02\n\x13GetExecutionMetrics\x12\x32.flyteidl.admin.WorkflowExecutionGetMetricsRequest\x1a\x33.flyteidl.admin.WorkflowExecutionGetMetricsResponse\"\x94\x01\x82\xd3\xe4\x93\x02\x8d\x01ZL\x12J/api/v1/metrics/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\x12=/api/v1/metrics/executions/{id.project}/{id.domain}/{id.name}B\xc2\x01\n\x14\x63om.flyteidl.serviceB\nAdminProtoP\x01Z=github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service\xa2\x02\x03\x46SX\xaa\x02\x10\x46lyteidl.Service\xca\x02\x10\x46lyteidl\\Service\xe2\x02\x1c\x46lyteidl\\Service\\GPBMetadata\xea\x02\x11\x46lyteidl::Serviceb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -86,6 +86,8 @@ _ADMINSERVICE.methods_by_name['TerminateExecution']._serialized_options = b'\202\323\344\223\002\203\001:\001*ZG:\001**B/api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}*5/api/v1/executions/{id.project}/{id.domain}/{id.name}' _ADMINSERVICE.methods_by_name['GetNodeExecution']._options = None _ADMINSERVICE.methods_by_name['GetNodeExecution']._serialized_options = b'\202\323\344\223\002\376\001Z\213\001\022\210\001/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}\022n/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}' + _ADMINSERVICE.methods_by_name['GetDynamicNodeWorkflow']._options = None + _ADMINSERVICE.methods_by_name['GetDynamicNodeWorkflow']._serialized_options = b'\202\323\344\223\002\240\002Z\234\001\022\231\001/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow\022\177/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow' _ADMINSERVICE.methods_by_name['ListNodeExecutions']._options = None _ADMINSERVICE.methods_by_name['ListNodeExecutions']._serialized_options = b'\202\323\344\223\002\216\002Z\226\001\022\223\001/api/v1/node_executions/org/{workflow_execution_id.org}/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}\022s/api/v1/node_executions/{workflow_execution_id.project}/{workflow_execution_id.domain}/{workflow_execution_id.name}' _ADMINSERVICE.methods_by_name['ListNodeExecutionsForTask']._options = None @@ -145,5 +147,5 @@ _ADMINSERVICE.methods_by_name['GetExecutionMetrics']._options = None _ADMINSERVICE.methods_by_name['GetExecutionMetrics']._serialized_options = b'\202\323\344\223\002\215\001ZL\022J/api/v1/metrics/executions/org/{id.org}/{id.project}/{id.domain}/{id.name}\022=/api/v1/metrics/executions/{id.project}/{id.domain}/{id.name}' _globals['_ADMINSERVICE']._serialized_start=609 - _globals['_ADMINSERVICE']._serialized_end=15736 + _globals['_ADMINSERVICE']._serialized_end=16153 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/service/admin_pb2_grpc.py b/flyteidl/gen/pb_python/flyteidl/service/admin_pb2_grpc.py index 7fe54c9f8c..5bb1baac60 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/admin_pb2_grpc.py +++ b/flyteidl/gen/pb_python/flyteidl/service/admin_pb2_grpc.py @@ -150,6 +150,11 @@ def __init__(self, channel): request_serializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecutionGetRequest.SerializeToString, response_deserializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecution.FromString, ) + self.GetDynamicNodeWorkflow = channel.unary_unary( + '/flyteidl.service.AdminService/GetDynamicNodeWorkflow', + request_serializer=flyteidl_dot_admin_dot_node__execution__pb2.GetDynamicNodeWorkflowRequest.SerializeToString, + response_deserializer=flyteidl_dot_admin_dot_node__execution__pb2.DynamicNodeWorkflowResponse.FromString, + ) self.ListNodeExecutions = channel.unary_unary( '/flyteidl.service.AdminService/ListNodeExecutions', request_serializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecutionListRequest.SerializeToString, @@ -474,6 +479,13 @@ def GetNodeExecution(self, request, context): context.set_details('Method not implemented!') raise NotImplementedError('Method not implemented!') + def GetDynamicNodeWorkflow(self, request, context): + """Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + def ListNodeExecutions(self, request, context): """Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. """ @@ -801,6 +813,11 @@ def add_AdminServiceServicer_to_server(servicer, server): request_deserializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecutionGetRequest.FromString, response_serializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecution.SerializeToString, ), + 'GetDynamicNodeWorkflow': grpc.unary_unary_rpc_method_handler( + servicer.GetDynamicNodeWorkflow, + request_deserializer=flyteidl_dot_admin_dot_node__execution__pb2.GetDynamicNodeWorkflowRequest.FromString, + response_serializer=flyteidl_dot_admin_dot_node__execution__pb2.DynamicNodeWorkflowResponse.SerializeToString, + ), 'ListNodeExecutions': grpc.unary_unary_rpc_method_handler( servicer.ListNodeExecutions, request_deserializer=flyteidl_dot_admin_dot_node__execution__pb2.NodeExecutionListRequest.FromString, @@ -1366,6 +1383,23 @@ def GetNodeExecution(request, options, channel_credentials, insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + @staticmethod + def GetDynamicNodeWorkflow(request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/flyteidl.service.AdminService/GetDynamicNodeWorkflow', + flyteidl_dot_admin_dot_node__execution__pb2.GetDynamicNodeWorkflowRequest.SerializeToString, + flyteidl_dot_admin_dot_node__execution__pb2.DynamicNodeWorkflowResponse.FromString, + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + @staticmethod def ListNodeExecutions(request, target, diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/README.md b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/README.md index 6490f1b9a8..ebc981f1b5 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/README.md +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/README.md @@ -94,6 +94,8 @@ Class | Method | HTTP request | Description *AdminServiceApi* | [**get_active_launch_plan2**](docs/AdminServiceApi.md#get_active_launch_plan2) | **GET** /api/v1/active_launch_plans/org/{id.org}/{id.project}/{id.domain}/{id.name} | Fetch the active version of a :ref:`ref_flyteidl.admin.LaunchPlan`. *AdminServiceApi* | [**get_description_entity**](docs/AdminServiceApi.md#get_description_entity) | **GET** /api/v1/description_entities/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version} | Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. *AdminServiceApi* | [**get_description_entity2**](docs/AdminServiceApi.md#get_description_entity2) | **GET** /api/v1/description_entities/org/{id.org}/{id.resource_type}/{id.project}/{id.domain}/{id.name}/{id.version} | Fetch a :ref:`ref_flyteidl.admin.DescriptionEntity` object. +*AdminServiceApi* | [**get_dynamic_node_workflow**](docs/AdminServiceApi.md#get_dynamic_node_workflow) | **GET** /api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow | Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. +*AdminServiceApi* | [**get_dynamic_node_workflow2**](docs/AdminServiceApi.md#get_dynamic_node_workflow2) | **GET** /api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow | Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. *AdminServiceApi* | [**get_execution**](docs/AdminServiceApi.md#get_execution) | **GET** /api/v1/executions/{id.project}/{id.domain}/{id.name} | Fetches a :ref:`ref_flyteidl.admin.Execution`. *AdminServiceApi* | [**get_execution2**](docs/AdminServiceApi.md#get_execution2) | **GET** /api/v1/executions/org/{id.org}/{id.project}/{id.domain}/{id.name} | Fetches a :ref:`ref_flyteidl.admin.Execution`. *AdminServiceApi* | [**get_execution_data**](docs/AdminServiceApi.md#get_execution_data) | **GET** /api/v1/data/executions/{id.project}/{id.domain}/{id.name} | Fetches input and output data for a :ref:`ref_flyteidl.admin.Execution`. @@ -199,6 +201,7 @@ Class | Method | HTTP request | Description - [AdminDescriptionEntityList](docs/AdminDescriptionEntityList.md) - [AdminDescriptionFormat](docs/AdminDescriptionFormat.md) - [AdminDomain](docs/AdminDomain.md) + - [AdminDynamicNodeWorkflowResponse](docs/AdminDynamicNodeWorkflowResponse.md) - [AdminEmailNotification](docs/AdminEmailNotification.md) - [AdminEnvs](docs/AdminEnvs.md) - [AdminExecution](docs/AdminExecution.md) diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/__init__.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/__init__.py index 6cbddc0b5b..83fb266b30 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/__init__.py +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/__init__.py @@ -34,6 +34,7 @@ from flyteadmin.models.admin_description_entity_list import AdminDescriptionEntityList from flyteadmin.models.admin_description_format import AdminDescriptionFormat from flyteadmin.models.admin_domain import AdminDomain +from flyteadmin.models.admin_dynamic_node_workflow_response import AdminDynamicNodeWorkflowResponse from flyteadmin.models.admin_email_notification import AdminEmailNotification from flyteadmin.models.admin_envs import AdminEnvs from flyteadmin.models.admin_execution import AdminExecution diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/api/admin_service_api.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/api/admin_service_api.py index ee0e11ddb3..f612c2274d 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/api/admin_service_api.py +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/api/admin_service_api.py @@ -2657,6 +2657,260 @@ def get_description_entity2_with_http_info(self, id_org, id_resource_type, id_pr _request_timeout=params.get('_request_timeout'), collection_formats=collection_formats) + def get_dynamic_node_workflow(self, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs): # noqa: E501 + """Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + + This method makes a synchronous HTTP request by default. To make an + asynchronous HTTP request, please pass async_req=True + >>> thread = api.get_dynamic_node_workflow(id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, async_req=True) + >>> result = thread.get() + + :param async_req bool + :param str id_execution_id_project: Name of the project the resource belongs to. (required) + :param str id_execution_id_domain: Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. (required) + :param str id_execution_id_name: User or system provided value for the resource. (required) + :param str id_node_id: (required) + :param str id_execution_id_org: Optional, org key applied to the resource. + :return: AdminDynamicNodeWorkflowResponse + If the method is called asynchronously, + returns the request thread. + """ + kwargs['_return_http_data_only'] = True + if kwargs.get('async_req'): + return self.get_dynamic_node_workflow_with_http_info(id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs) # noqa: E501 + else: + (data) = self.get_dynamic_node_workflow_with_http_info(id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs) # noqa: E501 + return data + + def get_dynamic_node_workflow_with_http_info(self, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs): # noqa: E501 + """Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + + This method makes a synchronous HTTP request by default. To make an + asynchronous HTTP request, please pass async_req=True + >>> thread = api.get_dynamic_node_workflow_with_http_info(id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, async_req=True) + >>> result = thread.get() + + :param async_req bool + :param str id_execution_id_project: Name of the project the resource belongs to. (required) + :param str id_execution_id_domain: Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. (required) + :param str id_execution_id_name: User or system provided value for the resource. (required) + :param str id_node_id: (required) + :param str id_execution_id_org: Optional, org key applied to the resource. + :return: AdminDynamicNodeWorkflowResponse + If the method is called asynchronously, + returns the request thread. + """ + + all_params = ['id_execution_id_project', 'id_execution_id_domain', 'id_execution_id_name', 'id_node_id', 'id_execution_id_org'] # noqa: E501 + all_params.append('async_req') + all_params.append('_return_http_data_only') + all_params.append('_preload_content') + all_params.append('_request_timeout') + + params = locals() + for key, val in six.iteritems(params['kwargs']): + if key not in all_params: + raise TypeError( + "Got an unexpected keyword argument '%s'" + " to method get_dynamic_node_workflow" % key + ) + params[key] = val + del params['kwargs'] + # verify the required parameter 'id_execution_id_project' is set + if ('id_execution_id_project' not in params or + params['id_execution_id_project'] is None): + raise ValueError("Missing the required parameter `id_execution_id_project` when calling `get_dynamic_node_workflow`") # noqa: E501 + # verify the required parameter 'id_execution_id_domain' is set + if ('id_execution_id_domain' not in params or + params['id_execution_id_domain'] is None): + raise ValueError("Missing the required parameter `id_execution_id_domain` when calling `get_dynamic_node_workflow`") # noqa: E501 + # verify the required parameter 'id_execution_id_name' is set + if ('id_execution_id_name' not in params or + params['id_execution_id_name'] is None): + raise ValueError("Missing the required parameter `id_execution_id_name` when calling `get_dynamic_node_workflow`") # noqa: E501 + # verify the required parameter 'id_node_id' is set + if ('id_node_id' not in params or + params['id_node_id'] is None): + raise ValueError("Missing the required parameter `id_node_id` when calling `get_dynamic_node_workflow`") # noqa: E501 + + collection_formats = {} + + path_params = {} + if 'id_execution_id_project' in params: + path_params['id.execution_id.project'] = params['id_execution_id_project'] # noqa: E501 + if 'id_execution_id_domain' in params: + path_params['id.execution_id.domain'] = params['id_execution_id_domain'] # noqa: E501 + if 'id_execution_id_name' in params: + path_params['id.execution_id.name'] = params['id_execution_id_name'] # noqa: E501 + if 'id_node_id' in params: + path_params['id.node_id'] = params['id_node_id'] # noqa: E501 + + query_params = [] + if 'id_execution_id_org' in params: + query_params.append(('id.execution_id.org', params['id_execution_id_org'])) # noqa: E501 + + header_params = {} + + form_params = [] + local_var_files = {} + + body_params = None + # HTTP header `Accept` + header_params['Accept'] = self.api_client.select_header_accept( + ['application/json']) # noqa: E501 + + # HTTP header `Content-Type` + header_params['Content-Type'] = self.api_client.select_header_content_type( # noqa: E501 + ['application/json']) # noqa: E501 + + # Authentication setting + auth_settings = [] # noqa: E501 + + return self.api_client.call_api( + '/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow', 'GET', + path_params, + query_params, + header_params, + body=body_params, + post_params=form_params, + files=local_var_files, + response_type='AdminDynamicNodeWorkflowResponse', # noqa: E501 + auth_settings=auth_settings, + async_req=params.get('async_req'), + _return_http_data_only=params.get('_return_http_data_only'), + _preload_content=params.get('_preload_content', True), + _request_timeout=params.get('_request_timeout'), + collection_formats=collection_formats) + + def get_dynamic_node_workflow2(self, id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs): # noqa: E501 + """Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + + This method makes a synchronous HTTP request by default. To make an + asynchronous HTTP request, please pass async_req=True + >>> thread = api.get_dynamic_node_workflow2(id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, async_req=True) + >>> result = thread.get() + + :param async_req bool + :param str id_execution_id_org: Optional, org key applied to the resource. (required) + :param str id_execution_id_project: Name of the project the resource belongs to. (required) + :param str id_execution_id_domain: Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. (required) + :param str id_execution_id_name: User or system provided value for the resource. (required) + :param str id_node_id: (required) + :return: AdminDynamicNodeWorkflowResponse + If the method is called asynchronously, + returns the request thread. + """ + kwargs['_return_http_data_only'] = True + if kwargs.get('async_req'): + return self.get_dynamic_node_workflow2_with_http_info(id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs) # noqa: E501 + else: + (data) = self.get_dynamic_node_workflow2_with_http_info(id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs) # noqa: E501 + return data + + def get_dynamic_node_workflow2_with_http_info(self, id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, **kwargs): # noqa: E501 + """Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + + This method makes a synchronous HTTP request by default. To make an + asynchronous HTTP request, please pass async_req=True + >>> thread = api.get_dynamic_node_workflow2_with_http_info(id_execution_id_org, id_execution_id_project, id_execution_id_domain, id_execution_id_name, id_node_id, async_req=True) + >>> result = thread.get() + + :param async_req bool + :param str id_execution_id_org: Optional, org key applied to the resource. (required) + :param str id_execution_id_project: Name of the project the resource belongs to. (required) + :param str id_execution_id_domain: Name of the domain the resource belongs to. A domain can be considered as a subset within a specific project. (required) + :param str id_execution_id_name: User or system provided value for the resource. (required) + :param str id_node_id: (required) + :return: AdminDynamicNodeWorkflowResponse + If the method is called asynchronously, + returns the request thread. + """ + + all_params = ['id_execution_id_org', 'id_execution_id_project', 'id_execution_id_domain', 'id_execution_id_name', 'id_node_id'] # noqa: E501 + all_params.append('async_req') + all_params.append('_return_http_data_only') + all_params.append('_preload_content') + all_params.append('_request_timeout') + + params = locals() + for key, val in six.iteritems(params['kwargs']): + if key not in all_params: + raise TypeError( + "Got an unexpected keyword argument '%s'" + " to method get_dynamic_node_workflow2" % key + ) + params[key] = val + del params['kwargs'] + # verify the required parameter 'id_execution_id_org' is set + if ('id_execution_id_org' not in params or + params['id_execution_id_org'] is None): + raise ValueError("Missing the required parameter `id_execution_id_org` when calling `get_dynamic_node_workflow2`") # noqa: E501 + # verify the required parameter 'id_execution_id_project' is set + if ('id_execution_id_project' not in params or + params['id_execution_id_project'] is None): + raise ValueError("Missing the required parameter `id_execution_id_project` when calling `get_dynamic_node_workflow2`") # noqa: E501 + # verify the required parameter 'id_execution_id_domain' is set + if ('id_execution_id_domain' not in params or + params['id_execution_id_domain'] is None): + raise ValueError("Missing the required parameter `id_execution_id_domain` when calling `get_dynamic_node_workflow2`") # noqa: E501 + # verify the required parameter 'id_execution_id_name' is set + if ('id_execution_id_name' not in params or + params['id_execution_id_name'] is None): + raise ValueError("Missing the required parameter `id_execution_id_name` when calling `get_dynamic_node_workflow2`") # noqa: E501 + # verify the required parameter 'id_node_id' is set + if ('id_node_id' not in params or + params['id_node_id'] is None): + raise ValueError("Missing the required parameter `id_node_id` when calling `get_dynamic_node_workflow2`") # noqa: E501 + + collection_formats = {} + + path_params = {} + if 'id_execution_id_org' in params: + path_params['id.execution_id.org'] = params['id_execution_id_org'] # noqa: E501 + if 'id_execution_id_project' in params: + path_params['id.execution_id.project'] = params['id_execution_id_project'] # noqa: E501 + if 'id_execution_id_domain' in params: + path_params['id.execution_id.domain'] = params['id_execution_id_domain'] # noqa: E501 + if 'id_execution_id_name' in params: + path_params['id.execution_id.name'] = params['id_execution_id_name'] # noqa: E501 + if 'id_node_id' in params: + path_params['id.node_id'] = params['id_node_id'] # noqa: E501 + + query_params = [] + + header_params = {} + + form_params = [] + local_var_files = {} + + body_params = None + # HTTP header `Accept` + header_params['Accept'] = self.api_client.select_header_accept( + ['application/json']) # noqa: E501 + + # HTTP header `Content-Type` + header_params['Content-Type'] = self.api_client.select_header_content_type( # noqa: E501 + ['application/json']) # noqa: E501 + + # Authentication setting + auth_settings = [] # noqa: E501 + + return self.api_client.call_api( + '/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow', 'GET', + path_params, + query_params, + header_params, + body=body_params, + post_params=form_params, + files=local_var_files, + response_type='AdminDynamicNodeWorkflowResponse', # noqa: E501 + auth_settings=auth_settings, + async_req=params.get('async_req'), + _return_http_data_only=params.get('_return_http_data_only'), + _preload_content=params.get('_preload_content', True), + _request_timeout=params.get('_request_timeout'), + collection_formats=collection_formats) + def get_execution(self, id_project, id_domain, id_name, **kwargs): # noqa: E501 """Fetches a :ref:`ref_flyteidl.admin.Execution`. # noqa: E501 diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/__init__.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/__init__.py index f2604bcd01..f5ee9c3e3e 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/__init__.py +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/__init__.py @@ -27,6 +27,7 @@ from flyteadmin.models.admin_description_entity_list import AdminDescriptionEntityList from flyteadmin.models.admin_description_format import AdminDescriptionFormat from flyteadmin.models.admin_domain import AdminDomain +from flyteadmin.models.admin_dynamic_node_workflow_response import AdminDynamicNodeWorkflowResponse from flyteadmin.models.admin_email_notification import AdminEmailNotification from flyteadmin.models.admin_envs import AdminEnvs from flyteadmin.models.admin_execution import AdminExecution diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/admin_dynamic_node_workflow_response.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/admin_dynamic_node_workflow_response.py new file mode 100644 index 0000000000..e721c762bc --- /dev/null +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/flyteadmin/models/admin_dynamic_node_workflow_response.py @@ -0,0 +1,117 @@ +# coding: utf-8 + +""" + flyteidl/service/admin.proto + + No description provided (generated by Swagger Codegen https://github.com/swagger-api/swagger-codegen) # noqa: E501 + + OpenAPI spec version: version not set + + Generated by: https://github.com/swagger-api/swagger-codegen.git +""" + + +import pprint +import re # noqa: F401 + +import six + +from flyteadmin.models.core_compiled_workflow_closure import CoreCompiledWorkflowClosure # noqa: F401,E501 + + +class AdminDynamicNodeWorkflowResponse(object): + """NOTE: This class is auto generated by the swagger code generator program. + + Do not edit the class manually. + """ + + """ + Attributes: + swagger_types (dict): The key is attribute name + and the value is attribute type. + attribute_map (dict): The key is attribute name + and the value is json key in definition. + """ + swagger_types = { + 'compiled_workflow': 'CoreCompiledWorkflowClosure' + } + + attribute_map = { + 'compiled_workflow': 'compiled_workflow' + } + + def __init__(self, compiled_workflow=None): # noqa: E501 + """AdminDynamicNodeWorkflowResponse - a model defined in Swagger""" # noqa: E501 + + self._compiled_workflow = None + self.discriminator = None + + if compiled_workflow is not None: + self.compiled_workflow = compiled_workflow + + @property + def compiled_workflow(self): + """Gets the compiled_workflow of this AdminDynamicNodeWorkflowResponse. # noqa: E501 + + + :return: The compiled_workflow of this AdminDynamicNodeWorkflowResponse. # noqa: E501 + :rtype: CoreCompiledWorkflowClosure + """ + return self._compiled_workflow + + @compiled_workflow.setter + def compiled_workflow(self, compiled_workflow): + """Sets the compiled_workflow of this AdminDynamicNodeWorkflowResponse. + + + :param compiled_workflow: The compiled_workflow of this AdminDynamicNodeWorkflowResponse. # noqa: E501 + :type: CoreCompiledWorkflowClosure + """ + + self._compiled_workflow = compiled_workflow + + def to_dict(self): + """Returns the model properties as a dict""" + result = {} + + for attr, _ in six.iteritems(self.swagger_types): + value = getattr(self, attr) + if isinstance(value, list): + result[attr] = list(map( + lambda x: x.to_dict() if hasattr(x, "to_dict") else x, + value + )) + elif hasattr(value, "to_dict"): + result[attr] = value.to_dict() + elif isinstance(value, dict): + result[attr] = dict(map( + lambda item: (item[0], item[1].to_dict()) + if hasattr(item[1], "to_dict") else item, + value.items() + )) + else: + result[attr] = value + if issubclass(AdminDynamicNodeWorkflowResponse, dict): + for key, value in self.items(): + result[key] = value + + return result + + def to_str(self): + """Returns the string representation of the model""" + return pprint.pformat(self.to_dict()) + + def __repr__(self): + """For `print` and `pprint`""" + return self.to_str() + + def __eq__(self, other): + """Returns true if both objects are equal""" + if not isinstance(other, AdminDynamicNodeWorkflowResponse): + return False + + return self.__dict__ == other.__dict__ + + def __ne__(self, other): + """Returns true if both objects are not equal""" + return not self == other diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_dynamic_node_workflow_response.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_dynamic_node_workflow_response.py new file mode 100644 index 0000000000..c6b3cb0860 --- /dev/null +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_dynamic_node_workflow_response.py @@ -0,0 +1,40 @@ +# coding: utf-8 + +""" + flyteidl/service/admin.proto + + No description provided (generated by Swagger Codegen https://github.com/swagger-api/swagger-codegen) # noqa: E501 + + OpenAPI spec version: version not set + + Generated by: https://github.com/swagger-api/swagger-codegen.git +""" + + +from __future__ import absolute_import + +import unittest + +import flyteadmin +from flyteadmin.models.admin_dynamic_node_workflow_response import AdminDynamicNodeWorkflowResponse # noqa: E501 +from flyteadmin.rest import ApiException + + +class TestAdminDynamicNodeWorkflowResponse(unittest.TestCase): + """AdminDynamicNodeWorkflowResponse unit test stubs""" + + def setUp(self): + pass + + def tearDown(self): + pass + + def testAdminDynamicNodeWorkflowResponse(self): + """Test AdminDynamicNodeWorkflowResponse""" + # FIXME: construct object with mandatory attributes with example values + # model = flyteadmin.models.admin_dynamic_node_workflow_response.AdminDynamicNodeWorkflowResponse() # noqa: E501 + pass + + +if __name__ == '__main__': + unittest.main() diff --git a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_service_api.py b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_service_api.py index 573c16b8d3..03e412d036 100644 --- a/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_service_api.py +++ b/flyteidl/gen/pb_python/flyteidl/service/flyteadmin/test/test_admin_service_api.py @@ -197,6 +197,20 @@ def test_get_description_entity2(self): """ pass + def test_get_dynamic_node_workflow(self): + """Test case for get_dynamic_node_workflow + + Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + """ + pass + + def test_get_dynamic_node_workflow2(self): + """Test case for get_dynamic_node_workflow2 + + Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. # noqa: E501 + """ + pass + def test_get_execution(self): """Test case for get_execution diff --git a/flyteidl/gen/pb_rust/flyteidl.admin.rs b/flyteidl/gen/pb_rust/flyteidl.admin.rs index 87932411a0..7159073a8e 100644 --- a/flyteidl/gen/pb_rust/flyteidl.admin.rs +++ b/flyteidl/gen/pb_rust/flyteidl.admin.rs @@ -2276,6 +2276,18 @@ pub struct NodeExecutionGetDataResponse { #[prost(message, optional, tag="17")] pub flyte_urls: ::core::option::Option, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetDynamicNodeWorkflowRequest { + #[prost(message, optional, tag="1")] + pub id: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DynamicNodeWorkflowResponse { + #[prost(message, optional, tag="1")] + pub compiled_workflow: ::core::option::Option, +} /// Represents the Email object that is sent to a publisher/subscriber /// to forward the notification. /// Note: This is internal to Admin and doesn't need to be exposed to other components. diff --git a/flyteidl/protos/flyteidl/admin/node_execution.proto b/flyteidl/protos/flyteidl/admin/node_execution.proto index 9c80e22efe..411201ea45 100644 --- a/flyteidl/protos/flyteidl/admin/node_execution.proto +++ b/flyteidl/protos/flyteidl/admin/node_execution.proto @@ -235,3 +235,11 @@ message NodeExecutionGetDataResponse { FlyteURLs flyte_urls = 17; } + +message GetDynamicNodeWorkflowRequest { + core.NodeExecutionIdentifier id = 1; +} + +message DynamicNodeWorkflowResponse { + core.CompiledWorkflowClosure compiled_workflow = 1; +} diff --git a/flyteidl/protos/flyteidl/service/admin.proto b/flyteidl/protos/flyteidl/service/admin.proto index f03f2a1e4e..a8083ccaf1 100644 --- a/flyteidl/protos/flyteidl/service/admin.proto +++ b/flyteidl/protos/flyteidl/service/admin.proto @@ -420,6 +420,19 @@ service AdminService { // }; } + // Fetches a :ref:`ref_flyteidl.admin.DynamicNodeWorkflowResponse`. + rpc GetDynamicNodeWorkflow (flyteidl.admin.GetDynamicNodeWorkflowRequest) returns (flyteidl.admin.DynamicNodeWorkflowResponse) { + option (google.api.http) = { + get: "/api/v1/node_executions/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow" + additional_bindings { + get: "/api/v1/node_executions/org/{id.execution_id.org}/{id.execution_id.project}/{id.execution_id.domain}/{id.execution_id.name}/{id.node_id}/dynamic_workflow" + } + }; + // option (grpc.gateway.protoc_gen_swagger.options.openapiv2_operation) = { + // description: "Retrieve a workflow closure from a dynamic node execution." + // }; + } + // Fetch a list of :ref:`ref_flyteidl.admin.NodeExecution`. rpc ListNodeExecutions (flyteidl.admin.NodeExecutionListRequest) returns (flyteidl.admin.NodeExecutionList) { option (google.api.http) = { @@ -824,4 +837,4 @@ service AdminService { // description: "Retrieve metrics from an existing workflow execution." // }; }; -} \ No newline at end of file +}