From 8f516e839074d0c1013ab58650a1a0fec575d990 Mon Sep 17 00:00:00 2001 From: Yee Hing Tong Date: Thu, 25 Jan 2024 10:12:53 -0800 Subject: [PATCH] Proto changes (#4778) Signed-off-by: Yee Hing Tong --- flyte-single-binary-local.yaml | 3 - flyteadmin/pkg/artifacts/registry.go | 101 - flyteadmin/pkg/artifacts/registry_test.go | 29 - .../manager/impl/exec_manager_other_test.go | 69 - .../pkg/manager/impl/execution_manager.go | 191 +- .../manager/impl/execution_manager_test.go | 149 +- .../pkg/manager/impl/launch_plan_manager.go | 48 +- .../manager/impl/launch_plan_manager_test.go | 65 +- flyteadmin/pkg/manager/impl/task_manager.go | 39 +- .../pkg/manager/impl/task_manager_test.go | 23 +- .../pkg/manager/impl/workflow_manager.go | 47 +- .../pkg/manager/impl/workflow_manager_test.go | 31 +- flyteadmin/pkg/rpc/adminservice/base.go | 20 +- flyteidl/clients/go/admin/client.go | 8 - .../flyteidl/artifact/artifacts.grpc.pb.cc | 505 - .../flyteidl/artifact/artifacts.grpc.pb.h | 1865 -- .../pb-cpp/flyteidl/artifact/artifacts.pb.cc | 10654 -------- .../pb-cpp/flyteidl/artifact/artifacts.pb.h | 6099 ----- .../pb-cpp/flyteidl/core/artifact_id.pb.cc | 401 +- .../gen/pb-cpp/flyteidl/core/artifact_id.pb.h | 215 +- .../pb-go/flyteidl/artifact/artifacts.pb.go | 1908 -- .../flyteidl/artifact/artifacts.pb.gw.go | 390 - .../flyteidl/artifact/artifacts.swagger.json | 1783 -- .../gen/pb-go/flyteidl/core/artifact_id.pb.go | 142 +- .../pb-java/flyteidl/artifact/Artifacts.java | 22252 ---------------- .../gen/pb-java/flyteidl/core/ArtifactId.java | 1197 +- flyteidl/gen/pb-js/flyteidl.d.ts | 58 - flyteidl/gen/pb-js/flyteidl.js | 139 - .../pb_python/flyteidl/artifact/__init__.py | 0 .../flyteidl/artifact/artifacts_pb2.py | 105 - .../flyteidl/artifact/artifacts_pb2.pyi | 244 - .../flyteidl/artifact/artifacts_pb2_grpc.py | 396 - .../flyteidl/core/artifact_id_pb2.py | 4 +- .../flyteidl/core/artifact_id_pb2.pyi | 10 +- flyteidl/gen/pb_rust/flyteidl.artifact.rs | 266 - flyteidl/gen/pb_rust/flyteidl.core.rs | 11 - flyteidl/generate_protos.sh | 3 +- .../protos/flyteidl/artifact/artifacts.proto | 247 - .../protos/flyteidl/core/artifact_id.proto | 9 - 39 files changed, 229 insertions(+), 49497 deletions(-) delete mode 100644 flyteadmin/pkg/artifacts/registry.go delete mode 100644 flyteadmin/pkg/artifacts/registry_test.go delete mode 100644 flyteadmin/pkg/manager/impl/exec_manager_other_test.go delete mode 100644 flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.cc delete mode 100644 flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.h delete mode 100644 flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.cc delete mode 100644 flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.h delete mode 100644 flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.go delete mode 100644 flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.gw.go delete mode 100644 flyteidl/gen/pb-go/flyteidl/artifact/artifacts.swagger.json delete mode 100644 flyteidl/gen/pb-java/flyteidl/artifact/Artifacts.java delete mode 100644 flyteidl/gen/pb_python/flyteidl/artifact/__init__.py delete mode 100644 flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.py delete mode 100644 flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.pyi delete mode 100644 flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2_grpc.py delete mode 100644 flyteidl/gen/pb_rust/flyteidl.artifact.rs delete mode 100644 flyteidl/protos/flyteidl/artifact/artifacts.proto diff --git a/flyte-single-binary-local.yaml b/flyte-single-binary-local.yaml index ca63458b03..c369424795 100644 --- a/flyte-single-binary-local.yaml +++ b/flyte-single-binary-local.yaml @@ -6,9 +6,6 @@ admin: # and _also_, admin to talk to artifacts endpoint: localhost:30080 insecure: true -flyteadmin: - featureGates: - enableArtifacts: true catalog-cache: endpoint: localhost:8081 diff --git a/flyteadmin/pkg/artifacts/registry.go b/flyteadmin/pkg/artifacts/registry.go deleted file mode 100644 index 0ea2e11a5c..0000000000 --- a/flyteadmin/pkg/artifacts/registry.go +++ /dev/null @@ -1,101 +0,0 @@ -package artifacts - -import ( - "context" - "fmt" - - "google.golang.org/grpc" - - admin2 "github.com/flyteorg/flyte/flyteidl/clients/go/admin" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" - "github.com/flyteorg/flyte/flytestdlib/logger" -) - -// ArtifactRegistry contains a client to talk to an Artifact service and has helper methods -type ArtifactRegistry struct { - client artifact.ArtifactRegistryClient -} - -func (a *ArtifactRegistry) RegisterArtifactProducer(ctx context.Context, id *core.Identifier, ti core.TypedInterface) { - if a == nil || a.client == nil { - logger.Debugf(ctx, "Artifact client not configured, skipping registration for task [%+v]", id) - return - } - - ap := &artifact.ArtifactProducer{ - EntityId: id, - Outputs: ti.Outputs, - } - _, err := a.client.RegisterProducer(ctx, &artifact.RegisterProducerRequest{ - Producers: []*artifact.ArtifactProducer{ap}, - }) - if err != nil { - logger.Errorf(ctx, "Failed to register artifact producer for task [%+v] with err: %v", id, err) - } - logger.Debugf(ctx, "Registered artifact producer [%+v]", id) -} - -func (a *ArtifactRegistry) RegisterArtifactConsumer(ctx context.Context, id *core.Identifier, pm core.ParameterMap) { - if a == nil || a.client == nil { - logger.Debugf(ctx, "Artifact client not configured, skipping registration for consumer [%+v]", id) - return - } - ac := &artifact.ArtifactConsumer{ - EntityId: id, - Inputs: &pm, - } - _, err := a.client.RegisterConsumer(ctx, &artifact.RegisterConsumerRequest{ - Consumers: []*artifact.ArtifactConsumer{ac}, - }) - if err != nil { - logger.Errorf(ctx, "Failed to register artifact consumer for entity [%+v] with err: %v", id, err) - } - logger.Debugf(ctx, "Registered artifact consumer [%+v]", id) -} - -func (a *ArtifactRegistry) RegisterTrigger(ctx context.Context, plan *admin.LaunchPlan) error { - if a == nil || a.client == nil { - logger.Debugf(ctx, "Artifact client not configured, skipping trigger [%+v]", plan) - return fmt.Errorf("artifact client not configured") - } - _, err := a.client.CreateTrigger(ctx, &artifact.CreateTriggerRequest{ - TriggerLaunchPlan: plan, - }) - if err != nil { - logger.Errorf(ctx, "Failed to register trigger for [%+v] with err: %v", plan.Id, err) - return err - } - logger.Debugf(ctx, "Registered trigger for [%+v]", plan.Id) - return nil -} - -func (a *ArtifactRegistry) GetClient() artifact.ArtifactRegistryClient { - if a == nil { - return nil - } - return a.client -} - -// NewArtifactRegistry todo: update this to return error, and proper cfg handling. -// if nil, should either call the default config or return an error -func NewArtifactRegistry(ctx context.Context, connCfg *admin2.Config, _ ...grpc.DialOption) *ArtifactRegistry { - - if connCfg == nil { - return &ArtifactRegistry{ - client: nil, - } - } - - clients, err := admin2.NewClientsetBuilder().WithConfig(connCfg).Build(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to create Artifact client") - // too many calls to this function to update, just panic for now. - panic(err) - } - - return &ArtifactRegistry{ - client: clients.ArtifactServiceClient(), - } -} diff --git a/flyteadmin/pkg/artifacts/registry_test.go b/flyteadmin/pkg/artifacts/registry_test.go deleted file mode 100644 index 2a0e337255..0000000000 --- a/flyteadmin/pkg/artifacts/registry_test.go +++ /dev/null @@ -1,29 +0,0 @@ -package artifacts - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" -) - -func TestRegistryNoClient(t *testing.T) { - r := NewArtifactRegistry(context.Background(), nil) - assert.Nil(t, r.GetClient()) -} - -type Parent struct { - R *ArtifactRegistry -} - -func TestPointerReceivers(t *testing.T) { - p := Parent{} - nilClient := p.R.GetClient() - assert.Nil(t, nilClient) -} - -func TestNilCheck(t *testing.T) { - r := NewArtifactRegistry(context.Background(), nil) - err := r.RegisterTrigger(context.Background(), nil) - assert.NotNil(t, err) -} diff --git a/flyteadmin/pkg/manager/impl/exec_manager_other_test.go b/flyteadmin/pkg/manager/impl/exec_manager_other_test.go deleted file mode 100644 index 2534d4055f..0000000000 --- a/flyteadmin/pkg/manager/impl/exec_manager_other_test.go +++ /dev/null @@ -1,69 +0,0 @@ -package impl - -import ( - "context" - "fmt" - "testing" - - "github.com/stretchr/testify/assert" - - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" - eventWriterMocks "github.com/flyteorg/flyte/flyteadmin/pkg/async/events/mocks" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" - mockScope "github.com/flyteorg/flyte/flytestdlib/promutils" -) - -func TestResolveNotWorking(t *testing.T) { - mockConfig := getMockExecutionsConfigProvider() - - execManager := NewExecutionManager(nil, nil, mockConfig, nil, mockScope.NewTestScope(), mockScope.NewTestScope(), nil, nil, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)).(*ExecutionManager) - - pm, artifactIDs, err := execManager.ResolveParameterMapArtifacts(context.Background(), nil, nil) - assert.Nil(t, err) - fmt.Println(pm, artifactIDs) - -} - -func TestTrackingBitExtract(t *testing.T) { - mockConfig := getMockExecutionsConfigProvider() - - execManager := NewExecutionManager(nil, nil, mockConfig, nil, mockScope.NewTestScope(), mockScope.NewTestScope(), nil, nil, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)).(*ExecutionManager) - - lit := core.Literal{ - Value: &core.Literal_Scalar{ - Scalar: &core.Scalar{ - Value: &core.Scalar_Primitive{ - Primitive: &core.Primitive{ - Value: &core.Primitive_Integer{ - Integer: 1, - }, - }, - }, - }, - }, - Metadata: map[string]string{"_ua": "proj/domain/name@version"}, - } - inputMap := core.LiteralMap{ - Literals: map[string]*core.Literal{ - "a": &lit, - }, - } - inputColl := core.LiteralCollection{ - Literals: []*core.Literal{ - &lit, - }, - } - - var trackers = make(map[string]string) - execManager.ExtractArtifactTrackers(trackers, &lit) - assert.Equal(t, 1, len(trackers)) - - trackers = make(map[string]string) - execManager.ExtractArtifactTrackers(trackers, &core.Literal{Value: &core.Literal_Map{Map: &inputMap}}) - assert.Equal(t, 1, len(trackers)) - - trackers = make(map[string]string) - execManager.ExtractArtifactTrackers(trackers, &core.Literal{Value: &core.Literal_Collection{Collection: &inputColl}}) - assert.Equal(t, 1, len(trackers)) - assert.Equal(t, "", trackers["proj/domain/name@version"]) -} diff --git a/flyteadmin/pkg/manager/impl/execution_manager.go b/flyteadmin/pkg/manager/impl/execution_manager.go index 37dcf2e6d4..b61dd93788 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager.go +++ b/flyteadmin/pkg/manager/impl/execution_manager.go @@ -14,7 +14,6 @@ import ( "google.golang.org/grpc/codes" "github.com/flyteorg/flyte/flyteadmin/auth" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" cloudeventInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/async/cloudevent/interfaces" eventWriter "github.com/flyteorg/flyte/flyteadmin/pkg/async/events/interfaces" "github.com/flyteorg/flyte/flyteadmin/pkg/async/notifications" @@ -35,9 +34,7 @@ import ( workflowengineInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/workflowengine/interfaces" "github.com/flyteorg/flyte/flyteadmin/plugins" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/flytek8s" "github.com/flyteorg/flyte/flytestdlib/contextutils" "github.com/flyteorg/flyte/flytestdlib/logger" @@ -47,7 +44,6 @@ import ( ) const childContainerQueueKey = "child_queue" -const artifactTrackerKey = "_ua" // Map of [project] -> map of [domain] -> stop watch type projectDomainScopedStopWatchMap = map[string]map[string]*promutils.StopWatch @@ -95,7 +91,6 @@ type ExecutionManager struct { cloudEventPublisher notificationInterfaces.Publisher dbEventWriter eventWriter.WorkflowExecutionEventWriter pluginRegistry *plugins.Registry - artifactRegistry *artifacts.ArtifactRegistry } func getExecutionContext(ctx context.Context, id *core.WorkflowExecutionIdentifier) context.Context { @@ -701,28 +696,6 @@ func resolveSecurityCtx(ctx context.Context, executionConfigSecurityCtx *core.Se } } -// ExtractArtifactTrackers pulls out artifact tracker strings from Literals for lineage -func (m *ExecutionManager) ExtractArtifactTrackers(artifactTrackers map[string]string, input *core.Literal) { - - if input == nil { - return - } - if input.GetMetadata() != nil { - if tracker, ok := input.GetMetadata()[artifactTrackerKey]; ok { - artifactTrackers[tracker] = "" - } - } - if input.GetCollection() != nil { - for _, v := range input.GetCollection().Literals { - m.ExtractArtifactTrackers(artifactTrackers, v) - } - } else if input.GetMap() != nil { - for _, v := range input.GetMap().Literals { - m.ExtractArtifactTrackers(artifactTrackers, v) - } - } -} - // getStringFromInput should be called when a tag or partition value is a binding to an input. the input is looked up // from the input map and the binding, and an error is returned if the input key is not in the map. func (m *ExecutionManager) getStringFromInput(ctx context.Context, inputBinding core.InputBindingData, inputs map[string]*core.Literal) (string, error) { @@ -857,86 +830,10 @@ func (m *ExecutionManager) fillInTemplateArgs(ctx context.Context, query core.Ar return query, errors.NewFlyteAdminErrorf(codes.InvalidArgument, "query doesn't have uri, tag, or id") } -// ResolveParameterMapArtifacts will go through the parameter map, and resolve any artifact queries. -func (m *ExecutionManager) ResolveParameterMapArtifacts(ctx context.Context, inputs *core.ParameterMap, inputsForQueryTemplating map[string]*core.Literal) (*core.ParameterMap, []*core.ArtifactID, error) { - - // only top level replace for now. Need to make this recursive - var artifactIDs []*core.ArtifactID - if inputs == nil { - return nil, artifactIDs, nil - } - outputs := map[string]*core.Parameter{} - - for k, v := range inputs.Parameters { - if inputsForQueryTemplating != nil { - if _, ok := inputsForQueryTemplating[k]; ok { - // Mark these as required as they're already provided by the other two LiteralMaps - outputs[k] = &core.Parameter{ - Var: v.Var, - Behavior: &core.Parameter_Required{Required: true}, - } - continue - } - } - if v.GetArtifactQuery() != nil { - // This case handles when an Artifact query is specified as a default value. - if m.artifactRegistry.GetClient() == nil { - return nil, nil, errors.NewFlyteAdminErrorf(codes.Internal, "artifact client is not initialized, can't resolve queries") - } - filledInQuery, err := m.fillInTemplateArgs(ctx, *v.GetArtifactQuery(), inputsForQueryTemplating) - if err != nil { - logger.Errorf(ctx, "Failed to fill in template args for [%s] [%v]", k, err) - return nil, nil, err - } - req := &artifact.GetArtifactRequest{ - Query: &filledInQuery, - Details: false, - } - - resp, err := m.artifactRegistry.GetClient().GetArtifact(ctx, req) - if err != nil { - return nil, nil, err - } - artifactIDs = append(artifactIDs, resp.Artifact.GetArtifactId()) - logger.Debugf(ctx, "Resolved query for [%s] to [%+v]", k, resp.Artifact.ArtifactId) - outputs[k] = &core.Parameter{ - Var: v.Var, - Behavior: &core.Parameter_Default{Default: resp.Artifact.Spec.Value}, - } - } else if v.GetArtifactId() != nil { - // This case is for when someone hard-codes a known ArtifactID as a default value. - req := &artifact.GetArtifactRequest{ - Query: &core.ArtifactQuery{ - Identifier: &core.ArtifactQuery_ArtifactId{ - ArtifactId: v.GetArtifactId(), - }, - }, - Details: false, - } - resp, err := m.artifactRegistry.GetClient().GetArtifact(ctx, req) - if err != nil { - return nil, nil, err - } - artifactIDs = append(artifactIDs, v.GetArtifactId()) - logger.Debugf(ctx, "Using specified artifactID for [%+v] for [%s]", v.GetArtifactId(), k) - outputs[k] = &core.Parameter{ - Var: v.Var, - Behavior: &core.Parameter_Default{Default: resp.Artifact.Spec.Value}, - } - } else { - outputs[k] = v - } - } - pm := &core.ParameterMap{Parameters: outputs} - return pm, artifactIDs, nil -} - func (m *ExecutionManager) launchExecutionAndPrepareModel( ctx context.Context, request admin.ExecutionCreateRequest, requestedAt time.Time) ( context.Context, *models.Execution, error) { - ctxPD := contextutils.WithProjectDomain(ctx, request.Project, request.Domain) - err := validation.ValidateExecutionRequest(ctx, request, m.db, m.config.ApplicationConfiguration()) if err != nil { logger.Debugf(ctx, "Failed to validate ExecutionCreateRequest %+v with err %v", request, err) @@ -960,55 +857,9 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( return nil, nil, err } - // TODO: Artifact feature gate, remove when ready var lpExpectedInputs *core.ParameterMap - var artifactTrackers = make(map[string]string) var usedArtifactIDs []*core.ArtifactID - if m.artifactRegistry.GetClient() != nil { - // Literals may have an artifact key in the metadata field. This is something the artifact service should have - // added. Pull these back out so we can keep track of them for lineage purposes. Use a dummy wrapper object for - // easier recursion. - requestInputMap := &core.Literal{ - Value: &core.Literal_Map{Map: request.Inputs}, - } - fixedInputMap := &core.Literal{ - Value: &core.Literal_Map{Map: launchPlan.Spec.FixedInputs}, - } - m.ExtractArtifactTrackers(artifactTrackers, requestInputMap) - m.ExtractArtifactTrackers(artifactTrackers, fixedInputMap) - - // Put together the inputs that we've already resolved so that the artifact querying bit can fill them in. - // This is to support artifact queries that depend on other inputs using the {{ .inputs.var }} construct. - var inputsForQueryTemplating = make(map[string]*core.Literal) - if request.Inputs != nil { - for k, v := range request.Inputs.Literals { - inputsForQueryTemplating[k] = v - } - } - for k, v := range launchPlan.Spec.FixedInputs.Literals { - inputsForQueryTemplating[k] = v - } - logger.Debugf(ctx, "Inputs for query templating: [%+v]", inputsForQueryTemplating) - - // Resolve artifact queries - // Within the launch plan, the artifact will be in the Parameter map, and can come in form of an ArtifactID, - // or as an ArtifactQuery. - // Also send in the inputsForQueryTemplating for two reasons, so we don't run queries for things we don't need to - // and so we can fill in template args. - // ArtifactIDs are also returned for lineage purposes. - lpExpectedInputs, usedArtifactIDs, err = m.ResolveParameterMapArtifacts(ctxPD, launchPlan.Closure.ExpectedInputs, inputsForQueryTemplating) - if err != nil { - logger.Errorf(ctx, "Error looking up launch plan closure parameter map: %v", err) - return nil, nil, err - } - - logger.Debugf(ctx, "Resolved launch plan closure expected inputs from [%+v] to [%+v]", launchPlan.Closure.ExpectedInputs, lpExpectedInputs) - logger.Debugf(ctx, "Found artifact trackers: %v", artifactTrackers) - logger.Debugf(ctx, "Found artifact IDs: %v", usedArtifactIDs) - - } else { - lpExpectedInputs = launchPlan.Closure.ExpectedInputs - } + lpExpectedInputs = launchPlan.Closure.ExpectedInputs // Artifacts retrieved will need to be stored somewhere to ensure that we can re-emit events if necessary // in the future, and also to make sure that relaunch and recover can use it if necessary. @@ -1155,13 +1006,6 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( notificationsSettings = make([]*admin.Notification, 0) } - // Publish of event is also gated on the artifact client being available, even though it's not directly required. - // TODO: Artifact feature gate, remove when ready - if m.artifactRegistry.GetClient() != nil { - // TODO: Add principal - m.publishExecutionStart(ctx, workflowExecutionID, request.Spec.LaunchPlan, workflow.Id, artifactTrackers, usedArtifactIDs) - } - createExecModelInput := transformers.CreateExecutionModelInput{ WorkflowExecutionID: workflowExecutionID, RequestSpec: requestSpec, @@ -1211,36 +1055,6 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( return ctx, executionModel, nil } -// publishExecutionStart is an event that Admin publishes for artifact lineage. -func (m *ExecutionManager) publishExecutionStart(ctx context.Context, executionID core.WorkflowExecutionIdentifier, - launchPlanID *core.Identifier, workflowID *core.Identifier, artifactTrackers map[string]string, usedArtifactIDs []*core.ArtifactID) { - - var artifactTrackerList []string - // Use a list instead of the fake set - for k := range artifactTrackers { - artifactTrackerList = append(artifactTrackerList, k) - } - - if len(artifactTrackerList) > 0 || len(usedArtifactIDs) > 0 { - logger.Debugf(ctx, "Sending execution start event for execution [%+v] with trackers [%+v] and artifact ids [%+v]", executionID, artifactTrackerList, usedArtifactIDs) - - request := event.CloudEventExecutionStart{ - ExecutionId: &executionID, - LaunchPlanId: launchPlanID, - WorkflowId: workflowID, - ArtifactIds: usedArtifactIDs, - ArtifactTrackers: artifactTrackerList, - } - go func() { - ceCtx := context.TODO() - if err := m.cloudEventPublisher.Publish(ceCtx, proto.MessageName(&request), &request); err != nil { - m.systemMetrics.PublishEventError.Inc() - logger.Infof(ctx, "error publishing cloud event [%+v] with err: [%v]", request, err) - } - }() - } -} - // Inserts an execution model into the database store and emits platform metrics. func (m *ExecutionManager) createExecutionModel( ctx context.Context, executionModel *models.Execution) (*core.WorkflowExecutionIdentifier, error) { @@ -1960,7 +1774,7 @@ func NewExecutionManager(db repositoryInterfaces.Repository, pluginRegistry *plu publisher notificationInterfaces.Publisher, urlData dataInterfaces.RemoteURLInterface, workflowManager interfaces.WorkflowInterface, namedEntityManager interfaces.NamedEntityInterface, eventPublisher notificationInterfaces.Publisher, cloudEventPublisher cloudeventInterfaces.Publisher, - eventWriter eventWriter.WorkflowExecutionEventWriter, artifactRegistry *artifacts.ArtifactRegistry) interfaces.ExecutionInterface { + eventWriter eventWriter.WorkflowExecutionEventWriter) interfaces.ExecutionInterface { queueAllocator := executions.NewQueueAllocator(config, db) systemMetrics := newExecutionSystemMetrics(systemScope) @@ -1994,7 +1808,6 @@ func NewExecutionManager(db repositoryInterfaces.Repository, pluginRegistry *plu cloudEventPublisher: cloudEventPublisher, dbEventWriter: eventWriter, pluginRegistry: pluginRegistry, - artifactRegistry: artifactRegistry, } } diff --git a/flyteadmin/pkg/manager/impl/execution_manager_test.go b/flyteadmin/pkg/manager/impl/execution_manager_test.go index 72c700008f..f1c5bb9eb5 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager_test.go +++ b/flyteadmin/pkg/manager/impl/execution_manager_test.go @@ -22,7 +22,6 @@ import ( "k8s.io/apimachinery/pkg/util/sets" "github.com/flyteorg/flyte/flyteadmin/auth" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" eventWriterMocks "github.com/flyteorg/flyte/flyteadmin/pkg/async/events/mocks" notificationMocks "github.com/flyteorg/flyte/flyteadmin/pkg/async/notifications/mocks" "github.com/flyteorg/flyte/flyteadmin/pkg/common" @@ -376,7 +375,7 @@ func TestCreateExecution(t *testing.T) { mockConfig := getMockExecutionsConfigProvider() mockConfig.(*runtimeMocks.MockConfigurationProvider).AddQualityOfServiceConfiguration(qosProvider) - execManager := NewExecutionManager(repository, r, mockConfig, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, mockConfig, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Spec.Metadata = &admin.ExecutionMetadata{ Principal: "unused - populated from authenticated context", @@ -473,7 +472,7 @@ func TestCreateExecutionFromWorkflowNode(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Spec.Metadata = &admin.ExecutionMetadata{ Mode: admin.ExecutionMetadata_CHILD_WORKFLOW, @@ -510,7 +509,7 @@ func TestCreateExecution_NoAssignedName(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Name = "" response, err := execManager.CreateExecution(context.Background(), request, requestedAt) @@ -561,7 +560,7 @@ func TestCreateExecution_TaggedQueue(t *testing.T) { mockExecutor.OnID().Return("customMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, configProvider, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, configProvider, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() response, err := execManager.CreateExecution(context.Background(), request, requestedAt) @@ -579,7 +578,7 @@ func TestCreateExecutionValidationError(t *testing.T) { setDefaultLpCallbackForExecTest(repository) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Domain = "" @@ -593,7 +592,7 @@ func TestCreateExecution_InvalidLpIdentifier(t *testing.T) { setDefaultLpCallbackForExecTest(repository) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Spec.LaunchPlan = nil @@ -607,7 +606,7 @@ func TestCreateExecutionInCompatibleInputs(t *testing.T) { setDefaultLpCallbackForExecTest(repository) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Inputs = &core.LiteralMap{ @@ -660,7 +659,7 @@ func TestCreateExecutionPropellerFailure(t *testing.T) { identity, err := auth.NewIdentityContext("", principal, "", time.Now(), sets.NewString(), nil, nil) assert.NoError(t, err) ctx := identity.WithContext(context.Background()) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) expectedResponse := &admin.ExecutionCreateResponse{Id: &executionIdentifier} @@ -684,7 +683,7 @@ func TestCreateExecutionDatabaseFailure(t *testing.T) { } repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetCreateCallback(exCreateFunc) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() response, err := execManager.CreateExecution(context.Background(), request, requestedAt) @@ -752,7 +751,7 @@ func TestCreateExecutionVerifyDbModel(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager)._clock = mockClock @@ -795,7 +794,7 @@ func TestCreateExecutionDefaultNotifications(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) response, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -834,7 +833,7 @@ func TestCreateExecutionDisableNotifications(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) response, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -904,7 +903,7 @@ func TestCreateExecutionNoNotifications(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) response, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -933,7 +932,7 @@ func TestCreateExecutionDynamicLabelsAndAnnotations(t *testing.T) { mockExecutor.OnID().Return("customMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Spec.Labels = &admin.Labels{ Values: map[string]string{ @@ -1052,7 +1051,7 @@ func TestCreateExecutionInterruptible(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -1132,7 +1131,7 @@ func TestCreateExecutionOverwriteCache(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -1216,7 +1215,7 @@ func TestCreateExecutionWithEnvs(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.CreateExecution(context.Background(), request, requestedAt) assert.Nil(t, err) @@ -1258,7 +1257,7 @@ func TestCreateExecution_CustomNamespaceMappingConfig(t *testing.T) { mockExecutionsConfigProvider.(*runtimeMocks.MockConfigurationProvider).AddRegistrationValidationConfiguration( runtimeMocks.NewMockRegistrationValidationProvider()) - execManager := NewExecutionManager(repository, r, mockExecutionsConfigProvider, storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, mockExecutionsConfigProvider, storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager)._clock = mockClock @@ -1431,7 +1430,7 @@ func TestRelaunchExecution(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1491,7 +1490,7 @@ func TestRelaunchExecution_GetExistingFailure(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) expectedErr := errors.New("expected error") repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback( @@ -1530,7 +1529,7 @@ func TestRelaunchExecution_CreateFailure(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1570,7 +1569,7 @@ func TestRelaunchExecutionInterruptibleOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1621,7 +1620,7 @@ func TestRelaunchExecutionOverwriteCacheOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1744,7 +1743,7 @@ func TestRelaunchExecutionEnvsOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1796,7 +1795,7 @@ func TestRecoverExecution(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1855,7 +1854,7 @@ func TestRecoverExecution_RecoveredChildNode(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -1956,7 +1955,7 @@ func TestRecoverExecution_GetExistingFailure(t *testing.T) { setDefaultLpCallbackForExecTest(repository) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) expectedErr := errors.New("expected error") repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback( @@ -1999,7 +1998,7 @@ func TestRecoverExecution_GetExistingInputsFailure(t *testing.T) { } r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -2033,7 +2032,7 @@ func TestRecoverExecutionInterruptibleOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -2096,7 +2095,7 @@ func TestRecoverExecutionOverwriteCacheOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -2157,7 +2156,7 @@ func TestRecoverExecutionEnvsOverride(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := admin.ExecutionClosure{ @@ -2270,7 +2269,7 @@ func TestCreateWorkflowEvent(t *testing.T) { mockDbEventWriter.On("Write", request) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter) resp, err := execManager.CreateWorkflowEvent(context.Background(), request) assert.Nil(t, err) assert.NotNil(t, resp) @@ -2300,7 +2299,7 @@ func TestCreateWorkflowEvent_TerminalState(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecutionFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", @@ -2340,7 +2339,7 @@ func TestCreateWorkflowEvent_NoRunningToQueued(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecutionFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", @@ -2388,7 +2387,7 @@ func TestCreateWorkflowEvent_CurrentlyAborting(t *testing.T) { mockDbEventWriter.On("Write", req) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter) resp, err := execManager.CreateWorkflowEvent(context.Background(), req) assert.NotNil(t, resp) @@ -2456,7 +2455,7 @@ func TestCreateWorkflowEvent_StartedRunning(t *testing.T) { mockDbEventWriter.On("Write", request) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter) resp, err := execManager.CreateWorkflowEvent(context.Background(), request) assert.Nil(t, err) assert.NotNil(t, resp) @@ -2489,7 +2488,7 @@ func TestCreateWorkflowEvent_DuplicateRunning(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) occurredAtTimestamp, _ := ptypes.TimestampProto(occurredAt) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", @@ -2532,7 +2531,7 @@ func TestCreateWorkflowEvent_InvalidPhaseChange(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) occurredAtTimestamp, _ := ptypes.TimestampProto(occurredAt) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", @@ -2599,7 +2598,7 @@ func TestCreateWorkflowEvent_ClusterReassignmentOnQueued(t *testing.T) { mockDbEventWriter.On("Write", request) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, &mockPublisher, mockDbEventWriter) resp, err := execManager.CreateWorkflowEvent(context.Background(), request) assert.Nil(t, err) @@ -2622,7 +2621,7 @@ func TestCreateWorkflowEvent_InvalidEvent(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecutionFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", Event: &event.WorkflowExecutionEvent{ @@ -2652,7 +2651,7 @@ func TestCreateWorkflowEvent_UpdateModelError(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", Event: &event.WorkflowExecutionEvent{ @@ -2687,7 +2686,7 @@ func TestCreateWorkflowEvent_DatabaseGetError(t *testing.T) { } r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", Event: &event.WorkflowExecutionEvent{ @@ -2723,7 +2722,7 @@ func TestCreateWorkflowEvent_DatabaseUpdateError(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecutionFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", Event: &event.WorkflowExecutionEvent{ @@ -2768,7 +2767,7 @@ func TestCreateWorkflowEvent_IncompatibleCluster(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) occurredAtTimestamp, _ := ptypes.TimestampProto(occurredAt) resp, err := execManager.CreateWorkflowEvent(context.Background(), admin.WorkflowExecutionEventRequest{ RequestId: "1", @@ -2826,7 +2825,7 @@ func TestGetExecution(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(executionGetFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execution, err := execManager.GetExecution(context.Background(), admin.WorkflowExecutionGetRequest{ Id: &executionIdentifier, }) @@ -2849,7 +2848,7 @@ func TestGetExecution_DatabaseError(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(executionGetFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execution, err := execManager.GetExecution(context.Background(), admin.WorkflowExecutionGetRequest{ Id: &executionIdentifier, }) @@ -2881,7 +2880,7 @@ func TestGetExecution_TransformerError(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(executionGetFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execution, err := execManager.GetExecution(context.Background(), admin.WorkflowExecutionGetRequest{ Id: &executionIdentifier, }) @@ -2894,7 +2893,7 @@ func TestUpdateExecution(t *testing.T) { repository := repositoryMocks.NewMockRepository() r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.UpdateExecution(context.Background(), admin.ExecutionUpdateRequest{ Id: &core.WorkflowExecutionIdentifier{ Project: "project", @@ -2916,7 +2915,7 @@ func TestUpdateExecution(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) updateResponse, err := execManager.UpdateExecution(context.Background(), admin.ExecutionUpdateRequest{ Id: &executionIdentifier, }, time.Now()) @@ -2937,7 +2936,7 @@ func TestUpdateExecution(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) updateResponse, err := execManager.UpdateExecution(context.Background(), admin.ExecutionUpdateRequest{ Id: &executionIdentifier, State: admin.ExecutionState_EXECUTION_ARCHIVED, @@ -2955,7 +2954,7 @@ func TestUpdateExecution(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetUpdateCallback(updateExecFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.UpdateExecution(context.Background(), admin.ExecutionUpdateRequest{ Id: &executionIdentifier, State: admin.ExecutionState_EXECUTION_ARCHIVED, @@ -2972,7 +2971,7 @@ func TestUpdateExecution(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(getExecFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.UpdateExecution(context.Background(), admin.ExecutionUpdateRequest{ Id: &executionIdentifier, State: admin.ExecutionState_EXECUTION_ARCHIVED, @@ -3042,7 +3041,7 @@ func TestListExecutions(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetListCallback(executionListFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) executionList, err := execManager.ListExecutions(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ @@ -3075,7 +3074,7 @@ func TestListExecutions(t *testing.T) { func TestListExecutions_MissingParameters(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.ListExecutions(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Domain: domainValue, @@ -3114,7 +3113,7 @@ func TestListExecutions_DatabaseError(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetListCallback(executionListFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.ListExecutions(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Project: projectValue, @@ -3147,7 +3146,7 @@ func TestListExecutions_TransformerError(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetListCallback(executionListFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) executionList, err := execManager.ListExecutions(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ @@ -3450,7 +3449,7 @@ func TestTerminateExecution(t *testing.T) { mockExecutor.OnID().Return("customMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) identity, err := auth.NewIdentityContext("", principal, "", time.Now(), sets.NewString(), nil, nil) assert.NoError(t, err) @@ -3485,7 +3484,7 @@ func TestTerminateExecution_PropellerError(t *testing.T) { assert.Equal(t, core.WorkflowExecution_ABORTING.String(), execution.Phase) return nil }) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.TerminateExecution(context.Background(), admin.ExecutionTerminateRequest{ Id: &core.WorkflowExecutionIdentifier{ @@ -3517,7 +3516,7 @@ func TestTerminateExecution_DatabaseError(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.TerminateExecution(context.Background(), admin.ExecutionTerminateRequest{ Id: &core.WorkflowExecutionIdentifier{ Project: "project", @@ -3547,7 +3546,7 @@ func TestTerminateExecution_AlreadyTerminated(t *testing.T) { Phase: core.WorkflowExecution_SUCCEEDED.String(), }, nil }) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) resp, err := execManager.TerminateExecution(context.Background(), admin.ExecutionTerminateRequest{ Id: &core.WorkflowExecutionIdentifier{ Project: "project", @@ -3639,7 +3638,7 @@ func TestGetExecutionData(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(executionGetFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) dataResponse, err := execManager.GetExecutionData(context.Background(), admin.WorkflowExecutionGetDataRequest{ Id: &executionIdentifier, }) @@ -3704,7 +3703,7 @@ func TestAddPluginOverrides(t *testing.T) { } r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(db, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(db, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) taskPluginOverrides, err := execManager.(*ExecutionManager).addPluginOverrides( context.Background(), executionID, workflowName, launchPlanName) @@ -3737,7 +3736,7 @@ func TestPluginOverrides_ResourceGetFailure(t *testing.T) { } r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(db, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(db, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) _, err := execManager.(*ExecutionManager).addPluginOverrides( context.Background(), executionID, workflowName, launchPlanName) @@ -3771,7 +3770,7 @@ func TestGetExecution_Legacy(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetGetCallback(executionGetFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execution, err := execManager.GetExecution(context.Background(), admin.WorkflowExecutionGetRequest{ Id: &executionIdentifier, }) @@ -3834,7 +3833,7 @@ func TestGetExecutionData_LegacyModel(t *testing.T) { storageClient := getMockStorageForExecTest(context.Background()) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) dataResponse, err := execManager.GetExecutionData(context.Background(), admin.WorkflowExecutionGetDataRequest{ Id: &executionIdentifier, }) @@ -3882,7 +3881,7 @@ func TestCreateExecution_LegacyClient(t *testing.T) { mockExecutor.OnID().Return("customMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) response, err := execManager.CreateExecution(context.Background(), *getLegacyExecutionRequest(), requestedAt) assert.Nil(t, err) @@ -3904,7 +3903,7 @@ func TestRelaunchExecution_LegacyModel(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), storageClient, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) startTime := time.Now() startTimeProto, _ := ptypes.TimestampProto(startTime) existingClosure := getLegacyClosure() @@ -4024,7 +4023,7 @@ func TestListExecutions_LegacyModel(t *testing.T) { repository.ExecutionRepo().(*repositoryMocks.MockExecutionRepo).SetListCallback(executionListFunc) r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) executionList, err := execManager.ListExecutions(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ @@ -4080,7 +4079,7 @@ func TestSetDefaults(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager).setCompiledTaskDefaults(context.Background(), task, workflowengineInterfaces.TaskResources{ Defaults: runtimeInterfaces.TaskResourceSet{ CPU: resource.MustParse("200m"), @@ -4165,7 +4164,7 @@ func TestSetDefaults_MissingRequests_ExistingRequestsPreserved(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager).setCompiledTaskDefaults(context.Background(), task, workflowengineInterfaces.TaskResources{ Defaults: runtimeInterfaces.TaskResourceSet{ CPU: resource.MustParse("200m"), @@ -4243,7 +4242,7 @@ func TestSetDefaults_OptionalRequiredResources(t *testing.T) { t.Run("don't inject ephemeral storage or gpu when only the limit is set in config", func(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager).setCompiledTaskDefaults(context.Background(), task, workflowengineInterfaces.TaskResources{ Defaults: runtimeInterfaces.TaskResourceSet{ CPU: resource.MustParse("200m"), @@ -4282,7 +4281,7 @@ func TestSetDefaults_OptionalRequiredResources(t *testing.T) { t.Run("respect non-required resources when defaults exist in config", func(t *testing.T) { r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) - execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repositoryMocks.NewMockRepository(), r, getMockExecutionsConfigProvider(), getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) execManager.(*ExecutionManager).setCompiledTaskDefaults(context.Background(), task, workflowengineInterfaces.TaskResources{ Limits: taskConfigLimits, Defaults: runtimeInterfaces.TaskResourceSet{ @@ -4480,7 +4479,7 @@ func TestCreateSingleTaskExecution(t *testing.T) { workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorage, - storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + storagePrefix, mockScope.NewTestScope()) namedEntityManager := NewNamedEntityManager(repository, getMockConfigForNETest(), mockScope.NewTestScope()) mockExecutor := workflowengineMocks.WorkflowExecutor{} @@ -4488,7 +4487,7 @@ func TestCreateSingleTaskExecution(t *testing.T) { mockExecutor.OnID().Return("testMockExecutor") r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &mockExecutor) - execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, workflowManager, namedEntityManager, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}, artifacts.NewArtifactRegistry(context.Background(), nil)) + execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, workflowManager, namedEntityManager, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := admin.ExecutionCreateRequest{ Project: "flytekit", Domain: "production", diff --git a/flyteadmin/pkg/manager/impl/launch_plan_manager.go b/flyteadmin/pkg/manager/impl/launch_plan_manager.go index 48441905a1..093b4d7cce 100644 --- a/flyteadmin/pkg/manager/impl/launch_plan_manager.go +++ b/flyteadmin/pkg/manager/impl/launch_plan_manager.go @@ -9,7 +9,6 @@ import ( "github.com/prometheus/client_golang/prometheus" "google.golang.org/grpc/codes" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" scheduleInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/async/schedule/interfaces" "github.com/flyteorg/flyte/flyteadmin/pkg/common" "github.com/flyteorg/flyte/flyteadmin/pkg/errors" @@ -35,11 +34,10 @@ type launchPlanMetrics struct { } type LaunchPlanManager struct { - db repoInterfaces.Repository - config runtimeInterfaces.Configuration - scheduler scheduleInterfaces.EventScheduler - metrics launchPlanMetrics - artifactRegistry *artifacts.ArtifactRegistry + db repoInterfaces.Repository + config runtimeInterfaces.Configuration + scheduler scheduleInterfaces.EventScheduler + metrics launchPlanMetrics } func getLaunchPlanContext(ctx context.Context, identifier *core.Identifier) context.Context { @@ -87,21 +85,6 @@ func (m *LaunchPlanManager) CreateLaunchPlan( return nil, err } - // The presence of this field indicates that this is a trigger launch plan - // Return true and send this request over to the artifact registry instead - if launchPlan.Spec.GetEntityMetadata() != nil && launchPlan.Spec.GetEntityMetadata().GetLaunchConditions() != nil { - // TODO: Artifact feature gate, remove when ready - if m.artifactRegistry.GetClient() == nil { - logger.Debugf(ctx, "artifact feature not enabled, skipping launch plan %v", launchPlan.Id) - return &admin.LaunchPlanCreateResponse{}, nil - } - err := m.artifactRegistry.RegisterTrigger(ctx, &launchPlan) - if err != nil { - return nil, err - } - return &admin.LaunchPlanCreateResponse{}, nil - } - existingLaunchPlanModel, err := util.GetLaunchPlanModel(ctx, m.db, *request.Id) if err == nil { if bytes.Equal(existingLaunchPlanModel.Digest, launchPlanDigest) { @@ -128,17 +111,6 @@ func (m *LaunchPlanManager) CreateLaunchPlan( } m.metrics.SpecSizeBytes.Observe(float64(len(launchPlanModel.Spec))) m.metrics.ClosureSizeBytes.Observe(float64(len(launchPlanModel.Closure))) - // TODO: Artifact feature gate, remove when ready - if m.artifactRegistry.GetClient() != nil { - go func() { - ceCtx := context.TODO() - if launchPlan.Spec.DefaultInputs == nil { - logger.Debugf(ceCtx, "Insufficient fields to submit launchplan interface %v", launchPlan.Id) - return - } - m.artifactRegistry.RegisterArtifactConsumer(ceCtx, launchPlan.Id, *launchPlan.Spec.DefaultInputs) - }() - } return &admin.LaunchPlanCreateResponse{}, nil } @@ -573,8 +545,7 @@ func NewLaunchPlanManager( db repoInterfaces.Repository, config runtimeInterfaces.Configuration, scheduler scheduleInterfaces.EventScheduler, - scope promutils.Scope, - artifactRegistry *artifacts.ArtifactRegistry) interfaces.LaunchPlanInterface { + scope promutils.Scope) interfaces.LaunchPlanInterface { metrics := launchPlanMetrics{ Scope: scope, @@ -584,10 +555,9 @@ func NewLaunchPlanManager( ClosureSizeBytes: scope.MustNewSummary("closure_size_bytes", "size in bytes of serialized launch plan closure"), } return &LaunchPlanManager{ - db: db, - config: config, - scheduler: scheduler, - metrics: metrics, - artifactRegistry: artifactRegistry, + db: db, + config: config, + scheduler: scheduler, + metrics: metrics, } } diff --git a/flyteadmin/pkg/manager/impl/launch_plan_manager_test.go b/flyteadmin/pkg/manager/impl/launch_plan_manager_test.go index 64e069f26d..2863f2747d 100644 --- a/flyteadmin/pkg/manager/impl/launch_plan_manager_test.go +++ b/flyteadmin/pkg/manager/impl/launch_plan_manager_test.go @@ -12,7 +12,6 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc/codes" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" scheduleInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/async/schedule/interfaces" "github.com/flyteorg/flyte/flyteadmin/pkg/async/schedule/mocks" "github.com/flyteorg/flyte/flyteadmin/pkg/common" @@ -90,7 +89,7 @@ func TestCreateLaunchPlan(t *testing.T) { return nil }) setDefaultWorkflowCallbackForLpTest(repository) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() response, err := lpManager.CreateLaunchPlan(context.Background(), request) assert.Nil(t, err) @@ -102,7 +101,7 @@ func TestCreateLaunchPlan(t *testing.T) { func TestLaunchPlanManager_GetLaunchPlan(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) state := int32(0) lpRequest := testutils.GetLaunchPlanRequest() workflowRequest := testutils.GetWorkflowRequest() @@ -138,7 +137,7 @@ func TestLaunchPlanManager_GetLaunchPlan(t *testing.T) { func TestLaunchPlanManager_GetActiveLaunchPlan(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) state := int32(1) lpRequest := testutils.GetLaunchPlanRequest() workflowRequest := testutils.GetWorkflowRequest() @@ -197,7 +196,7 @@ func TestLaunchPlanManager_GetActiveLaunchPlan(t *testing.T) { func TestLaunchPlanManager_GetActiveLaunchPlan_NoneActive(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) lpRequest := testutils.GetLaunchPlanRequest() launchPlanListFunc := func(input interfaces.ListResourceInput) (interfaces.LaunchPlanCollectionOutput, error) { @@ -217,7 +216,7 @@ func TestLaunchPlanManager_GetActiveLaunchPlan_NoneActive(t *testing.T) { func TestLaunchPlanManager_GetActiveLaunchPlan_InvalidRequest(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) response, err := lpManager.GetActiveLaunchPlan(context.Background(), admin.ActiveLaunchPlanRequest{ Id: &admin.NamedEntityIdentifier{ Domain: domain, @@ -229,7 +228,7 @@ func TestLaunchPlanManager_GetActiveLaunchPlan_InvalidRequest(t *testing.T) { } func TestLaunchPlan_ValidationError(t *testing.T) { - lpManager := NewLaunchPlanManager(repositoryMocks.NewMockRepository(), getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repositoryMocks.NewMockRepository(), getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() request.Id = nil response, err := lpManager.CreateLaunchPlan(context.Background(), request) @@ -239,7 +238,7 @@ func TestLaunchPlan_ValidationError(t *testing.T) { func TestLaunchPlanManager_CreateLaunchPlanErrorDueToBadLabels(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() request.Spec.Labels = &admin.Labels{ Values: map[string]string{ @@ -264,7 +263,7 @@ func TestLaunchPlan_DatabaseError(t *testing.T) { } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetCreateCallback(lpCreateFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() response, err := lpManager.CreateLaunchPlan(context.Background(), request) assert.EqualError(t, err, expectedErr.Error()) @@ -274,7 +273,7 @@ func TestLaunchPlan_DatabaseError(t *testing.T) { func TestCreateLaunchPlanInCompatibleInputs(t *testing.T) { repository := getMockRepositoryForLpTest() setDefaultWorkflowCallbackForLpTest(repository) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() request.Spec.DefaultInputs = &core.ParameterMap{ Parameters: map[string]*core.Parameter{ @@ -324,7 +323,7 @@ func TestCreateLaunchPlanValidateCreate(t *testing.T) { } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetCreateCallback(lpCreateFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() response, err := lpManager.CreateLaunchPlan(context.Background(), request) assert.Nil(t, err) @@ -365,7 +364,7 @@ func TestCreateLaunchPlanNoWorkflowInterface(t *testing.T) { } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetCreateCallback(lpCreateFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) request := testutils.GetLaunchPlanRequest() request.Spec.FixedInputs = nil request.Spec.DefaultInputs = nil @@ -413,7 +412,7 @@ func TestEnableSchedule(t *testing.T) { *input.Payload) return nil }) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).enableSchedule( context.Background(), launchPlanNamedIdentifier, @@ -434,7 +433,7 @@ func TestEnableSchedule_Error(t *testing.T) { func(ctx context.Context, input scheduleInterfaces.AddScheduleInput) error { return expectedErr }) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).enableSchedule( context.Background(), launchPlanNamedIdentifier, admin.LaunchPlanSpec{ @@ -453,7 +452,7 @@ func TestDisableSchedule(t *testing.T) { assert.True(t, proto.Equal(&launchPlanNamedIdentifier, &input.Identifier)) return nil }) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).disableSchedule(context.Background(), launchPlanNamedIdentifier) assert.Nil(t, err) } @@ -467,7 +466,7 @@ func TestDisableSchedule_Error(t *testing.T) { func(ctx context.Context, input scheduleInterfaces.RemoveScheduleInput) error { return expectedErr }) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).disableSchedule(context.Background(), launchPlanNamedIdentifier) assert.EqualError(t, err, expectedErr.Error()) } @@ -515,7 +514,7 @@ func TestUpdateSchedules(t *testing.T) { return nil }) repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).updateSchedules( context.Background(), models.LaunchPlan{ @@ -568,7 +567,7 @@ func TestUpdateSchedules_NothingToDisableButRedo(t *testing.T) { return nil }) repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).updateSchedules(context.Background(), models.LaunchPlan{ LaunchPlanKey: models.LaunchPlanKey{ Project: project, @@ -637,7 +636,7 @@ func TestUpdateSchedules_NothingToEnableButRedo(t *testing.T) { }) repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).updateSchedules(context.Background(), models.LaunchPlan{ LaunchPlanKey: models.LaunchPlanKey{ Project: project, @@ -686,7 +685,7 @@ func TestUpdateSchedules_NothingToDoButRedo(t *testing.T) { }) repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).updateSchedules(context.Background(), models.LaunchPlan{ LaunchPlanKey: models.LaunchPlanKey{ Project: project, @@ -747,7 +746,7 @@ func TestUpdateSchedules_EnableNoSchedule(t *testing.T) { }) repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) err := lpManager.(*LaunchPlanManager).updateSchedules(context.Background(), models.LaunchPlan{ LaunchPlanKey: models.LaunchPlanKey{ Project: project, @@ -828,7 +827,7 @@ func TestDisableLaunchPlan(t *testing.T) { repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetUpdateCallback(disableFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err := lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_INACTIVE, @@ -849,7 +848,7 @@ func TestDisableLaunchPlan_DatabaseError(t *testing.T) { return models.LaunchPlan{}, expectedError } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetGetCallback(lpGetFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err := lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_INACTIVE, @@ -882,7 +881,7 @@ func TestDisableLaunchPlan_DatabaseError(t *testing.T) { return expectedError } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetUpdateCallback(disableFunc) - lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err = lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_INACTIVE, @@ -939,7 +938,7 @@ func TestEnableLaunchPlan(t *testing.T) { } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetSetActiveCallback(enableFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err := lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_ACTIVE, @@ -968,7 +967,7 @@ func TestEnableLaunchPlan_NoCurrentlyActiveVersion(t *testing.T) { } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetSetActiveCallback(enableFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err := lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_ACTIVE, @@ -988,7 +987,7 @@ func TestEnableLaunchPlan_DatabaseError(t *testing.T) { return models.LaunchPlan{}, expectedError } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetGetCallback(lpGetFunc) - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err := lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_ACTIVE, @@ -996,7 +995,7 @@ func TestEnableLaunchPlan_DatabaseError(t *testing.T) { assert.EqualError(t, err, expectedError.Error(), "Failures on getting the existing launch plan should propagate") lpGetFunc = makeLaunchPlanRepoGetCallback(t) - lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) listFunc := func(input interfaces.ListResourceInput) (interfaces.LaunchPlanCollectionOutput, error) { return interfaces.LaunchPlanCollectionOutput{}, expectedError } @@ -1044,7 +1043,7 @@ func TestEnableLaunchPlan_DatabaseError(t *testing.T) { return expectedError } repository.LaunchPlanRepo().(*repositoryMocks.MockLaunchPlanRepo).SetSetActiveCallback(enableFunc) - lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager = NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) _, err = lpManager.UpdateLaunchPlan(context.Background(), admin.LaunchPlanUpdateRequest{ Id: &launchPlanIdentifier, State: admin.LaunchPlanState_ACTIVE, @@ -1054,7 +1053,7 @@ func TestEnableLaunchPlan_DatabaseError(t *testing.T) { func TestLaunchPlanManager_ListLaunchPlans(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) state := int32(0) lpRequest := testutils.GetLaunchPlanRequest() workflowRequest := testutils.GetWorkflowRequest() @@ -1161,7 +1160,7 @@ func TestLaunchPlanManager_ListLaunchPlans(t *testing.T) { func TestLaunchPlanManager_ListLaunchPlanIds(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) state := int32(0) lpRequest := testutils.GetLaunchPlanRequest() workflowRequest := testutils.GetWorkflowRequest() @@ -1244,7 +1243,7 @@ func TestLaunchPlanManager_ListLaunchPlanIds(t *testing.T) { func TestLaunchPlanManager_ListActiveLaunchPlans(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) state := int32(admin.LaunchPlanState_ACTIVE) lpRequest := testutils.GetLaunchPlanRequest() workflowRequest := testutils.GetWorkflowRequest() @@ -1331,7 +1330,7 @@ func TestLaunchPlanManager_ListActiveLaunchPlans(t *testing.T) { func TestLaunchPlanManager_ListActiveLaunchPlans_BadRequest(t *testing.T) { repository := getMockRepositoryForLpTest() - lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + lpManager := NewLaunchPlanManager(repository, getMockConfigForLpTest(), mockScheduler, mockScope.NewTestScope()) lpList, err := lpManager.ListActiveLaunchPlans(context.Background(), admin.ActiveLaunchPlanListRequest{ Domain: domain, Limit: 10, diff --git a/flyteadmin/pkg/manager/impl/task_manager.go b/flyteadmin/pkg/manager/impl/task_manager.go index 2d6bc2a91e..d42639c31e 100644 --- a/flyteadmin/pkg/manager/impl/task_manager.go +++ b/flyteadmin/pkg/manager/impl/task_manager.go @@ -6,12 +6,10 @@ import ( "strconv" "time" - "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes" "github.com/prometheus/client_golang/prometheus" "google.golang.org/grpc/codes" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" "github.com/flyteorg/flyte/flyteadmin/pkg/common" "github.com/flyteorg/flyte/flyteadmin/pkg/errors" "github.com/flyteorg/flyte/flyteadmin/pkg/manager/impl/resources" @@ -38,12 +36,11 @@ type taskMetrics struct { } type TaskManager struct { - db repoInterfaces.Repository - config runtimeInterfaces.Configuration - compiler workflowengine.Compiler - metrics taskMetrics - resourceManager interfaces.ResourceInterface - artifactRegistry *artifacts.ArtifactRegistry + db repoInterfaces.Repository + config runtimeInterfaces.Configuration + compiler workflowengine.Compiler + metrics taskMetrics + resourceManager interfaces.ResourceInterface } func getTaskContext(ctx context.Context, identifier *core.Identifier) context.Context { @@ -133,18 +130,6 @@ func (t *TaskManager) CreateTask( contextWithRuntimeMeta, common.RuntimeVersionKey, finalizedRequest.Spec.Template.Metadata.Runtime.Version) t.metrics.Registered.Inc(contextWithRuntimeMeta) } - // TODO: Artifact feature gate, remove when ready - if t.artifactRegistry.GetClient() != nil { - tIfaceCopy := proto.Clone(finalizedRequest.Spec.Template.Interface).(*core.TypedInterface) - go func() { - ceCtx := context.TODO() - if finalizedRequest.Spec.Template.Interface == nil { - logger.Debugf(ceCtx, "Task [%+v] has no interface, skipping registration", finalizedRequest.Id) - return - } - t.artifactRegistry.RegisterArtifactProducer(ceCtx, finalizedRequest.Id, *tIfaceCopy) - }() - } return &admin.TaskCreateResponse{}, nil } @@ -276,8 +261,7 @@ func (t *TaskManager) ListUniqueTaskIdentifiers(ctx context.Context, request adm func NewTaskManager( db repoInterfaces.Repository, config runtimeInterfaces.Configuration, compiler workflowengine.Compiler, - scope promutils.Scope, - artifactRegistry *artifacts.ArtifactRegistry) interfaces.TaskInterface { + scope promutils.Scope) interfaces.TaskInterface { metrics := taskMetrics{ Scope: scope, @@ -286,11 +270,10 @@ func NewTaskManager( } resourceManager := resources.NewResourceManager(db, config.ApplicationConfiguration()) return &TaskManager{ - db: db, - config: config, - compiler: compiler, - metrics: metrics, - resourceManager: resourceManager, - artifactRegistry: artifactRegistry, + db: db, + config: config, + compiler: compiler, + metrics: metrics, + resourceManager: resourceManager, } } diff --git a/flyteadmin/pkg/manager/impl/task_manager_test.go b/flyteadmin/pkg/manager/impl/task_manager_test.go index bc19311bb3..6037e431b7 100644 --- a/flyteadmin/pkg/manager/impl/task_manager_test.go +++ b/flyteadmin/pkg/manager/impl/task_manager_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc/codes" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" "github.com/flyteorg/flyte/flyteadmin/pkg/common" adminErrors "github.com/flyteorg/flyte/flyteadmin/pkg/errors" "github.com/flyteorg/flyte/flyteadmin/pkg/manager/impl/testutils" @@ -86,7 +85,7 @@ func TestCreateTask(t *testing.T) { return models.DescriptionEntity{}, adminErrors.NewFlyteAdminErrorf(codes.NotFound, "NotFound") }) taskManager := NewTaskManager(mockRepository, getMockConfigForTaskTest(), getMockTaskCompiler(), - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetValidTaskRequest() response, err := taskManager.CreateTask(context.Background(), request) assert.NoError(t, err) @@ -103,7 +102,7 @@ func TestCreateTask(t *testing.T) { func TestCreateTask_ValidationError(t *testing.T) { mockRepository := getMockTaskRepository() taskManager := NewTaskManager(mockRepository, getMockConfigForTaskTest(), getMockTaskCompiler(), - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetValidTaskRequest() request.Id = nil response, err := taskManager.CreateTask(context.Background(), request) @@ -120,7 +119,7 @@ func TestCreateTask_CompilerError(t *testing.T) { }) mockRepository := getMockTaskRepository() taskManager := NewTaskManager(mockRepository, getMockConfigForTaskTest(), mockCompiler, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetValidTaskRequest() response, err := taskManager.CreateTask(context.Background(), request) assert.EqualError(t, err, expectedErr.Error()) @@ -139,7 +138,7 @@ func TestCreateTask_DatabaseError(t *testing.T) { } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetCreateCallback(taskCreateFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) request := testutils.GetValidTaskRequest() response, err := taskManager.CreateTask(context.Background(), request) assert.EqualError(t, err, expectedErr.Error()) @@ -167,7 +166,7 @@ func TestGetTask(t *testing.T) { }, nil } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetGetCallback(taskGetFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) task, err := taskManager.GetTask(context.Background(), admin.ObjectGetRequest{ Id: &taskIdentifier, @@ -187,7 +186,7 @@ func TestGetTask_DatabaseError(t *testing.T) { return models.Task{}, expectedErr } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetGetCallback(taskGetFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) task, err := taskManager.GetTask(context.Background(), admin.ObjectGetRequest{ Id: &taskIdentifier, }) @@ -213,7 +212,7 @@ func TestGetTask_TransformerError(t *testing.T) { }, nil } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetGetCallback(taskGetFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) task, err := taskManager.GetTask(context.Background(), admin.ObjectGetRequest{ Id: &taskIdentifier, @@ -272,7 +271,7 @@ func TestListTasks(t *testing.T) { }, nil } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetListCallback(taskListFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) taskList, err := taskManager.ListTasks(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ @@ -304,7 +303,7 @@ func TestListTasks(t *testing.T) { func TestListTasks_MissingParameters(t *testing.T) { repository := getMockTaskRepository() - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) _, err := taskManager.ListTasks(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Domain: domainValue, @@ -334,7 +333,7 @@ func TestListTasks_DatabaseError(t *testing.T) { } repository.TaskRepo().(*repositoryMocks.MockTaskRepo).SetListCallback(taskListFunc) - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) _, err := taskManager.ListTasks(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Project: projectValue, @@ -348,7 +347,7 @@ func TestListTasks_DatabaseError(t *testing.T) { func TestListUniqueTaskIdentifiers(t *testing.T) { repository := getMockTaskRepository() - taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + taskManager := NewTaskManager(repository, getMockConfigForTaskTest(), getMockTaskCompiler(), mockScope.NewTestScope()) listFunc := func(input interfaces.ListResourceInput) (interfaces.TaskCollectionOutput, error) { // Test that parameters are being passed in diff --git a/flyteadmin/pkg/manager/impl/workflow_manager.go b/flyteadmin/pkg/manager/impl/workflow_manager.go index 726e3988c3..4b755a4707 100644 --- a/flyteadmin/pkg/manager/impl/workflow_manager.go +++ b/flyteadmin/pkg/manager/impl/workflow_manager.go @@ -6,12 +6,10 @@ import ( "strconv" "time" - "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes" "github.com/prometheus/client_golang/prometheus" "google.golang.org/grpc/codes" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" "github.com/flyteorg/flyte/flyteadmin/pkg/common" "github.com/flyteorg/flyte/flyteadmin/pkg/errors" "github.com/flyteorg/flyte/flyteadmin/pkg/manager/impl/util" @@ -41,13 +39,12 @@ type workflowMetrics struct { } type WorkflowManager struct { - db repoInterfaces.Repository - config runtimeInterfaces.Configuration - compiler workflowengineInterfaces.Compiler - storageClient *storage.DataStore - storagePrefix []string - metrics workflowMetrics - artifactRegistry *artifacts.ArtifactRegistry + db repoInterfaces.Repository + config runtimeInterfaces.Configuration + compiler workflowengineInterfaces.Compiler + storageClient *storage.DataStore + storagePrefix []string + metrics workflowMetrics } func getWorkflowContext(ctx context.Context, identifier *core.Identifier) context.Context { @@ -220,22 +217,6 @@ func (w *WorkflowManager) CreateWorkflow( } w.metrics.TypedInterfaceSizeBytes.Observe(float64(len(workflowModel.TypedInterface))) - // Send the interface definition to Artifact service, this is so that it can statically pick up one dimension of - // lineage information - tIfaceCopy := proto.Clone(workflowClosure.CompiledWorkflow.Primary.Template.Interface).(*core.TypedInterface) - // TODO: Artifact feature gate, remove when ready - if w.artifactRegistry.GetClient() != nil { - go func() { - ceCtx := context.TODO() - if workflowClosure.CompiledWorkflow == nil || workflowClosure.CompiledWorkflow.Primary == nil { - logger.Debugf(ceCtx, "Insufficient fields to submit workflow interface %v", finalizedRequest.Id) - return - } - - w.artifactRegistry.RegisterArtifactProducer(ceCtx, finalizedRequest.Id, *tIfaceCopy) - }() - } - return &admin.WorkflowCreateResponse{}, nil } @@ -367,8 +348,7 @@ func NewWorkflowManager( compiler workflowengineInterfaces.Compiler, storageClient *storage.DataStore, storagePrefix []string, - scope promutils.Scope, - artifactRegistry *artifacts.ArtifactRegistry) interfaces.WorkflowInterface { + scope promutils.Scope) interfaces.WorkflowInterface { metrics := workflowMetrics{ Scope: scope, @@ -378,12 +358,11 @@ func NewWorkflowManager( "size in bytes of serialized workflow TypedInterface"), } return &WorkflowManager{ - db: db, - config: config, - compiler: compiler, - storageClient: storageClient, - storagePrefix: storagePrefix, - metrics: metrics, - artifactRegistry: artifactRegistry, + db: db, + config: config, + compiler: compiler, + storageClient: storageClient, + storagePrefix: storagePrefix, + metrics: metrics, } } diff --git a/flyteadmin/pkg/manager/impl/workflow_manager_test.go b/flyteadmin/pkg/manager/impl/workflow_manager_test.go index 9b9fb611d7..99da70a64b 100644 --- a/flyteadmin/pkg/manager/impl/workflow_manager_test.go +++ b/flyteadmin/pkg/manager/impl/workflow_manager_test.go @@ -11,7 +11,6 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" "github.com/flyteorg/flyte/flyteadmin/pkg/common" commonMocks "github.com/flyteorg/flyte/flyteadmin/pkg/common/mocks" adminErrors "github.com/flyteorg/flyte/flyteadmin/pkg/errors" @@ -124,7 +123,7 @@ func TestSetWorkflowDefaults(t *testing.T) { workflowManager := NewWorkflowManager( getMockRepository(returnWorkflowOnGet), getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), commonMocks.GetMockStorageClient(), storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() finalizedRequest, err := workflowManager.(*WorkflowManager).setDefaults(request) assert.NoError(t, err) @@ -144,7 +143,7 @@ func TestCreateWorkflow(t *testing.T) { workflowManager := NewWorkflowManager( repository, - getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), getMockStorage(), storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), getMockStorage(), storagePrefix, mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) assert.NoError(t, err) @@ -165,7 +164,7 @@ func TestCreateWorkflow_ValidationError(t *testing.T) { workflowManager := NewWorkflowManager( repositoryMocks.NewMockRepository(), getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), commonMocks.GetMockStorageClient(), storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() request.Id = nil response, err := workflowManager.CreateWorkflow(context.Background(), request) @@ -184,7 +183,7 @@ func TestCreateWorkflow_ExistingWorkflow(t *testing.T) { } workflowManager := NewWorkflowManager( getMockRepository(returnWorkflowOnGet), - getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) assert.EqualError(t, err, "workflow with different structure already exists") @@ -201,7 +200,7 @@ func TestCreateWorkflow_ExistingWorkflow_Different(t *testing.T) { } workflowManager := NewWorkflowManager( getMockRepository(returnWorkflowOnGet), - getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) @@ -222,7 +221,7 @@ func TestCreateWorkflow_CompilerGetRequirementsError(t *testing.T) { workflowManager := NewWorkflowManager( getMockRepository(!returnWorkflowOnGet), - getMockWorkflowConfigProvider(), mockCompiler, getMockStorage(), storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + getMockWorkflowConfigProvider(), mockCompiler, getMockStorage(), storagePrefix, mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) assert.EqualError(t, err, fmt.Sprintf( @@ -242,7 +241,7 @@ func TestCreateWorkflow_CompileWorkflowError(t *testing.T) { workflowManager := NewWorkflowManager( getMockRepository(!returnWorkflowOnGet), - getMockWorkflowConfigProvider(), mockCompiler, getMockStorage(), storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + getMockWorkflowConfigProvider(), mockCompiler, getMockStorage(), storagePrefix, mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) assert.Nil(t, response) @@ -264,7 +263,7 @@ func TestCreateWorkflow_DatabaseError(t *testing.T) { repository.WorkflowRepo().(*repositoryMocks.MockWorkflowRepo).SetCreateCallback(workflowCreateFunc) workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), getMockStorage(), storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) request := testutils.GetWorkflowRequest() response, err := workflowManager.CreateWorkflow(context.Background(), request) assert.EqualError(t, err, expectedErr.Error()) @@ -304,7 +303,7 @@ func TestGetWorkflow(t *testing.T) { } workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) workflow, err := workflowManager.GetWorkflow(context.Background(), admin.ObjectGetRequest{ Id: &workflowIdentifier, }) @@ -326,7 +325,7 @@ func TestGetWorkflow_DatabaseError(t *testing.T) { repository.WorkflowRepo().(*repositoryMocks.MockWorkflowRepo).SetGetCallback(workflowGetFunc) workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), commonMocks.GetMockStorageClient(), - storagePrefix, mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + storagePrefix, mockScope.NewTestScope()) workflow, err := workflowManager.GetWorkflow(context.Background(), admin.ObjectGetRequest{ Id: &workflowIdentifier, }) @@ -362,7 +361,7 @@ func TestGetWorkflow_TransformerError(t *testing.T) { workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) workflow, err := workflowManager.GetWorkflow(context.Background(), admin.ObjectGetRequest{ Id: &workflowIdentifier, }) @@ -433,7 +432,7 @@ func TestListWorkflows(t *testing.T) { } workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) workflowList, err := workflowManager.ListWorkflows(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ @@ -475,7 +474,7 @@ func TestListWorkflows_MissingParameters(t *testing.T) { workflowManager := NewWorkflowManager( repositoryMocks.NewMockRepository(), getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), commonMocks.GetMockStorageClient(), storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) _, err := workflowManager.ListWorkflows(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Domain: domainValue, @@ -507,7 +506,7 @@ func TestListWorkflows_DatabaseError(t *testing.T) { repository.WorkflowRepo().(*repositoryMocks.MockWorkflowRepo).SetListCallback(workflowListFunc) workflowManager := NewWorkflowManager(repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), commonMocks.GetMockStorageClient(), storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) _, err := workflowManager.ListWorkflows(context.Background(), admin.ResourceListRequest{ Id: &admin.NamedEntityIdentifier{ Project: projectValue, @@ -570,7 +569,7 @@ func TestWorkflowManager_ListWorkflowIdentifiers(t *testing.T) { } workflowManager := NewWorkflowManager( repository, getMockWorkflowConfigProvider(), getMockWorkflowCompiler(), mockStorageClient, storagePrefix, - mockScope.NewTestScope(), artifacts.NewArtifactRegistry(context.Background(), nil)) + mockScope.NewTestScope()) workflowList, err := workflowManager.ListWorkflowIdentifiers(context.Background(), admin.NamedEntityIdentifierListRequest{ diff --git a/flyteadmin/pkg/rpc/adminservice/base.go b/flyteadmin/pkg/rpc/adminservice/base.go index 1e91fec5d4..5a2cb2ad89 100644 --- a/flyteadmin/pkg/rpc/adminservice/base.go +++ b/flyteadmin/pkg/rpc/adminservice/base.go @@ -7,7 +7,6 @@ import ( "github.com/golang/protobuf/proto" - "github.com/flyteorg/flyte/flyteadmin/pkg/artifacts" "github.com/flyteorg/flyte/flyteadmin/pkg/async/cloudevent" eventWriter "github.com/flyteorg/flyte/flyteadmin/pkg/async/events/implementations" "github.com/flyteorg/flyte/flyteadmin/pkg/async/notifications" @@ -22,7 +21,6 @@ import ( runtimeIfaces "github.com/flyteorg/flyte/flyteadmin/pkg/runtime/interfaces" workflowengineImpl "github.com/flyteorg/flyte/flyteadmin/pkg/workflowengine/impl" "github.com/flyteorg/flyte/flyteadmin/plugins" - admin2 "github.com/flyteorg/flyte/flyteidl/clients/go/admin" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service" "github.com/flyteorg/flyte/flytestdlib/logger" "github.com/flyteorg/flyte/flytestdlib/promutils" @@ -44,7 +42,6 @@ type AdminService struct { DescriptionEntityManager interfaces.DescriptionEntityInterface MetricsManager interfaces.MetricsInterface Metrics AdminMetrics - Artifacts *artifacts.ArtifactRegistry } // Intercepts all admin requests to handle panics during execution. @@ -114,14 +111,8 @@ func NewAdminServer(ctx context.Context, pluginRegistry *plugins.Registry, confi eventScheduler := workflowScheduler.GetEventScheduler() - var artifactRegistry *artifacts.ArtifactRegistry - if configuration.ApplicationConfiguration().GetTopLevelConfig().FeatureGates.EnableArtifacts { - adminClientCfg := admin2.GetConfig(ctx) - artifactRegistry = artifacts.NewArtifactRegistry(ctx, adminClientCfg) - } - launchPlanManager := manager.NewLaunchPlanManager( - repo, configuration, eventScheduler, adminScope.NewSubScope("launch_plan_manager"), artifactRegistry) + repo, configuration, eventScheduler, adminScope.NewSubScope("launch_plan_manager")) // Configure admin-specific remote data handler (separate from storage) remoteDataConfig := configuration.ApplicationConfiguration().GetRemoteDataConfig() @@ -138,7 +129,7 @@ func NewAdminServer(ctx context.Context, pluginRegistry *plugins.Registry, confi workflowManager := manager.NewWorkflowManager( repo, configuration, workflowengineImpl.NewCompiler(), dataStorageClient, applicationConfiguration.GetMetadataStoragePrefix(), - adminScope.NewSubScope("workflow_manager"), artifactRegistry) + adminScope.NewSubScope("workflow_manager")) namedEntityManager := manager.NewNamedEntityManager(repo, configuration, adminScope.NewSubScope("named_entity_manager")) descriptionEntityManager := manager.NewDescriptionEntityManager(repo, configuration, adminScope.NewSubScope("description_entity_manager")) @@ -149,7 +140,7 @@ func NewAdminServer(ctx context.Context, pluginRegistry *plugins.Registry, confi executionManager := manager.NewExecutionManager(repo, pluginRegistry, configuration, dataStorageClient, adminScope.NewSubScope("execution_manager"), adminScope.NewSubScope("user_execution_metrics"), - publisher, urlData, workflowManager, namedEntityManager, eventPublisher, cloudEventPublisher, executionEventWriter, artifactRegistry) + publisher, urlData, workflowManager, namedEntityManager, eventPublisher, cloudEventPublisher, executionEventWriter) versionManager := manager.NewVersionManager() scheduledWorkflowExecutor := workflowScheduler.GetWorkflowExecutor(executionManager, launchPlanManager) @@ -172,7 +163,7 @@ func NewAdminServer(ctx context.Context, pluginRegistry *plugins.Registry, confi logger.Info(ctx, "Initializing a new AdminService") return &AdminService{ TaskManager: manager.NewTaskManager(repo, configuration, workflowengineImpl.NewCompiler(), - adminScope.NewSubScope("task_manager"), artifactRegistry), + adminScope.NewSubScope("task_manager")), WorkflowManager: workflowManager, LaunchPlanManager: launchPlanManager, ExecutionManager: executionManager, @@ -185,7 +176,6 @@ func NewAdminServer(ctx context.Context, pluginRegistry *plugins.Registry, confi ResourceManager: resources.NewResourceManager(repo, configuration.ApplicationConfiguration()), MetricsManager: manager.NewMetricsManager(workflowManager, executionManager, nodeExecutionManager, taskExecutionManager, adminScope.NewSubScope("metrics_manager")), - Metrics: InitMetrics(adminScope), - Artifacts: artifactRegistry, + Metrics: InitMetrics(adminScope), } } diff --git a/flyteidl/clients/go/admin/client.go b/flyteidl/clients/go/admin/client.go index 521b6e1e4d..9758bd9dec 100644 --- a/flyteidl/clients/go/admin/client.go +++ b/flyteidl/clients/go/admin/client.go @@ -7,8 +7,6 @@ import ( "errors" "fmt" - "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact" - grpcRetry "github.com/grpc-ecosystem/go-grpc-middleware/retry" grpcPrometheus "github.com/grpc-ecosystem/go-grpc-prometheus" "google.golang.org/grpc" @@ -32,7 +30,6 @@ type Clientset struct { identityServiceClient service.IdentityServiceClient dataProxyServiceClient service.DataProxyServiceClient signalServiceClient service.SignalServiceClient - artifactServiceClient artifact.ArtifactRegistryClient } // AdminClient retrieves the AdminServiceClient @@ -62,10 +59,6 @@ func (c Clientset) SignalServiceClient() service.SignalServiceClient { return c.signalServiceClient } -func (c Clientset) ArtifactServiceClient() artifact.ArtifactRegistryClient { - return c.artifactServiceClient -} - func NewAdminClient(ctx context.Context, conn *grpc.ClientConn) service.AdminServiceClient { logger.Infof(ctx, "Initialized Admin client") return service.NewAdminServiceClient(conn) @@ -206,7 +199,6 @@ func initializeClients(ctx context.Context, cfg *Config, tokenCache cache.TokenC cs.healthServiceClient = grpc_health_v1.NewHealthClient(adminConnection) cs.dataProxyServiceClient = service.NewDataProxyServiceClient(adminConnection) cs.signalServiceClient = service.NewSignalServiceClient(adminConnection) - cs.artifactServiceClient = artifact.NewArtifactRegistryClient(adminConnection) return &cs, nil } diff --git a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.cc deleted file mode 100644 index 5315df5372..0000000000 --- a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.cc +++ /dev/null @@ -1,505 +0,0 @@ -// Generated by the gRPC C++ plugin. -// If you make any local change, they will be lost. -// source: flyteidl/artifact/artifacts.proto - -#include "flyteidl/artifact/artifacts.pb.h" -#include "flyteidl/artifact/artifacts.grpc.pb.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -namespace flyteidl { -namespace artifact { - -static const char* ArtifactRegistry_method_names[] = { - "/flyteidl.artifact.ArtifactRegistry/CreateArtifact", - "/flyteidl.artifact.ArtifactRegistry/GetArtifact", - "/flyteidl.artifact.ArtifactRegistry/SearchArtifacts", - "/flyteidl.artifact.ArtifactRegistry/CreateTrigger", - "/flyteidl.artifact.ArtifactRegistry/DeactivateTrigger", - "/flyteidl.artifact.ArtifactRegistry/AddTag", - "/flyteidl.artifact.ArtifactRegistry/RegisterProducer", - "/flyteidl.artifact.ArtifactRegistry/RegisterConsumer", - "/flyteidl.artifact.ArtifactRegistry/SetExecutionInputs", - "/flyteidl.artifact.ArtifactRegistry/FindByWorkflowExec", - "/flyteidl.artifact.ArtifactRegistry/ListUsage", -}; - -std::unique_ptr< ArtifactRegistry::Stub> ArtifactRegistry::NewStub(const std::shared_ptr< ::grpc::ChannelInterface>& channel, const ::grpc::StubOptions& options) { - (void)options; - std::unique_ptr< ArtifactRegistry::Stub> stub(new ArtifactRegistry::Stub(channel)); - return stub; -} - -ArtifactRegistry::Stub::Stub(const std::shared_ptr< ::grpc::ChannelInterface>& channel) - : channel_(channel), rpcmethod_CreateArtifact_(ArtifactRegistry_method_names[0], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_GetArtifact_(ArtifactRegistry_method_names[1], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_SearchArtifacts_(ArtifactRegistry_method_names[2], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_CreateTrigger_(ArtifactRegistry_method_names[3], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_DeactivateTrigger_(ArtifactRegistry_method_names[4], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_AddTag_(ArtifactRegistry_method_names[5], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_RegisterProducer_(ArtifactRegistry_method_names[6], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_RegisterConsumer_(ArtifactRegistry_method_names[7], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_SetExecutionInputs_(ArtifactRegistry_method_names[8], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_FindByWorkflowExec_(ArtifactRegistry_method_names[9], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - , rpcmethod_ListUsage_(ArtifactRegistry_method_names[10], ::grpc::internal::RpcMethod::NORMAL_RPC, channel) - {} - -::grpc::Status ArtifactRegistry::Stub::CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::flyteidl::artifact::CreateArtifactResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_CreateArtifact_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_CreateArtifact_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_CreateArtifact_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_CreateArtifact_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_CreateArtifact_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>* ArtifactRegistry::Stub::AsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::CreateArtifactResponse>::Create(channel_.get(), cq, rpcmethod_CreateArtifact_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>* ArtifactRegistry::Stub::PrepareAsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::CreateArtifactResponse>::Create(channel_.get(), cq, rpcmethod_CreateArtifact_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::flyteidl::artifact::GetArtifactResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_GetArtifact_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_GetArtifact_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_GetArtifact_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_GetArtifact_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_GetArtifact_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>* ArtifactRegistry::Stub::AsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::GetArtifactResponse>::Create(channel_.get(), cq, rpcmethod_GetArtifact_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>* ArtifactRegistry::Stub::PrepareAsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::GetArtifactResponse>::Create(channel_.get(), cq, rpcmethod_GetArtifact_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_SearchArtifacts_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_SearchArtifacts_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_SearchArtifacts_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_SearchArtifacts_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_SearchArtifacts_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* ArtifactRegistry::Stub::AsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::SearchArtifactsResponse>::Create(channel_.get(), cq, rpcmethod_SearchArtifacts_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* ArtifactRegistry::Stub::PrepareAsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::SearchArtifactsResponse>::Create(channel_.get(), cq, rpcmethod_SearchArtifacts_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::flyteidl::artifact::CreateTriggerResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_CreateTrigger_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_CreateTrigger_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_CreateTrigger_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_CreateTrigger_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_CreateTrigger_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>* ArtifactRegistry::Stub::AsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::CreateTriggerResponse>::Create(channel_.get(), cq, rpcmethod_CreateTrigger_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>* ArtifactRegistry::Stub::PrepareAsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::CreateTriggerResponse>::Create(channel_.get(), cq, rpcmethod_CreateTrigger_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::flyteidl::artifact::DeactivateTriggerResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_DeactivateTrigger_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_DeactivateTrigger_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_DeactivateTrigger_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_DeactivateTrigger_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_DeactivateTrigger_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>* ArtifactRegistry::Stub::AsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::DeactivateTriggerResponse>::Create(channel_.get(), cq, rpcmethod_DeactivateTrigger_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>* ArtifactRegistry::Stub::PrepareAsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::DeactivateTriggerResponse>::Create(channel_.get(), cq, rpcmethod_DeactivateTrigger_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::flyteidl::artifact::AddTagResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_AddTag_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_AddTag_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_AddTag_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_AddTag_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_AddTag_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>* ArtifactRegistry::Stub::AsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::AddTagResponse>::Create(channel_.get(), cq, rpcmethod_AddTag_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>* ArtifactRegistry::Stub::PrepareAsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::AddTagResponse>::Create(channel_.get(), cq, rpcmethod_AddTag_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::flyteidl::artifact::RegisterResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_RegisterProducer_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_RegisterProducer_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_RegisterProducer_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_RegisterProducer_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_RegisterProducer_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* ArtifactRegistry::Stub::AsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::RegisterResponse>::Create(channel_.get(), cq, rpcmethod_RegisterProducer_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* ArtifactRegistry::Stub::PrepareAsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::RegisterResponse>::Create(channel_.get(), cq, rpcmethod_RegisterProducer_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::flyteidl::artifact::RegisterResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_RegisterConsumer_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_RegisterConsumer_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_RegisterConsumer_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_RegisterConsumer_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_RegisterConsumer_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* ArtifactRegistry::Stub::AsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::RegisterResponse>::Create(channel_.get(), cq, rpcmethod_RegisterConsumer_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* ArtifactRegistry::Stub::PrepareAsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::RegisterResponse>::Create(channel_.get(), cq, rpcmethod_RegisterConsumer_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::flyteidl::artifact::ExecutionInputsResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_SetExecutionInputs_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_SetExecutionInputs_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_SetExecutionInputs_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_SetExecutionInputs_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_SetExecutionInputs_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>* ArtifactRegistry::Stub::AsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::ExecutionInputsResponse>::Create(channel_.get(), cq, rpcmethod_SetExecutionInputs_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>* ArtifactRegistry::Stub::PrepareAsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::ExecutionInputsResponse>::Create(channel_.get(), cq, rpcmethod_SetExecutionInputs_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_FindByWorkflowExec_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_FindByWorkflowExec_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_FindByWorkflowExec_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_FindByWorkflowExec_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_FindByWorkflowExec_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* ArtifactRegistry::Stub::AsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::SearchArtifactsResponse>::Create(channel_.get(), cq, rpcmethod_FindByWorkflowExec_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* ArtifactRegistry::Stub::PrepareAsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::SearchArtifactsResponse>::Create(channel_.get(), cq, rpcmethod_FindByWorkflowExec_, context, request, false); -} - -::grpc::Status ArtifactRegistry::Stub::ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::flyteidl::artifact::ListUsageResponse* response) { - return ::grpc::internal::BlockingUnaryCall(channel_.get(), rpcmethod_ListUsage_, context, request, response); -} - -void ArtifactRegistry::Stub::experimental_async::ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_ListUsage_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, std::function f) { - ::grpc::internal::CallbackUnaryCall(stub_->channel_.get(), stub_->rpcmethod_ListUsage_, context, request, response, std::move(f)); -} - -void ArtifactRegistry::Stub::experimental_async::ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_ListUsage_, context, request, response, reactor); -} - -void ArtifactRegistry::Stub::experimental_async::ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) { - ::grpc::internal::ClientCallbackUnaryFactory::Create(stub_->channel_.get(), stub_->rpcmethod_ListUsage_, context, request, response, reactor); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>* ArtifactRegistry::Stub::AsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::ListUsageResponse>::Create(channel_.get(), cq, rpcmethod_ListUsage_, context, request, true); -} - -::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>* ArtifactRegistry::Stub::PrepareAsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return ::grpc::internal::ClientAsyncResponseReaderFactory< ::flyteidl::artifact::ListUsageResponse>::Create(channel_.get(), cq, rpcmethod_ListUsage_, context, request, false); -} - -ArtifactRegistry::Service::Service() { - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[0], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::CreateArtifactRequest, ::flyteidl::artifact::CreateArtifactResponse>( - std::mem_fn(&ArtifactRegistry::Service::CreateArtifact), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[1], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::GetArtifactRequest, ::flyteidl::artifact::GetArtifactResponse>( - std::mem_fn(&ArtifactRegistry::Service::GetArtifact), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[2], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::SearchArtifactsRequest, ::flyteidl::artifact::SearchArtifactsResponse>( - std::mem_fn(&ArtifactRegistry::Service::SearchArtifacts), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[3], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::CreateTriggerRequest, ::flyteidl::artifact::CreateTriggerResponse>( - std::mem_fn(&ArtifactRegistry::Service::CreateTrigger), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[4], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::DeactivateTriggerRequest, ::flyteidl::artifact::DeactivateTriggerResponse>( - std::mem_fn(&ArtifactRegistry::Service::DeactivateTrigger), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[5], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::AddTagRequest, ::flyteidl::artifact::AddTagResponse>( - std::mem_fn(&ArtifactRegistry::Service::AddTag), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[6], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::RegisterProducerRequest, ::flyteidl::artifact::RegisterResponse>( - std::mem_fn(&ArtifactRegistry::Service::RegisterProducer), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[7], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::RegisterConsumerRequest, ::flyteidl::artifact::RegisterResponse>( - std::mem_fn(&ArtifactRegistry::Service::RegisterConsumer), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[8], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::ExecutionInputsRequest, ::flyteidl::artifact::ExecutionInputsResponse>( - std::mem_fn(&ArtifactRegistry::Service::SetExecutionInputs), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[9], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::FindByWorkflowExecRequest, ::flyteidl::artifact::SearchArtifactsResponse>( - std::mem_fn(&ArtifactRegistry::Service::FindByWorkflowExec), this))); - AddMethod(new ::grpc::internal::RpcServiceMethod( - ArtifactRegistry_method_names[10], - ::grpc::internal::RpcMethod::NORMAL_RPC, - new ::grpc::internal::RpcMethodHandler< ArtifactRegistry::Service, ::flyteidl::artifact::ListUsageRequest, ::flyteidl::artifact::ListUsageResponse>( - std::mem_fn(&ArtifactRegistry::Service::ListUsage), this))); -} - -ArtifactRegistry::Service::~Service() { -} - -::grpc::Status ArtifactRegistry::Service::CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - -::grpc::Status ArtifactRegistry::Service::ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) { - (void) context; - (void) request; - (void) response; - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); -} - - -} // namespace flyteidl -} // namespace artifact - diff --git a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.h b/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.h deleted file mode 100644 index d113f04e20..0000000000 --- a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.grpc.pb.h +++ /dev/null @@ -1,1865 +0,0 @@ -// Generated by the gRPC C++ plugin. -// If you make any local change, they will be lost. -// source: flyteidl/artifact/artifacts.proto -#ifndef GRPC_flyteidl_2fartifact_2fartifacts_2eproto__INCLUDED -#define GRPC_flyteidl_2fartifact_2fartifacts_2eproto__INCLUDED - -#include "flyteidl/artifact/artifacts.pb.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace grpc_impl { -class Channel; -class CompletionQueue; -class ServerCompletionQueue; -} // namespace grpc_impl - -namespace grpc { -namespace experimental { -template -class MessageAllocator; -} // namespace experimental -} // namespace grpc_impl - -namespace grpc { -class ServerContext; -} // namespace grpc - -namespace flyteidl { -namespace artifact { - -class ArtifactRegistry final { - public: - static constexpr char const* service_full_name() { - return "flyteidl.artifact.ArtifactRegistry"; - } - class StubInterface { - public: - virtual ~StubInterface() {} - virtual ::grpc::Status CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::flyteidl::artifact::CreateArtifactResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>> AsyncCreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>>(AsyncCreateArtifactRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>> PrepareAsyncCreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>>(PrepareAsyncCreateArtifactRaw(context, request, cq)); - } - virtual ::grpc::Status GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::flyteidl::artifact::GetArtifactResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>> AsyncGetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>>(AsyncGetArtifactRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>> PrepareAsyncGetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>>(PrepareAsyncGetArtifactRaw(context, request, cq)); - } - virtual ::grpc::Status SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>> AsyncSearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>>(AsyncSearchArtifactsRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>> PrepareAsyncSearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>>(PrepareAsyncSearchArtifactsRaw(context, request, cq)); - } - virtual ::grpc::Status CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::flyteidl::artifact::CreateTriggerResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>> AsyncCreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>>(AsyncCreateTriggerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>> PrepareAsyncCreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>>(PrepareAsyncCreateTriggerRaw(context, request, cq)); - } - virtual ::grpc::Status DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::flyteidl::artifact::DeactivateTriggerResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>> AsyncDeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>>(AsyncDeactivateTriggerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>> PrepareAsyncDeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>>(PrepareAsyncDeactivateTriggerRaw(context, request, cq)); - } - virtual ::grpc::Status AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::flyteidl::artifact::AddTagResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>> AsyncAddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>>(AsyncAddTagRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>> PrepareAsyncAddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>>(PrepareAsyncAddTagRaw(context, request, cq)); - } - virtual ::grpc::Status RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::flyteidl::artifact::RegisterResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>> AsyncRegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>>(AsyncRegisterProducerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>> PrepareAsyncRegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>>(PrepareAsyncRegisterProducerRaw(context, request, cq)); - } - virtual ::grpc::Status RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::flyteidl::artifact::RegisterResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>> AsyncRegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>>(AsyncRegisterConsumerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>> PrepareAsyncRegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>>(PrepareAsyncRegisterConsumerRaw(context, request, cq)); - } - virtual ::grpc::Status SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::flyteidl::artifact::ExecutionInputsResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>> AsyncSetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>>(AsyncSetExecutionInputsRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>> PrepareAsyncSetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>>(PrepareAsyncSetExecutionInputsRaw(context, request, cq)); - } - virtual ::grpc::Status FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>> AsyncFindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>>(AsyncFindByWorkflowExecRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>> PrepareAsyncFindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>>(PrepareAsyncFindByWorkflowExecRaw(context, request, cq)); - } - virtual ::grpc::Status ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::flyteidl::artifact::ListUsageResponse* response) = 0; - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>> AsyncListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>>(AsyncListUsageRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>> PrepareAsyncListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>>(PrepareAsyncListUsageRaw(context, request, cq)); - } - class experimental_async_interface { - public: - virtual ~experimental_async_interface() {} - virtual void CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function) = 0; - virtual void CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function) = 0; - virtual void CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function) = 0; - virtual void GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function) = 0; - virtual void GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) = 0; - virtual void SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) = 0; - virtual void SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function) = 0; - virtual void CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function) = 0; - virtual void CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function) = 0; - virtual void DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function) = 0; - virtual void DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, std::function) = 0; - virtual void AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, std::function) = 0; - virtual void AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function) = 0; - virtual void RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function) = 0; - virtual void RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function) = 0; - virtual void RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function) = 0; - virtual void RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function) = 0; - virtual void SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function) = 0; - virtual void SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) = 0; - virtual void FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) = 0; - virtual void FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, std::function) = 0; - virtual void ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, std::function) = 0; - virtual void ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - virtual void ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) = 0; - }; - virtual class experimental_async_interface* experimental_async() { return nullptr; } - private: - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>* AsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateArtifactResponse>* PrepareAsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>* AsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::GetArtifactResponse>* PrepareAsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>* AsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>* PrepareAsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>* AsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::CreateTriggerResponse>* PrepareAsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>* AsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::DeactivateTriggerResponse>* PrepareAsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>* AsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::AddTagResponse>* PrepareAsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>* AsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>* PrepareAsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>* AsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::RegisterResponse>* PrepareAsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>* AsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ExecutionInputsResponse>* PrepareAsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>* AsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::SearchArtifactsResponse>* PrepareAsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>* AsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) = 0; - virtual ::grpc::ClientAsyncResponseReaderInterface< ::flyteidl::artifact::ListUsageResponse>* PrepareAsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) = 0; - }; - class Stub final : public StubInterface { - public: - Stub(const std::shared_ptr< ::grpc::ChannelInterface>& channel); - ::grpc::Status CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::flyteidl::artifact::CreateArtifactResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>> AsyncCreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>>(AsyncCreateArtifactRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>> PrepareAsyncCreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>>(PrepareAsyncCreateArtifactRaw(context, request, cq)); - } - ::grpc::Status GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::flyteidl::artifact::GetArtifactResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>> AsyncGetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>>(AsyncGetArtifactRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>> PrepareAsyncGetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>>(PrepareAsyncGetArtifactRaw(context, request, cq)); - } - ::grpc::Status SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>> AsyncSearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>>(AsyncSearchArtifactsRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>> PrepareAsyncSearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>>(PrepareAsyncSearchArtifactsRaw(context, request, cq)); - } - ::grpc::Status CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::flyteidl::artifact::CreateTriggerResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>> AsyncCreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>>(AsyncCreateTriggerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>> PrepareAsyncCreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>>(PrepareAsyncCreateTriggerRaw(context, request, cq)); - } - ::grpc::Status DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>> AsyncDeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>>(AsyncDeactivateTriggerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>> PrepareAsyncDeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>>(PrepareAsyncDeactivateTriggerRaw(context, request, cq)); - } - ::grpc::Status AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::flyteidl::artifact::AddTagResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>> AsyncAddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>>(AsyncAddTagRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>> PrepareAsyncAddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>>(PrepareAsyncAddTagRaw(context, request, cq)); - } - ::grpc::Status RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::flyteidl::artifact::RegisterResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>> AsyncRegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>>(AsyncRegisterProducerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>> PrepareAsyncRegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>>(PrepareAsyncRegisterProducerRaw(context, request, cq)); - } - ::grpc::Status RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::flyteidl::artifact::RegisterResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>> AsyncRegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>>(AsyncRegisterConsumerRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>> PrepareAsyncRegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>>(PrepareAsyncRegisterConsumerRaw(context, request, cq)); - } - ::grpc::Status SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::flyteidl::artifact::ExecutionInputsResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>> AsyncSetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>>(AsyncSetExecutionInputsRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>> PrepareAsyncSetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>>(PrepareAsyncSetExecutionInputsRaw(context, request, cq)); - } - ::grpc::Status FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::flyteidl::artifact::SearchArtifactsResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>> AsyncFindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>>(AsyncFindByWorkflowExecRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>> PrepareAsyncFindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>>(PrepareAsyncFindByWorkflowExecRaw(context, request, cq)); - } - ::grpc::Status ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::flyteidl::artifact::ListUsageResponse* response) override; - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>> AsyncListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>>(AsyncListUsageRaw(context, request, cq)); - } - std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>> PrepareAsyncListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) { - return std::unique_ptr< ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>>(PrepareAsyncListUsageRaw(context, request, cq)); - } - class experimental_async final : - public StubInterface::experimental_async_interface { - public: - void CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function) override; - void CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, std::function) override; - void CreateArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void CreateArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function) override; - void GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, std::function) override; - void GetArtifact(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void GetArtifact(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) override; - void SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) override; - void SearchArtifacts(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void SearchArtifacts(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function) override; - void CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, std::function) override; - void CreateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void CreateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function) override; - void DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, std::function) override; - void DeactivateTrigger(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void DeactivateTrigger(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, std::function) override; - void AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, std::function) override; - void AddTag(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void AddTag(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function) override; - void RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function) override; - void RegisterProducer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void RegisterProducer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, std::function) override; - void RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, std::function) override; - void RegisterConsumer(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void RegisterConsumer(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function) override; - void SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, std::function) override; - void SetExecutionInputs(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void SetExecutionInputs(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) override; - void FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, std::function) override; - void FindByWorkflowExec(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void FindByWorkflowExec(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, std::function) override; - void ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, std::function) override; - void ListUsage(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - void ListUsage(::grpc::ClientContext* context, const ::grpc::ByteBuffer* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ClientUnaryReactor* reactor) override; - private: - friend class Stub; - explicit experimental_async(Stub* stub): stub_(stub) { } - Stub* stub() { return stub_; } - Stub* stub_; - }; - class experimental_async_interface* experimental_async() override { return &async_stub_; } - - private: - std::shared_ptr< ::grpc::ChannelInterface> channel_; - class experimental_async async_stub_{this}; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>* AsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateArtifactResponse>* PrepareAsyncCreateArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateArtifactRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>* AsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::GetArtifactResponse>* PrepareAsyncGetArtifactRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::GetArtifactRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* AsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* PrepareAsyncSearchArtifactsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::SearchArtifactsRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>* AsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::CreateTriggerResponse>* PrepareAsyncCreateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::CreateTriggerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>* AsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::DeactivateTriggerResponse>* PrepareAsyncDeactivateTriggerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>* AsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::AddTagResponse>* PrepareAsyncAddTagRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::AddTagRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* AsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* PrepareAsyncRegisterProducerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterProducerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* AsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::RegisterResponse>* PrepareAsyncRegisterConsumerRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::RegisterConsumerRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>* AsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ExecutionInputsResponse>* PrepareAsyncSetExecutionInputsRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ExecutionInputsRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* AsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::SearchArtifactsResponse>* PrepareAsyncFindByWorkflowExecRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>* AsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) override; - ::grpc::ClientAsyncResponseReader< ::flyteidl::artifact::ListUsageResponse>* PrepareAsyncListUsageRaw(::grpc::ClientContext* context, const ::flyteidl::artifact::ListUsageRequest& request, ::grpc::CompletionQueue* cq) override; - const ::grpc::internal::RpcMethod rpcmethod_CreateArtifact_; - const ::grpc::internal::RpcMethod rpcmethod_GetArtifact_; - const ::grpc::internal::RpcMethod rpcmethod_SearchArtifacts_; - const ::grpc::internal::RpcMethod rpcmethod_CreateTrigger_; - const ::grpc::internal::RpcMethod rpcmethod_DeactivateTrigger_; - const ::grpc::internal::RpcMethod rpcmethod_AddTag_; - const ::grpc::internal::RpcMethod rpcmethod_RegisterProducer_; - const ::grpc::internal::RpcMethod rpcmethod_RegisterConsumer_; - const ::grpc::internal::RpcMethod rpcmethod_SetExecutionInputs_; - const ::grpc::internal::RpcMethod rpcmethod_FindByWorkflowExec_; - const ::grpc::internal::RpcMethod rpcmethod_ListUsage_; - }; - static std::unique_ptr NewStub(const std::shared_ptr< ::grpc::ChannelInterface>& channel, const ::grpc::StubOptions& options = ::grpc::StubOptions()); - - class Service : public ::grpc::Service { - public: - Service(); - virtual ~Service(); - virtual ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response); - virtual ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response); - virtual ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response); - virtual ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response); - virtual ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response); - virtual ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response); - virtual ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response); - virtual ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response); - virtual ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response); - virtual ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response); - virtual ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response); - }; - template - class WithAsyncMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_CreateArtifact() { - ::grpc::Service::MarkMethodAsync(0); - } - ~WithAsyncMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestCreateArtifact(::grpc::ServerContext* context, ::flyteidl::artifact::CreateArtifactRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::CreateArtifactResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(0, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_GetArtifact() { - ::grpc::Service::MarkMethodAsync(1); - } - ~WithAsyncMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestGetArtifact(::grpc::ServerContext* context, ::flyteidl::artifact::GetArtifactRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::GetArtifactResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(1, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_SearchArtifacts() { - ::grpc::Service::MarkMethodAsync(2); - } - ~WithAsyncMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestSearchArtifacts(::grpc::ServerContext* context, ::flyteidl::artifact::SearchArtifactsRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::SearchArtifactsResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(2, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_CreateTrigger() { - ::grpc::Service::MarkMethodAsync(3); - } - ~WithAsyncMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestCreateTrigger(::grpc::ServerContext* context, ::flyteidl::artifact::CreateTriggerRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::CreateTriggerResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(3, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_DeactivateTrigger() { - ::grpc::Service::MarkMethodAsync(4); - } - ~WithAsyncMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestDeactivateTrigger(::grpc::ServerContext* context, ::flyteidl::artifact::DeactivateTriggerRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::DeactivateTriggerResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(4, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_AddTag() { - ::grpc::Service::MarkMethodAsync(5); - } - ~WithAsyncMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestAddTag(::grpc::ServerContext* context, ::flyteidl::artifact::AddTagRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::AddTagResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(5, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_RegisterProducer() { - ::grpc::Service::MarkMethodAsync(6); - } - ~WithAsyncMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestRegisterProducer(::grpc::ServerContext* context, ::flyteidl::artifact::RegisterProducerRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::RegisterResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(6, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_RegisterConsumer() { - ::grpc::Service::MarkMethodAsync(7); - } - ~WithAsyncMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestRegisterConsumer(::grpc::ServerContext* context, ::flyteidl::artifact::RegisterConsumerRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::RegisterResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(7, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_SetExecutionInputs() { - ::grpc::Service::MarkMethodAsync(8); - } - ~WithAsyncMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestSetExecutionInputs(::grpc::ServerContext* context, ::flyteidl::artifact::ExecutionInputsRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::ExecutionInputsResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(8, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_FindByWorkflowExec() { - ::grpc::Service::MarkMethodAsync(9); - } - ~WithAsyncMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestFindByWorkflowExec(::grpc::ServerContext* context, ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::SearchArtifactsResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(9, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithAsyncMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithAsyncMethod_ListUsage() { - ::grpc::Service::MarkMethodAsync(10); - } - ~WithAsyncMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestListUsage(::grpc::ServerContext* context, ::flyteidl::artifact::ListUsageRequest* request, ::grpc::ServerAsyncResponseWriter< ::flyteidl::artifact::ListUsageResponse>* response, ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) { - ::grpc::Service::RequestAsyncUnary(10, context, request, response, new_call_cq, notification_cq, tag); - } - }; - typedef WithAsyncMethod_CreateArtifact > > > > > > > > > > AsyncService; - template - class ExperimentalWithCallbackMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_CreateArtifact() { - ::grpc::Service::experimental().MarkMethodCallback(0, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::CreateArtifactRequest, ::flyteidl::artifact::CreateArtifactResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::CreateArtifactRequest* request, - ::flyteidl::artifact::CreateArtifactResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->CreateArtifact(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_CreateArtifact( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::CreateArtifactRequest, ::flyteidl::artifact::CreateArtifactResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::CreateArtifactRequest, ::flyteidl::artifact::CreateArtifactResponse>*>( - ::grpc::Service::experimental().GetHandler(0)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_GetArtifact() { - ::grpc::Service::experimental().MarkMethodCallback(1, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::GetArtifactRequest, ::flyteidl::artifact::GetArtifactResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::GetArtifactRequest* request, - ::flyteidl::artifact::GetArtifactResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->GetArtifact(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_GetArtifact( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::GetArtifactRequest, ::flyteidl::artifact::GetArtifactResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::GetArtifactRequest, ::flyteidl::artifact::GetArtifactResponse>*>( - ::grpc::Service::experimental().GetHandler(1)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_SearchArtifacts() { - ::grpc::Service::experimental().MarkMethodCallback(2, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::SearchArtifactsRequest, ::flyteidl::artifact::SearchArtifactsResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::SearchArtifactsRequest* request, - ::flyteidl::artifact::SearchArtifactsResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->SearchArtifacts(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_SearchArtifacts( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::SearchArtifactsRequest, ::flyteidl::artifact::SearchArtifactsResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::SearchArtifactsRequest, ::flyteidl::artifact::SearchArtifactsResponse>*>( - ::grpc::Service::experimental().GetHandler(2)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_CreateTrigger() { - ::grpc::Service::experimental().MarkMethodCallback(3, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::CreateTriggerRequest, ::flyteidl::artifact::CreateTriggerResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::CreateTriggerRequest* request, - ::flyteidl::artifact::CreateTriggerResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->CreateTrigger(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_CreateTrigger( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::CreateTriggerRequest, ::flyteidl::artifact::CreateTriggerResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::CreateTriggerRequest, ::flyteidl::artifact::CreateTriggerResponse>*>( - ::grpc::Service::experimental().GetHandler(3)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_DeactivateTrigger() { - ::grpc::Service::experimental().MarkMethodCallback(4, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::DeactivateTriggerRequest, ::flyteidl::artifact::DeactivateTriggerResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::DeactivateTriggerRequest* request, - ::flyteidl::artifact::DeactivateTriggerResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->DeactivateTrigger(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_DeactivateTrigger( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::DeactivateTriggerRequest, ::flyteidl::artifact::DeactivateTriggerResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::DeactivateTriggerRequest, ::flyteidl::artifact::DeactivateTriggerResponse>*>( - ::grpc::Service::experimental().GetHandler(4)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_AddTag() { - ::grpc::Service::experimental().MarkMethodCallback(5, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::AddTagRequest, ::flyteidl::artifact::AddTagResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::AddTagRequest* request, - ::flyteidl::artifact::AddTagResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->AddTag(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_AddTag( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::AddTagRequest, ::flyteidl::artifact::AddTagResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::AddTagRequest, ::flyteidl::artifact::AddTagResponse>*>( - ::grpc::Service::experimental().GetHandler(5)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_RegisterProducer() { - ::grpc::Service::experimental().MarkMethodCallback(6, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::RegisterProducerRequest, ::flyteidl::artifact::RegisterResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::RegisterProducerRequest* request, - ::flyteidl::artifact::RegisterResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->RegisterProducer(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_RegisterProducer( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::RegisterProducerRequest, ::flyteidl::artifact::RegisterResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::RegisterProducerRequest, ::flyteidl::artifact::RegisterResponse>*>( - ::grpc::Service::experimental().GetHandler(6)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_RegisterConsumer() { - ::grpc::Service::experimental().MarkMethodCallback(7, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::RegisterConsumerRequest, ::flyteidl::artifact::RegisterResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::RegisterConsumerRequest* request, - ::flyteidl::artifact::RegisterResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->RegisterConsumer(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_RegisterConsumer( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::RegisterConsumerRequest, ::flyteidl::artifact::RegisterResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::RegisterConsumerRequest, ::flyteidl::artifact::RegisterResponse>*>( - ::grpc::Service::experimental().GetHandler(7)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_SetExecutionInputs() { - ::grpc::Service::experimental().MarkMethodCallback(8, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::ExecutionInputsRequest, ::flyteidl::artifact::ExecutionInputsResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::ExecutionInputsRequest* request, - ::flyteidl::artifact::ExecutionInputsResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->SetExecutionInputs(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_SetExecutionInputs( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::ExecutionInputsRequest, ::flyteidl::artifact::ExecutionInputsResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::ExecutionInputsRequest, ::flyteidl::artifact::ExecutionInputsResponse>*>( - ::grpc::Service::experimental().GetHandler(8)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_FindByWorkflowExec() { - ::grpc::Service::experimental().MarkMethodCallback(9, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::FindByWorkflowExecRequest, ::flyteidl::artifact::SearchArtifactsResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::FindByWorkflowExecRequest* request, - ::flyteidl::artifact::SearchArtifactsResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->FindByWorkflowExec(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_FindByWorkflowExec( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::FindByWorkflowExecRequest, ::flyteidl::artifact::SearchArtifactsResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::FindByWorkflowExecRequest, ::flyteidl::artifact::SearchArtifactsResponse>*>( - ::grpc::Service::experimental().GetHandler(9)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithCallbackMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithCallbackMethod_ListUsage() { - ::grpc::Service::experimental().MarkMethodCallback(10, - new ::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::ListUsageRequest, ::flyteidl::artifact::ListUsageResponse>( - [this](::grpc::ServerContext* context, - const ::flyteidl::artifact::ListUsageRequest* request, - ::flyteidl::artifact::ListUsageResponse* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - return this->ListUsage(context, request, response, controller); - })); - } - void SetMessageAllocatorFor_ListUsage( - ::grpc::experimental::MessageAllocator< ::flyteidl::artifact::ListUsageRequest, ::flyteidl::artifact::ListUsageResponse>* allocator) { - static_cast<::grpc::internal::CallbackUnaryHandler< ::flyteidl::artifact::ListUsageRequest, ::flyteidl::artifact::ListUsageResponse>*>( - ::grpc::Service::experimental().GetHandler(10)) - ->SetMessageAllocator(allocator); - } - ~ExperimentalWithCallbackMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - typedef ExperimentalWithCallbackMethod_CreateArtifact > > > > > > > > > > ExperimentalCallbackService; - template - class WithGenericMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_CreateArtifact() { - ::grpc::Service::MarkMethodGeneric(0); - } - ~WithGenericMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_GetArtifact() { - ::grpc::Service::MarkMethodGeneric(1); - } - ~WithGenericMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_SearchArtifacts() { - ::grpc::Service::MarkMethodGeneric(2); - } - ~WithGenericMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_CreateTrigger() { - ::grpc::Service::MarkMethodGeneric(3); - } - ~WithGenericMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_DeactivateTrigger() { - ::grpc::Service::MarkMethodGeneric(4); - } - ~WithGenericMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_AddTag() { - ::grpc::Service::MarkMethodGeneric(5); - } - ~WithGenericMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_RegisterProducer() { - ::grpc::Service::MarkMethodGeneric(6); - } - ~WithGenericMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_RegisterConsumer() { - ::grpc::Service::MarkMethodGeneric(7); - } - ~WithGenericMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_SetExecutionInputs() { - ::grpc::Service::MarkMethodGeneric(8); - } - ~WithGenericMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_FindByWorkflowExec() { - ::grpc::Service::MarkMethodGeneric(9); - } - ~WithGenericMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithGenericMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithGenericMethod_ListUsage() { - ::grpc::Service::MarkMethodGeneric(10); - } - ~WithGenericMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - }; - template - class WithRawMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_CreateArtifact() { - ::grpc::Service::MarkMethodRaw(0); - } - ~WithRawMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestCreateArtifact(::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(0, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_GetArtifact() { - ::grpc::Service::MarkMethodRaw(1); - } - ~WithRawMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestGetArtifact(::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(1, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_SearchArtifacts() { - ::grpc::Service::MarkMethodRaw(2); - } - ~WithRawMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestSearchArtifacts(::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(2, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_CreateTrigger() { - ::grpc::Service::MarkMethodRaw(3); - } - ~WithRawMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestCreateTrigger(::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(3, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_DeactivateTrigger() { - ::grpc::Service::MarkMethodRaw(4); - } - ~WithRawMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestDeactivateTrigger(::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(4, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_AddTag() { - ::grpc::Service::MarkMethodRaw(5); - } - ~WithRawMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestAddTag(::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(5, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_RegisterProducer() { - ::grpc::Service::MarkMethodRaw(6); - } - ~WithRawMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestRegisterProducer(::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(6, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_RegisterConsumer() { - ::grpc::Service::MarkMethodRaw(7); - } - ~WithRawMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestRegisterConsumer(::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(7, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_SetExecutionInputs() { - ::grpc::Service::MarkMethodRaw(8); - } - ~WithRawMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestSetExecutionInputs(::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(8, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_FindByWorkflowExec() { - ::grpc::Service::MarkMethodRaw(9); - } - ~WithRawMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestFindByWorkflowExec(::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(9, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class WithRawMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithRawMethod_ListUsage() { - ::grpc::Service::MarkMethodRaw(10); - } - ~WithRawMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - void RequestListUsage(::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(10, context, request, response, new_call_cq, notification_cq, tag); - } - }; - template - class ExperimentalWithRawCallbackMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_CreateArtifact() { - ::grpc::Service::experimental().MarkMethodRawCallback(0, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->CreateArtifact(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void CreateArtifact(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_GetArtifact() { - ::grpc::Service::experimental().MarkMethodRawCallback(1, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->GetArtifact(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void GetArtifact(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_SearchArtifacts() { - ::grpc::Service::experimental().MarkMethodRawCallback(2, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->SearchArtifacts(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void SearchArtifacts(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_CreateTrigger() { - ::grpc::Service::experimental().MarkMethodRawCallback(3, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->CreateTrigger(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void CreateTrigger(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_DeactivateTrigger() { - ::grpc::Service::experimental().MarkMethodRawCallback(4, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->DeactivateTrigger(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void DeactivateTrigger(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_AddTag() { - ::grpc::Service::experimental().MarkMethodRawCallback(5, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->AddTag(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void AddTag(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_RegisterProducer() { - ::grpc::Service::experimental().MarkMethodRawCallback(6, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->RegisterProducer(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void RegisterProducer(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_RegisterConsumer() { - ::grpc::Service::experimental().MarkMethodRawCallback(7, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->RegisterConsumer(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void RegisterConsumer(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_SetExecutionInputs() { - ::grpc::Service::experimental().MarkMethodRawCallback(8, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->SetExecutionInputs(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void SetExecutionInputs(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_FindByWorkflowExec() { - ::grpc::Service::experimental().MarkMethodRawCallback(9, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->FindByWorkflowExec(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void FindByWorkflowExec(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class ExperimentalWithRawCallbackMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - ExperimentalWithRawCallbackMethod_ListUsage() { - ::grpc::Service::experimental().MarkMethodRawCallback(10, - new ::grpc::internal::CallbackUnaryHandler< ::grpc::ByteBuffer, ::grpc::ByteBuffer>( - [this](::grpc::ServerContext* context, - const ::grpc::ByteBuffer* request, - ::grpc::ByteBuffer* response, - ::grpc::experimental::ServerCallbackRpcController* controller) { - this->ListUsage(context, request, response, controller); - })); - } - ~ExperimentalWithRawCallbackMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable synchronous version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - virtual void ListUsage(::grpc::ServerContext* context, const ::grpc::ByteBuffer* request, ::grpc::ByteBuffer* response, ::grpc::experimental::ServerCallbackRpcController* controller) { controller->Finish(::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, "")); } - }; - template - class WithStreamedUnaryMethod_CreateArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_CreateArtifact() { - ::grpc::Service::MarkMethodStreamed(0, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::CreateArtifactRequest, ::flyteidl::artifact::CreateArtifactResponse>(std::bind(&WithStreamedUnaryMethod_CreateArtifact::StreamedCreateArtifact, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_CreateArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status CreateArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateArtifactRequest* request, ::flyteidl::artifact::CreateArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedCreateArtifact(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::CreateArtifactRequest,::flyteidl::artifact::CreateArtifactResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_GetArtifact : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_GetArtifact() { - ::grpc::Service::MarkMethodStreamed(1, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::GetArtifactRequest, ::flyteidl::artifact::GetArtifactResponse>(std::bind(&WithStreamedUnaryMethod_GetArtifact::StreamedGetArtifact, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_GetArtifact() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status GetArtifact(::grpc::ServerContext* context, const ::flyteidl::artifact::GetArtifactRequest* request, ::flyteidl::artifact::GetArtifactResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedGetArtifact(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::GetArtifactRequest,::flyteidl::artifact::GetArtifactResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_SearchArtifacts : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_SearchArtifacts() { - ::grpc::Service::MarkMethodStreamed(2, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::SearchArtifactsRequest, ::flyteidl::artifact::SearchArtifactsResponse>(std::bind(&WithStreamedUnaryMethod_SearchArtifacts::StreamedSearchArtifacts, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_SearchArtifacts() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status SearchArtifacts(::grpc::ServerContext* context, const ::flyteidl::artifact::SearchArtifactsRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedSearchArtifacts(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::SearchArtifactsRequest,::flyteidl::artifact::SearchArtifactsResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_CreateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_CreateTrigger() { - ::grpc::Service::MarkMethodStreamed(3, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::CreateTriggerRequest, ::flyteidl::artifact::CreateTriggerResponse>(std::bind(&WithStreamedUnaryMethod_CreateTrigger::StreamedCreateTrigger, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_CreateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status CreateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::CreateTriggerRequest* request, ::flyteidl::artifact::CreateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedCreateTrigger(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::CreateTriggerRequest,::flyteidl::artifact::CreateTriggerResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_DeactivateTrigger : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_DeactivateTrigger() { - ::grpc::Service::MarkMethodStreamed(4, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::DeactivateTriggerRequest, ::flyteidl::artifact::DeactivateTriggerResponse>(std::bind(&WithStreamedUnaryMethod_DeactivateTrigger::StreamedDeactivateTrigger, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_DeactivateTrigger() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status DeactivateTrigger(::grpc::ServerContext* context, const ::flyteidl::artifact::DeactivateTriggerRequest* request, ::flyteidl::artifact::DeactivateTriggerResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedDeactivateTrigger(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::DeactivateTriggerRequest,::flyteidl::artifact::DeactivateTriggerResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_AddTag : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_AddTag() { - ::grpc::Service::MarkMethodStreamed(5, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::AddTagRequest, ::flyteidl::artifact::AddTagResponse>(std::bind(&WithStreamedUnaryMethod_AddTag::StreamedAddTag, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_AddTag() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status AddTag(::grpc::ServerContext* context, const ::flyteidl::artifact::AddTagRequest* request, ::flyteidl::artifact::AddTagResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedAddTag(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::AddTagRequest,::flyteidl::artifact::AddTagResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_RegisterProducer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_RegisterProducer() { - ::grpc::Service::MarkMethodStreamed(6, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::RegisterProducerRequest, ::flyteidl::artifact::RegisterResponse>(std::bind(&WithStreamedUnaryMethod_RegisterProducer::StreamedRegisterProducer, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_RegisterProducer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status RegisterProducer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterProducerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedRegisterProducer(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::RegisterProducerRequest,::flyteidl::artifact::RegisterResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_RegisterConsumer : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_RegisterConsumer() { - ::grpc::Service::MarkMethodStreamed(7, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::RegisterConsumerRequest, ::flyteidl::artifact::RegisterResponse>(std::bind(&WithStreamedUnaryMethod_RegisterConsumer::StreamedRegisterConsumer, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_RegisterConsumer() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status RegisterConsumer(::grpc::ServerContext* context, const ::flyteidl::artifact::RegisterConsumerRequest* request, ::flyteidl::artifact::RegisterResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedRegisterConsumer(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::RegisterConsumerRequest,::flyteidl::artifact::RegisterResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_SetExecutionInputs : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_SetExecutionInputs() { - ::grpc::Service::MarkMethodStreamed(8, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::ExecutionInputsRequest, ::flyteidl::artifact::ExecutionInputsResponse>(std::bind(&WithStreamedUnaryMethod_SetExecutionInputs::StreamedSetExecutionInputs, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_SetExecutionInputs() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status SetExecutionInputs(::grpc::ServerContext* context, const ::flyteidl::artifact::ExecutionInputsRequest* request, ::flyteidl::artifact::ExecutionInputsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedSetExecutionInputs(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::ExecutionInputsRequest,::flyteidl::artifact::ExecutionInputsResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_FindByWorkflowExec : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_FindByWorkflowExec() { - ::grpc::Service::MarkMethodStreamed(9, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::FindByWorkflowExecRequest, ::flyteidl::artifact::SearchArtifactsResponse>(std::bind(&WithStreamedUnaryMethod_FindByWorkflowExec::StreamedFindByWorkflowExec, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_FindByWorkflowExec() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status FindByWorkflowExec(::grpc::ServerContext* context, const ::flyteidl::artifact::FindByWorkflowExecRequest* request, ::flyteidl::artifact::SearchArtifactsResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedFindByWorkflowExec(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::FindByWorkflowExecRequest,::flyteidl::artifact::SearchArtifactsResponse>* server_unary_streamer) = 0; - }; - template - class WithStreamedUnaryMethod_ListUsage : public BaseClass { - private: - void BaseClassMustBeDerivedFromService(const Service *service) {} - public: - WithStreamedUnaryMethod_ListUsage() { - ::grpc::Service::MarkMethodStreamed(10, - new ::grpc::internal::StreamedUnaryHandler< ::flyteidl::artifact::ListUsageRequest, ::flyteidl::artifact::ListUsageResponse>(std::bind(&WithStreamedUnaryMethod_ListUsage::StreamedListUsage, this, std::placeholders::_1, std::placeholders::_2))); - } - ~WithStreamedUnaryMethod_ListUsage() override { - BaseClassMustBeDerivedFromService(this); - } - // disable regular version of this method - ::grpc::Status ListUsage(::grpc::ServerContext* context, const ::flyteidl::artifact::ListUsageRequest* request, ::flyteidl::artifact::ListUsageResponse* response) override { - abort(); - return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); - } - // replace default version of method with streamed unary - virtual ::grpc::Status StreamedListUsage(::grpc::ServerContext* context, ::grpc::ServerUnaryStreamer< ::flyteidl::artifact::ListUsageRequest,::flyteidl::artifact::ListUsageResponse>* server_unary_streamer) = 0; - }; - typedef WithStreamedUnaryMethod_CreateArtifact > > > > > > > > > > StreamedUnaryService; - typedef Service SplitStreamedService; - typedef WithStreamedUnaryMethod_CreateArtifact > > > > > > > > > > StreamedService; -}; - -} // namespace artifact -} // namespace flyteidl - - -#endif // GRPC_flyteidl_2fartifact_2fartifacts_2eproto__INCLUDED diff --git a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.cc deleted file mode 100644 index a8d21adedf..0000000000 --- a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.cc +++ /dev/null @@ -1,10654 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: flyteidl/artifact/artifacts.proto - -#include "flyteidl/artifact/artifacts.pb.h" - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -// @@protoc_insertion_point(includes) -#include - -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fadmin_2flaunch_5fplan_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_LaunchPlan_flyteidl_2fadmin_2flaunch_5fplan_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_CreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_SearchOptions_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_ArtifactKey_flyteidl_2fcore_2fartifact_5fid_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_Partitions_flyteidl_2fcore_2fartifact_5fid_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ArtifactQuery_flyteidl_2fcore_2fartifact_5fid_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fidentifier_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fidentifier_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_WorkflowExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2finterface_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_ParameterMap_flyteidl_2fcore_2finterface_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2finterface_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_VariableMap_flyteidl_2fcore_2finterface_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fliterals_2eproto ::google::protobuf::internal::SCCInfo<10> scc_info_Literal_flyteidl_2fcore_2fliterals_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2ftypes_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_LiteralType_flyteidl_2fcore_2ftypes_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_google_2fprotobuf_2fany_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Any_google_2fprotobuf_2fany_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_google_2fprotobuf_2ftimestamp_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Timestamp_google_2fprotobuf_2ftimestamp_2eproto; -namespace flyteidl { -namespace artifact { -class ArtifactDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _Artifact_default_instance_; -class CreateArtifactRequest_PartitionsEntry_DoNotUseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _CreateArtifactRequest_PartitionsEntry_DoNotUse_default_instance_; -class CreateArtifactRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _CreateArtifactRequest_default_instance_; -class ArtifactSourceDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ArtifactSource_default_instance_; -class ArtifactSpecDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ArtifactSpec_default_instance_; -class CreateArtifactResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _CreateArtifactResponse_default_instance_; -class GetArtifactRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _GetArtifactRequest_default_instance_; -class GetArtifactResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _GetArtifactResponse_default_instance_; -class SearchOptionsDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _SearchOptions_default_instance_; -class SearchArtifactsRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _SearchArtifactsRequest_default_instance_; -class SearchArtifactsResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _SearchArtifactsResponse_default_instance_; -class FindByWorkflowExecRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _FindByWorkflowExecRequest_default_instance_; -class AddTagRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _AddTagRequest_default_instance_; -class AddTagResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _AddTagResponse_default_instance_; -class CreateTriggerRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _CreateTriggerRequest_default_instance_; -class CreateTriggerResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _CreateTriggerResponse_default_instance_; -class DeactivateTriggerRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _DeactivateTriggerRequest_default_instance_; -class DeactivateTriggerResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _DeactivateTriggerResponse_default_instance_; -class ArtifactProducerDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ArtifactProducer_default_instance_; -class RegisterProducerRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _RegisterProducerRequest_default_instance_; -class ArtifactConsumerDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ArtifactConsumer_default_instance_; -class RegisterConsumerRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _RegisterConsumerRequest_default_instance_; -class RegisterResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _RegisterResponse_default_instance_; -class ExecutionInputsRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ExecutionInputsRequest_default_instance_; -class ExecutionInputsResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ExecutionInputsResponse_default_instance_; -class ListUsageRequestDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ListUsageRequest_default_instance_; -class ListUsageResponseDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _ListUsageResponse_default_instance_; -} // namespace artifact -} // namespace flyteidl -static void InitDefaultsArtifact_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_Artifact_default_instance_; - new (ptr) ::flyteidl::artifact::Artifact(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::Artifact::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<3> scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 3, InitDefaultsArtifact_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base, - &scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto.base, - &scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsCreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_CreateArtifactRequest_PartitionsEntry_DoNotUse_default_instance_; - new (ptr) ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse(); - } - ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_CreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsCreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsCreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_CreateArtifactRequest_default_instance_; - new (ptr) ::flyteidl::artifact::CreateArtifactRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::CreateArtifactRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<5> scc_info_CreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 5, InitDefaultsCreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactKey_flyteidl_2fcore_2fartifact_5fid_2eproto.base, - &scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto.base, - &scc_info_CreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto.base, - &scc_info_Timestamp_google_2fprotobuf_2ftimestamp_2eproto.base, - &scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ArtifactSource_default_instance_; - new (ptr) ::flyteidl::artifact::ArtifactSource(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ArtifactSource::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_WorkflowExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto.base, - &scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto.base,}}; - -static void InitDefaultsArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ArtifactSpec_default_instance_; - new (ptr) ::flyteidl::artifact::ArtifactSpec(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ArtifactSpec::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<4> scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 4, InitDefaultsArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Literal_flyteidl_2fcore_2fliterals_2eproto.base, - &scc_info_LiteralType_flyteidl_2fcore_2ftypes_2eproto.base, - &scc_info_Any_google_2fprotobuf_2fany_2eproto.base, - &scc_info_Timestamp_google_2fprotobuf_2ftimestamp_2eproto.base,}}; - -static void InitDefaultsCreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_CreateArtifactResponse_default_instance_; - new (ptr) ::flyteidl::artifact::CreateArtifactResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::CreateArtifactResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_CreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsCreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsGetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_GetArtifactRequest_default_instance_; - new (ptr) ::flyteidl::artifact::GetArtifactRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::GetArtifactRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_GetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsGetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactQuery_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; - -static void InitDefaultsGetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_GetArtifactResponse_default_instance_; - new (ptr) ::flyteidl::artifact::GetArtifactResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::GetArtifactResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_GetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsGetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsSearchOptions_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_SearchOptions_default_instance_; - new (ptr) ::flyteidl::artifact::SearchOptions(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::SearchOptions::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_SearchOptions_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsSearchOptions_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsSearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_SearchArtifactsRequest_default_instance_; - new (ptr) ::flyteidl::artifact::SearchArtifactsRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::SearchArtifactsRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<4> scc_info_SearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 4, InitDefaultsSearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactKey_flyteidl_2fcore_2fartifact_5fid_2eproto.base, - &scc_info_Partitions_flyteidl_2fcore_2fartifact_5fid_2eproto.base, - &scc_info_Timestamp_google_2fprotobuf_2ftimestamp_2eproto.base, - &scc_info_SearchOptions_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsSearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_SearchArtifactsResponse_default_instance_; - new (ptr) ::flyteidl::artifact::SearchArtifactsResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::SearchArtifactsResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_SearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsSearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsFindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_FindByWorkflowExecRequest_default_instance_; - new (ptr) ::flyteidl::artifact::FindByWorkflowExecRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::FindByWorkflowExecRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_FindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsFindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_WorkflowExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto.base,}}; - -static void InitDefaultsAddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_AddTagRequest_default_instance_; - new (ptr) ::flyteidl::artifact::AddTagRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::AddTagRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_AddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsAddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; - -static void InitDefaultsAddTagResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_AddTagResponse_default_instance_; - new (ptr) ::flyteidl::artifact::AddTagResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::AddTagResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_AddTagResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsAddTagResponse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsCreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_CreateTriggerRequest_default_instance_; - new (ptr) ::flyteidl::artifact::CreateTriggerRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::CreateTriggerRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_CreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsCreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_LaunchPlan_flyteidl_2fadmin_2flaunch_5fplan_2eproto.base,}}; - -static void InitDefaultsCreateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_CreateTriggerResponse_default_instance_; - new (ptr) ::flyteidl::artifact::CreateTriggerResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::CreateTriggerResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_CreateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsCreateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsDeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_DeactivateTriggerRequest_default_instance_; - new (ptr) ::flyteidl::artifact::DeactivateTriggerRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::DeactivateTriggerRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_DeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsDeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto.base,}}; - -static void InitDefaultsDeactivateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_DeactivateTriggerResponse_default_instance_; - new (ptr) ::flyteidl::artifact::DeactivateTriggerResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::DeactivateTriggerResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_DeactivateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsDeactivateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ArtifactProducer_default_instance_; - new (ptr) ::flyteidl::artifact::ArtifactProducer(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ArtifactProducer::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto.base, - &scc_info_VariableMap_flyteidl_2fcore_2finterface_2eproto.base,}}; - -static void InitDefaultsRegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_RegisterProducerRequest_default_instance_; - new (ptr) ::flyteidl::artifact::RegisterProducerRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::RegisterProducerRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_RegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsRegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ArtifactConsumer_default_instance_; - new (ptr) ::flyteidl::artifact::ArtifactConsumer(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ArtifactConsumer::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto.base, - &scc_info_ParameterMap_flyteidl_2fcore_2finterface_2eproto.base,}}; - -static void InitDefaultsRegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_RegisterConsumerRequest_default_instance_; - new (ptr) ::flyteidl::artifact::RegisterConsumerRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::RegisterConsumerRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_RegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsRegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto.base,}}; - -static void InitDefaultsRegisterResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_RegisterResponse_default_instance_; - new (ptr) ::flyteidl::artifact::RegisterResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::RegisterResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_RegisterResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsRegisterResponse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ExecutionInputsRequest_default_instance_; - new (ptr) ::flyteidl::artifact::ExecutionInputsRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ExecutionInputsRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<2> scc_info_ExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_WorkflowExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto.base, - &scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; - -static void InitDefaultsExecutionInputsResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ExecutionInputsResponse_default_instance_; - new (ptr) ::flyteidl::artifact::ExecutionInputsResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ExecutionInputsResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<0> scc_info_ExecutionInputsResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsExecutionInputsResponse_flyteidl_2fartifact_2fartifacts_2eproto}, {}}; - -static void InitDefaultsListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ListUsageRequest_default_instance_; - new (ptr) ::flyteidl::artifact::ListUsageRequest(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ListUsageRequest::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_ListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; - -static void InitDefaultsListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::artifact::_ListUsageResponse_default_instance_; - new (ptr) ::flyteidl::artifact::ListUsageResponse(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::artifact::ListUsageResponse::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<1> scc_info_ListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto}, { - &scc_info_WorkflowExecutionIdentifier_flyteidl_2fcore_2fidentifier_2eproto.base,}}; - -void InitDefaults_flyteidl_2fartifact_2fartifacts_2eproto() { - ::google::protobuf::internal::InitSCC(&scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_CreateArtifactRequest_PartitionsEntry_DoNotUse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_CreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_CreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_GetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_GetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_SearchOptions_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_SearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_SearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_FindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_AddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_AddTagResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_CreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_CreateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_DeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_DeactivateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_RegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_RegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_RegisterResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ExecutionInputsResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_ListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); -} - -::google::protobuf::Metadata file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[27]; -const ::google::protobuf::EnumDescriptor* file_level_enum_descriptors_flyteidl_2fartifact_2fartifacts_2eproto[1]; -constexpr ::google::protobuf::ServiceDescriptor const** file_level_service_descriptors_flyteidl_2fartifact_2fartifacts_2eproto = nullptr; - -const ::google::protobuf::uint32 TableStruct_flyteidl_2fartifact_2fartifacts_2eproto::offsets[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::Artifact, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::Artifact, artifact_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::Artifact, spec_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::Artifact, tags_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::Artifact, source_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse, _has_bits_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse, key_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse, value_), - 0, - 1, - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, artifact_key_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, version_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, spec_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, partitions_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, time_partition_value_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactRequest, source_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, workflow_execution_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, node_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, task_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, retry_attempt_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSource, principal_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, value_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, type_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, short_description_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, user_metadata_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, metadata_type_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, created_at_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactSpec, file_format_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateArtifactResponse, artifact_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::GetArtifactRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::GetArtifactRequest, query_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::GetArtifactRequest, details_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::GetArtifactResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::GetArtifactResponse, artifact_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchOptions, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchOptions, strict_partitions_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchOptions, latest_by_key_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, artifact_key_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, partitions_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, time_partition_value_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, principal_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, version_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, options_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, token_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsRequest, limit_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsResponse, artifacts_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::SearchArtifactsResponse, token_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::FindByWorkflowExecRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::FindByWorkflowExecRequest, exec_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::FindByWorkflowExecRequest, direction_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::AddTagRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::AddTagRequest, artifact_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::AddTagRequest, value_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::AddTagRequest, overwrite_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::AddTagResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateTriggerRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateTriggerRequest, trigger_launch_plan_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::CreateTriggerResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::DeactivateTriggerRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::DeactivateTriggerRequest, trigger_id_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::DeactivateTriggerResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactProducer, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactProducer, entity_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactProducer, outputs_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::RegisterProducerRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::RegisterProducerRequest, producers_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactConsumer, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactConsumer, entity_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ArtifactConsumer, inputs_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::RegisterConsumerRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::RegisterConsumerRequest, consumers_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::RegisterResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ExecutionInputsRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ExecutionInputsRequest, execution_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ExecutionInputsRequest, inputs_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ExecutionInputsResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ListUsageRequest, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ListUsageRequest, artifact_id_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ListUsageResponse, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::artifact::ListUsageResponse, executions_), -}; -static const ::google::protobuf::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { - { 0, -1, sizeof(::flyteidl::artifact::Artifact)}, - { 9, 16, sizeof(::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse)}, - { 18, -1, sizeof(::flyteidl::artifact::CreateArtifactRequest)}, - { 29, -1, sizeof(::flyteidl::artifact::ArtifactSource)}, - { 39, -1, sizeof(::flyteidl::artifact::ArtifactSpec)}, - { 51, -1, sizeof(::flyteidl::artifact::CreateArtifactResponse)}, - { 57, -1, sizeof(::flyteidl::artifact::GetArtifactRequest)}, - { 64, -1, sizeof(::flyteidl::artifact::GetArtifactResponse)}, - { 70, -1, sizeof(::flyteidl::artifact::SearchOptions)}, - { 77, -1, sizeof(::flyteidl::artifact::SearchArtifactsRequest)}, - { 90, -1, sizeof(::flyteidl::artifact::SearchArtifactsResponse)}, - { 97, -1, sizeof(::flyteidl::artifact::FindByWorkflowExecRequest)}, - { 104, -1, sizeof(::flyteidl::artifact::AddTagRequest)}, - { 112, -1, sizeof(::flyteidl::artifact::AddTagResponse)}, - { 117, -1, sizeof(::flyteidl::artifact::CreateTriggerRequest)}, - { 123, -1, sizeof(::flyteidl::artifact::CreateTriggerResponse)}, - { 128, -1, sizeof(::flyteidl::artifact::DeactivateTriggerRequest)}, - { 134, -1, sizeof(::flyteidl::artifact::DeactivateTriggerResponse)}, - { 139, -1, sizeof(::flyteidl::artifact::ArtifactProducer)}, - { 146, -1, sizeof(::flyteidl::artifact::RegisterProducerRequest)}, - { 152, -1, sizeof(::flyteidl::artifact::ArtifactConsumer)}, - { 159, -1, sizeof(::flyteidl::artifact::RegisterConsumerRequest)}, - { 165, -1, sizeof(::flyteidl::artifact::RegisterResponse)}, - { 170, -1, sizeof(::flyteidl::artifact::ExecutionInputsRequest)}, - { 177, -1, sizeof(::flyteidl::artifact::ExecutionInputsResponse)}, - { 182, -1, sizeof(::flyteidl::artifact::ListUsageRequest)}, - { 188, -1, sizeof(::flyteidl::artifact::ListUsageResponse)}, -}; - -static ::google::protobuf::Message const * const file_default_instances[] = { - reinterpret_cast(&::flyteidl::artifact::_Artifact_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_CreateArtifactRequest_PartitionsEntry_DoNotUse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_CreateArtifactRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ArtifactSource_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ArtifactSpec_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_CreateArtifactResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_GetArtifactRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_GetArtifactResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_SearchOptions_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_SearchArtifactsRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_SearchArtifactsResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_FindByWorkflowExecRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_AddTagRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_AddTagResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_CreateTriggerRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_CreateTriggerResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_DeactivateTriggerRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_DeactivateTriggerResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ArtifactProducer_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_RegisterProducerRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ArtifactConsumer_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_RegisterConsumerRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_RegisterResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ExecutionInputsRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ExecutionInputsResponse_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ListUsageRequest_default_instance_), - reinterpret_cast(&::flyteidl::artifact::_ListUsageResponse_default_instance_), -}; - -::google::protobuf::internal::AssignDescriptorsTable assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto = { - {}, AddDescriptors_flyteidl_2fartifact_2fartifacts_2eproto, "flyteidl/artifact/artifacts.proto", schemas, - file_default_instances, TableStruct_flyteidl_2fartifact_2fartifacts_2eproto::offsets, - file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto, 27, file_level_enum_descriptors_flyteidl_2fartifact_2fartifacts_2eproto, file_level_service_descriptors_flyteidl_2fartifact_2fartifacts_2eproto, -}; - -const char descriptor_table_protodef_flyteidl_2fartifact_2fartifacts_2eproto[] = - "\n!flyteidl/artifact/artifacts.proto\022\021fly" - "teidl.artifact\032\031google/protobuf/any.prot" - "o\032\034google/api/annotations.proto\032\037google/" - "protobuf/timestamp.proto\032 flyteidl/admin" - "/launch_plan.proto\032\034flyteidl/core/litera" - "ls.proto\032\031flyteidl/core/types.proto\032\036fly" - "teidl/core/identifier.proto\032\037flyteidl/co" - "re/artifact_id.proto\032\035flyteidl/core/inte" - "rface.proto\032 flyteidl/event/cloudevents." - "proto\"\252\001\n\010Artifact\022.\n\013artifact_id\030\001 \001(\0132" - "\031.flyteidl.core.ArtifactID\022-\n\004spec\030\002 \001(\013" - "2\037.flyteidl.artifact.ArtifactSpec\022\014\n\004tag" - "s\030\003 \003(\t\0221\n\006source\030\004 \001(\0132!.flyteidl.artif" - "act.ArtifactSource\"\367\002\n\025CreateArtifactReq" - "uest\0220\n\014artifact_key\030\001 \001(\0132\032.flyteidl.co" - "re.ArtifactKey\022\017\n\007version\030\003 \001(\t\022-\n\004spec\030" - "\002 \001(\0132\037.flyteidl.artifact.ArtifactSpec\022L" - "\n\npartitions\030\004 \003(\01328.flyteidl.artifact.C" - "reateArtifactRequest.PartitionsEntry\0228\n\024" - "time_partition_value\030\005 \001(\0132\032.google.prot" - "obuf.Timestamp\0221\n\006source\030\006 \001(\0132!.flyteid" - "l.artifact.ArtifactSource\0321\n\017PartitionsE" - "ntry\022\013\n\003key\030\001 \001(\t\022\r\n\005value\030\002 \001(\t:\0028\001\"\277\001\n" - "\016ArtifactSource\022F\n\022workflow_execution\030\001 " - "\001(\0132*.flyteidl.core.WorkflowExecutionIde" - "ntifier\022\017\n\007node_id\030\002 \001(\t\022*\n\007task_id\030\003 \001(" - "\0132\031.flyteidl.core.Identifier\022\025\n\rretry_at" - "tempt\030\004 \001(\r\022\021\n\tprincipal\030\005 \001(\t\"\203\002\n\014Artif" - "actSpec\022%\n\005value\030\001 \001(\0132\026.flyteidl.core.L" - "iteral\022(\n\004type\030\002 \001(\0132\032.flyteidl.core.Lit" - "eralType\022\031\n\021short_description\030\003 \001(\t\022+\n\ru" - "ser_metadata\030\004 \001(\0132\024.google.protobuf.Any" - "\022\025\n\rmetadata_type\030\005 \001(\t\022.\n\ncreated_at\030\006 " - "\001(\0132\032.google.protobuf.Timestamp\022\023\n\013file_" - "format\030\007 \001(\t\"G\n\026CreateArtifactResponse\022-" - "\n\010artifact\030\001 \001(\0132\033.flyteidl.artifact.Art" - "ifact\"R\n\022GetArtifactRequest\022+\n\005query\030\001 \001" - "(\0132\034.flyteidl.core.ArtifactQuery\022\017\n\007deta" - "ils\030\002 \001(\010\"D\n\023GetArtifactResponse\022-\n\010arti" - "fact\030\001 \001(\0132\033.flyteidl.artifact.Artifact\"" - "A\n\rSearchOptions\022\031\n\021strict_partitions\030\001 " - "\001(\010\022\025\n\rlatest_by_key\030\002 \001(\010\"\250\002\n\026SearchArt" - "ifactsRequest\0220\n\014artifact_key\030\001 \001(\0132\032.fl" - "yteidl.core.ArtifactKey\022-\n\npartitions\030\002 " - "\001(\0132\031.flyteidl.core.Partitions\0228\n\024time_p" - "artition_value\030\003 \001(\0132\032.google.protobuf.T" - "imestamp\022\021\n\tprincipal\030\004 \001(\t\022\017\n\007version\030\005" - " \001(\t\0221\n\007options\030\006 \001(\0132 .flyteidl.artifac" - "t.SearchOptions\022\r\n\005token\030\007 \001(\t\022\r\n\005limit\030" - "\010 \001(\005\"X\n\027SearchArtifactsResponse\022.\n\tarti" - "facts\030\001 \003(\0132\033.flyteidl.artifact.Artifact" - "\022\r\n\005token\030\002 \001(\t\"\311\001\n\031FindByWorkflowExecRe" - "quest\022;\n\007exec_id\030\001 \001(\0132*.flyteidl.core.W" - "orkflowExecutionIdentifier\022I\n\tdirection\030" - "\002 \001(\01626.flyteidl.artifact.FindByWorkflow" - "ExecRequest.Direction\"$\n\tDirection\022\n\n\006IN" - "PUTS\020\000\022\013\n\007OUTPUTS\020\001\"a\n\rAddTagRequest\022.\n\013" - "artifact_id\030\001 \001(\0132\031.flyteidl.core.Artifa" - "ctID\022\r\n\005value\030\002 \001(\t\022\021\n\toverwrite\030\003 \001(\010\"\020" - "\n\016AddTagResponse\"O\n\024CreateTriggerRequest" - "\0227\n\023trigger_launch_plan\030\001 \001(\0132\032.flyteidl" - ".admin.LaunchPlan\"\027\n\025CreateTriggerRespon" - "se\"I\n\030DeactivateTriggerRequest\022-\n\ntrigge" - "r_id\030\001 \001(\0132\031.flyteidl.core.Identifier\"\033\n" - "\031DeactivateTriggerResponse\"m\n\020ArtifactPr" - "oducer\022,\n\tentity_id\030\001 \001(\0132\031.flyteidl.cor" - "e.Identifier\022+\n\007outputs\030\002 \001(\0132\032.flyteidl" - ".core.VariableMap\"Q\n\027RegisterProducerReq" - "uest\0226\n\tproducers\030\001 \003(\0132#.flyteidl.artif" - "act.ArtifactProducer\"m\n\020ArtifactConsumer" - "\022,\n\tentity_id\030\001 \001(\0132\031.flyteidl.core.Iden" - "tifier\022+\n\006inputs\030\002 \001(\0132\033.flyteidl.core.P" - "arameterMap\"Q\n\027RegisterConsumerRequest\0226" - "\n\tconsumers\030\001 \003(\0132#.flyteidl.artifact.Ar" - "tifactConsumer\"\022\n\020RegisterResponse\"\205\001\n\026E" - "xecutionInputsRequest\022@\n\014execution_id\030\001 " - "\001(\0132*.flyteidl.core.WorkflowExecutionIde" - "ntifier\022)\n\006inputs\030\002 \003(\0132\031.flyteidl.core." - "ArtifactID\"\031\n\027ExecutionInputsResponse\"B\n" - "\020ListUsageRequest\022.\n\013artifact_id\030\001 \001(\0132\031" - ".flyteidl.core.ArtifactID\"S\n\021ListUsageRe" - "sponse\022>\n\nexecutions\030\001 \003(\0132*.flyteidl.co" - "re.WorkflowExecutionIdentifier2\373\013\n\020Artif" - "actRegistry\022g\n\016CreateArtifact\022(.flyteidl" - ".artifact.CreateArtifactRequest\032).flytei" - "dl.artifact.CreateArtifactResponse\"\000\022\204\001\n" - "\013GetArtifact\022%.flyteidl.artifact.GetArti" - "factRequest\032&.flyteidl.artifact.GetArtif" - "actResponse\"&\202\323\344\223\002 \"\033/artifacts/api/v1/a" - "rtifacts:\001*\022\215\001\n\017SearchArtifacts\022).flytei" - "dl.artifact.SearchArtifactsRequest\032*.fly" - "teidl.artifact.SearchArtifactsResponse\"#" - "\202\323\344\223\002\035\"\030/artifacts/api/v1/search:\001*\022d\n\rC" - "reateTrigger\022\'.flyteidl.artifact.CreateT" - "riggerRequest\032(.flyteidl.artifact.Create" - "TriggerResponse\"\000\022\237\001\n\021DeactivateTrigger\022" - "+.flyteidl.artifact.DeactivateTriggerReq" - "uest\032,.flyteidl.artifact.DeactivateTrigg" - "erResponse\"/\202\323\344\223\002)2$/artifacts/api/v1/tr" - "igger/deactivate:\001*\022O\n\006AddTag\022 .flyteidl" - ".artifact.AddTagRequest\032!.flyteidl.artif" - "act.AddTagResponse\"\000\022e\n\020RegisterProducer" - "\022*.flyteidl.artifact.RegisterProducerReq" - "uest\032#.flyteidl.artifact.RegisterRespons" - "e\"\000\022e\n\020RegisterConsumer\022*.flyteidl.artif" - "act.RegisterConsumerRequest\032#.flyteidl.a" - "rtifact.RegisterResponse\"\000\022m\n\022SetExecuti" - "onInputs\022).flyteidl.artifact.ExecutionIn" - "putsRequest\032*.flyteidl.artifact.Executio" - "nInputsResponse\"\000\022\330\001\n\022FindByWorkflowExec" - "\022,.flyteidl.artifact.FindByWorkflowExecR" - "equest\032*.flyteidl.artifact.SearchArtifac" - "tsResponse\"h\202\323\344\223\002b\022`/artifacts/api/v1/se" - "arch/execution/{exec_id.project}/{exec_i" - "d.domain}/{exec_id.name}/{direction}\022\365\001\n" - "\tListUsage\022#.flyteidl.artifact.ListUsage" - "Request\032$.flyteidl.artifact.ListUsageRes" - "ponse\"\234\001\202\323\344\223\002\225\001\022\222\001/artifacts/api/v1/usag" - "e/{artifact_id.artifact_key.project}/{ar" - "tifact_id.artifact_key.domain}/{artifact" - "_id.artifact_key.name}/{artifact_id.vers" - "ion}B@Z>github.com/flyteorg/flyte/flytei" - "dl/gen/pb-go/flyteidl/artifactb\006proto3" - ; -::google::protobuf::internal::DescriptorTable descriptor_table_flyteidl_2fartifact_2fartifacts_2eproto = { - false, InitDefaults_flyteidl_2fartifact_2fartifacts_2eproto, - descriptor_table_protodef_flyteidl_2fartifact_2fartifacts_2eproto, - "flyteidl/artifact/artifacts.proto", &assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto, 4918, -}; - -void AddDescriptors_flyteidl_2fartifact_2fartifacts_2eproto() { - static constexpr ::google::protobuf::internal::InitFunc deps[10] = - { - ::AddDescriptors_google_2fprotobuf_2fany_2eproto, - ::AddDescriptors_google_2fapi_2fannotations_2eproto, - ::AddDescriptors_google_2fprotobuf_2ftimestamp_2eproto, - ::AddDescriptors_flyteidl_2fadmin_2flaunch_5fplan_2eproto, - ::AddDescriptors_flyteidl_2fcore_2fliterals_2eproto, - ::AddDescriptors_flyteidl_2fcore_2ftypes_2eproto, - ::AddDescriptors_flyteidl_2fcore_2fidentifier_2eproto, - ::AddDescriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, - ::AddDescriptors_flyteidl_2fcore_2finterface_2eproto, - ::AddDescriptors_flyteidl_2fevent_2fcloudevents_2eproto, - }; - ::google::protobuf::internal::AddDescriptors(&descriptor_table_flyteidl_2fartifact_2fartifacts_2eproto, deps, 10); -} - -// Force running AddDescriptors() at dynamic initialization time. -static bool dynamic_init_dummy_flyteidl_2fartifact_2fartifacts_2eproto = []() { AddDescriptors_flyteidl_2fartifact_2fartifacts_2eproto(); return true; }(); -namespace flyteidl { -namespace artifact { -const ::google::protobuf::EnumDescriptor* FindByWorkflowExecRequest_Direction_descriptor() { - ::google::protobuf::internal::AssignDescriptors(&assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return file_level_enum_descriptors_flyteidl_2fartifact_2fartifacts_2eproto[0]; -} -bool FindByWorkflowExecRequest_Direction_IsValid(int value) { - switch (value) { - case 0: - case 1: - return true; - default: - return false; - } -} - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest::INPUTS; -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest::OUTPUTS; -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest::Direction_MIN; -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest::Direction_MAX; -const int FindByWorkflowExecRequest::Direction_ARRAYSIZE; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -// =================================================================== - -void Artifact::InitAsDefaultInstance() { - ::flyteidl::artifact::_Artifact_default_instance_._instance.get_mutable()->artifact_id_ = const_cast< ::flyteidl::core::ArtifactID*>( - ::flyteidl::core::ArtifactID::internal_default_instance()); - ::flyteidl::artifact::_Artifact_default_instance_._instance.get_mutable()->spec_ = const_cast< ::flyteidl::artifact::ArtifactSpec*>( - ::flyteidl::artifact::ArtifactSpec::internal_default_instance()); - ::flyteidl::artifact::_Artifact_default_instance_._instance.get_mutable()->source_ = const_cast< ::flyteidl::artifact::ArtifactSource*>( - ::flyteidl::artifact::ArtifactSource::internal_default_instance()); -} -class Artifact::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactID& artifact_id(const Artifact* msg); - static const ::flyteidl::artifact::ArtifactSpec& spec(const Artifact* msg); - static const ::flyteidl::artifact::ArtifactSource& source(const Artifact* msg); -}; - -const ::flyteidl::core::ArtifactID& -Artifact::HasBitSetters::artifact_id(const Artifact* msg) { - return *msg->artifact_id_; -} -const ::flyteidl::artifact::ArtifactSpec& -Artifact::HasBitSetters::spec(const Artifact* msg) { - return *msg->spec_; -} -const ::flyteidl::artifact::ArtifactSource& -Artifact::HasBitSetters::source(const Artifact* msg) { - return *msg->source_; -} -void Artifact::clear_artifact_id() { - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int Artifact::kArtifactIdFieldNumber; -const int Artifact::kSpecFieldNumber; -const int Artifact::kTagsFieldNumber; -const int Artifact::kSourceFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -Artifact::Artifact() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.Artifact) -} -Artifact::Artifact(const Artifact& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - tags_(from.tags_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_artifact_id()) { - artifact_id_ = new ::flyteidl::core::ArtifactID(*from.artifact_id_); - } else { - artifact_id_ = nullptr; - } - if (from.has_spec()) { - spec_ = new ::flyteidl::artifact::ArtifactSpec(*from.spec_); - } else { - spec_ = nullptr; - } - if (from.has_source()) { - source_ = new ::flyteidl::artifact::ArtifactSource(*from.source_); - } else { - source_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.Artifact) -} - -void Artifact::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::memset(&artifact_id_, 0, static_cast( - reinterpret_cast(&source_) - - reinterpret_cast(&artifact_id_)) + sizeof(source_)); -} - -Artifact::~Artifact() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.Artifact) - SharedDtor(); -} - -void Artifact::SharedDtor() { - if (this != internal_default_instance()) delete artifact_id_; - if (this != internal_default_instance()) delete spec_; - if (this != internal_default_instance()) delete source_; -} - -void Artifact::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const Artifact& Artifact::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_Artifact_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void Artifact::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.Artifact) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - tags_.Clear(); - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; - if (GetArenaNoVirtual() == nullptr && spec_ != nullptr) { - delete spec_; - } - spec_ = nullptr; - if (GetArenaNoVirtual() == nullptr && source_ != nullptr) { - delete source_; - } - source_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* Artifact::_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.ArtifactID artifact_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::ArtifactID::_InternalParse; - object = msg->mutable_artifact_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; - } - // .flyteidl.artifact.ArtifactSpec spec = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactSpec::_InternalParse; - object = msg->mutable_spec(); - 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; - } - // repeated string tags = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 26) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.Artifact.tags"); - object = msg->add_tags(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 26 && (ptr += 1)); - break; - } - // .flyteidl.artifact.ArtifactSource source = 4; - case 4: { - if (static_cast<::google::protobuf::uint8>(tag) != 34) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactSource::_InternalParse; - object = msg->mutable_source(); - 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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool Artifact::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.artifact.Artifact) - 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.ArtifactID artifact_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact_id())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_spec())); - } else { - goto handle_unusual; - } - break; - } - - // repeated string tags = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (26 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->add_tags())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->tags(this->tags_size() - 1).data(), - static_cast(this->tags(this->tags_size() - 1).length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.Artifact.tags")); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.ArtifactSource source = 4; - case 4: { - if (static_cast< ::google::protobuf::uint8>(tag) == (34 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_source())); - } 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.artifact.Artifact) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.Artifact) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void Artifact::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.Artifact) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact_id(this), output); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::spec(this), output); - } - - // repeated string tags = 3; - for (int i = 0, n = this->tags_size(); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->tags(i).data(), static_cast(this->tags(i).length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.Artifact.tags"); - ::google::protobuf::internal::WireFormatLite::WriteString( - 3, this->tags(i), output); - } - - // .flyteidl.artifact.ArtifactSource source = 4; - if (this->has_source()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 4, HasBitSetters::source(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.Artifact) -} - -::google::protobuf::uint8* Artifact::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.Artifact) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact_id(this), target); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::spec(this), target); - } - - // repeated string tags = 3; - for (int i = 0, n = this->tags_size(); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->tags(i).data(), static_cast(this->tags(i).length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.Artifact.tags"); - target = ::google::protobuf::internal::WireFormatLite:: - WriteStringToArray(3, this->tags(i), target); - } - - // .flyteidl.artifact.ArtifactSource source = 4; - if (this->has_source()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 4, HasBitSetters::source(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.artifact.Artifact) - return target; -} - -size_t Artifact::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.Artifact) - 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; - - // repeated string tags = 3; - total_size += 1 * - ::google::protobuf::internal::FromIntSize(this->tags_size()); - for (int i = 0, n = this->tags_size(); i < n; i++) { - total_size += ::google::protobuf::internal::WireFormatLite::StringSize( - this->tags(i)); - } - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_id_); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *spec_); - } - - // .flyteidl.artifact.ArtifactSource source = 4; - if (this->has_source()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *source_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void Artifact::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.Artifact) - GOOGLE_DCHECK_NE(&from, this); - const Artifact* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.Artifact) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.Artifact) - MergeFrom(*source); - } -} - -void Artifact::MergeFrom(const Artifact& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.Artifact) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - tags_.MergeFrom(from.tags_); - if (from.has_artifact_id()) { - mutable_artifact_id()->::flyteidl::core::ArtifactID::MergeFrom(from.artifact_id()); - } - if (from.has_spec()) { - mutable_spec()->::flyteidl::artifact::ArtifactSpec::MergeFrom(from.spec()); - } - if (from.has_source()) { - mutable_source()->::flyteidl::artifact::ArtifactSource::MergeFrom(from.source()); - } -} - -void Artifact::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.Artifact) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void Artifact::CopyFrom(const Artifact& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.Artifact) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool Artifact::IsInitialized() const { - return true; -} - -void Artifact::Swap(Artifact* other) { - if (other == this) return; - InternalSwap(other); -} -void Artifact::InternalSwap(Artifact* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - tags_.InternalSwap(CastToBase(&other->tags_)); - swap(artifact_id_, other->artifact_id_); - swap(spec_, other->spec_); - swap(source_, other->source_); -} - -::google::protobuf::Metadata Artifact::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -CreateArtifactRequest_PartitionsEntry_DoNotUse::CreateArtifactRequest_PartitionsEntry_DoNotUse() {} -CreateArtifactRequest_PartitionsEntry_DoNotUse::CreateArtifactRequest_PartitionsEntry_DoNotUse(::google::protobuf::Arena* arena) - : SuperType(arena) {} -void CreateArtifactRequest_PartitionsEntry_DoNotUse::MergeFrom(const CreateArtifactRequest_PartitionsEntry_DoNotUse& other) { - MergeFromInternal(other); -} -::google::protobuf::Metadata CreateArtifactRequest_PartitionsEntry_DoNotUse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[1]; -} -void CreateArtifactRequest_PartitionsEntry_DoNotUse::MergeFrom( - const ::google::protobuf::Message& other) { - ::google::protobuf::Message::MergeFrom(other); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool CreateArtifactRequest_PartitionsEntry_DoNotUse::_ParseMap(const char* begin, const char* end, void* object, ::google::protobuf::internal::ParseContext* ctx) { - using MF = ::google::protobuf::internal::MapField< - CreateArtifactRequest_PartitionsEntry_DoNotUse, EntryKeyType, EntryValueType, - kEntryKeyFieldType, kEntryValueFieldType, - kEntryDefaultEnumValue>; - auto mf = static_cast(object); - Parser> parser(mf); -#define DO_(x) if (!(x)) return false - DO_(parser.ParseMap(begin, end)); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - parser.key().data(), static_cast(parser.key().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.key")); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - parser.value().data(), static_cast(parser.value().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.value")); -#undef DO_ - return true; -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - - -// =================================================================== - -void CreateArtifactRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_CreateArtifactRequest_default_instance_._instance.get_mutable()->artifact_key_ = const_cast< ::flyteidl::core::ArtifactKey*>( - ::flyteidl::core::ArtifactKey::internal_default_instance()); - ::flyteidl::artifact::_CreateArtifactRequest_default_instance_._instance.get_mutable()->spec_ = const_cast< ::flyteidl::artifact::ArtifactSpec*>( - ::flyteidl::artifact::ArtifactSpec::internal_default_instance()); - ::flyteidl::artifact::_CreateArtifactRequest_default_instance_._instance.get_mutable()->time_partition_value_ = const_cast< ::google::protobuf::Timestamp*>( - ::google::protobuf::Timestamp::internal_default_instance()); - ::flyteidl::artifact::_CreateArtifactRequest_default_instance_._instance.get_mutable()->source_ = const_cast< ::flyteidl::artifact::ArtifactSource*>( - ::flyteidl::artifact::ArtifactSource::internal_default_instance()); -} -class CreateArtifactRequest::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactKey& artifact_key(const CreateArtifactRequest* msg); - static const ::flyteidl::artifact::ArtifactSpec& spec(const CreateArtifactRequest* msg); - static const ::google::protobuf::Timestamp& time_partition_value(const CreateArtifactRequest* msg); - static const ::flyteidl::artifact::ArtifactSource& source(const CreateArtifactRequest* msg); -}; - -const ::flyteidl::core::ArtifactKey& -CreateArtifactRequest::HasBitSetters::artifact_key(const CreateArtifactRequest* msg) { - return *msg->artifact_key_; -} -const ::flyteidl::artifact::ArtifactSpec& -CreateArtifactRequest::HasBitSetters::spec(const CreateArtifactRequest* msg) { - return *msg->spec_; -} -const ::google::protobuf::Timestamp& -CreateArtifactRequest::HasBitSetters::time_partition_value(const CreateArtifactRequest* msg) { - return *msg->time_partition_value_; -} -const ::flyteidl::artifact::ArtifactSource& -CreateArtifactRequest::HasBitSetters::source(const CreateArtifactRequest* msg) { - return *msg->source_; -} -void CreateArtifactRequest::clear_artifact_key() { - if (GetArenaNoVirtual() == nullptr && artifact_key_ != nullptr) { - delete artifact_key_; - } - artifact_key_ = nullptr; -} -void CreateArtifactRequest::clear_time_partition_value() { - if (GetArenaNoVirtual() == nullptr && time_partition_value_ != nullptr) { - delete time_partition_value_; - } - time_partition_value_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int CreateArtifactRequest::kArtifactKeyFieldNumber; -const int CreateArtifactRequest::kVersionFieldNumber; -const int CreateArtifactRequest::kSpecFieldNumber; -const int CreateArtifactRequest::kPartitionsFieldNumber; -const int CreateArtifactRequest::kTimePartitionValueFieldNumber; -const int CreateArtifactRequest::kSourceFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -CreateArtifactRequest::CreateArtifactRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.CreateArtifactRequest) -} -CreateArtifactRequest::CreateArtifactRequest(const CreateArtifactRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - partitions_.MergeFrom(from.partitions_); - version_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.version().size() > 0) { - version_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.version_); - } - if (from.has_artifact_key()) { - artifact_key_ = new ::flyteidl::core::ArtifactKey(*from.artifact_key_); - } else { - artifact_key_ = nullptr; - } - if (from.has_spec()) { - spec_ = new ::flyteidl::artifact::ArtifactSpec(*from.spec_); - } else { - spec_ = nullptr; - } - if (from.has_time_partition_value()) { - time_partition_value_ = new ::google::protobuf::Timestamp(*from.time_partition_value_); - } else { - time_partition_value_ = nullptr; - } - if (from.has_source()) { - source_ = new ::flyteidl::artifact::ArtifactSource(*from.source_); - } else { - source_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.CreateArtifactRequest) -} - -void CreateArtifactRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_CreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - version_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - ::memset(&artifact_key_, 0, static_cast( - reinterpret_cast(&source_) - - reinterpret_cast(&artifact_key_)) + sizeof(source_)); -} - -CreateArtifactRequest::~CreateArtifactRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.CreateArtifactRequest) - SharedDtor(); -} - -void CreateArtifactRequest::SharedDtor() { - version_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (this != internal_default_instance()) delete artifact_key_; - if (this != internal_default_instance()) delete spec_; - if (this != internal_default_instance()) delete time_partition_value_; - if (this != internal_default_instance()) delete source_; -} - -void CreateArtifactRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const CreateArtifactRequest& CreateArtifactRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_CreateArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void CreateArtifactRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.CreateArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - partitions_.Clear(); - version_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (GetArenaNoVirtual() == nullptr && artifact_key_ != nullptr) { - delete artifact_key_; - } - artifact_key_ = nullptr; - if (GetArenaNoVirtual() == nullptr && spec_ != nullptr) { - delete spec_; - } - spec_ = nullptr; - if (GetArenaNoVirtual() == nullptr && time_partition_value_ != nullptr) { - delete time_partition_value_; - } - time_partition_value_ = nullptr; - if (GetArenaNoVirtual() == nullptr && source_ != nullptr) { - delete source_; - } - source_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* CreateArtifactRequest::_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.ArtifactKey artifact_key = 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::ArtifactKey::_InternalParse; - object = msg->mutable_artifact_key(); - 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; - } - // .flyteidl.artifact.ArtifactSpec spec = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactSpec::_InternalParse; - object = msg->mutable_spec(); - 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; - } - // string version = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 26) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.CreateArtifactRequest.version"); - object = msg->mutable_version(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // map partitions = 4; - case 4: { - if (static_cast<::google::protobuf::uint8>(tag) != 34) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::google::protobuf::internal::SlowMapEntryParser; - auto parse_map = ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse::_ParseMap; - ctx->extra_parse_data().payload.clear(); - ctx->extra_parse_data().parse_map = parse_map; - object = &msg->partitions_; - if (size > end - ptr) goto len_delim_till_end; - auto newend = ptr + size; - GOOGLE_PROTOBUF_PARSER_ASSERT(parse_map(ptr, newend, object, ctx)); - ptr = newend; - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 34 && (ptr += 1)); - break; - } - // .google.protobuf.Timestamp time_partition_value = 5; - case 5: { - if (static_cast<::google::protobuf::uint8>(tag) != 42) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::google::protobuf::Timestamp::_InternalParse; - object = msg->mutable_time_partition_value(); - 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; - } - // .flyteidl.artifact.ArtifactSource source = 6; - case 6: { - if (static_cast<::google::protobuf::uint8>(tag) != 50) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactSource::_InternalParse; - object = msg->mutable_source(); - 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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool CreateArtifactRequest::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.artifact.CreateArtifactRequest) - 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.ArtifactKey artifact_key = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact_key())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_spec())); - } else { - goto handle_unusual; - } - break; - } - - // string version = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (26 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_version())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.CreateArtifactRequest.version")); - } else { - goto handle_unusual; - } - break; - } - - // map partitions = 4; - case 4: { - if (static_cast< ::google::protobuf::uint8>(tag) == (34 & 0xFF)) { - CreateArtifactRequest_PartitionsEntry_DoNotUse::Parser< ::google::protobuf::internal::MapField< - CreateArtifactRequest_PartitionsEntry_DoNotUse, - ::std::string, ::std::string, - ::google::protobuf::internal::WireFormatLite::TYPE_STRING, - ::google::protobuf::internal::WireFormatLite::TYPE_STRING, - 0 >, - ::google::protobuf::Map< ::std::string, ::std::string > > parser(&partitions_); - DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual( - input, &parser)); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - parser.key().data(), static_cast(parser.key().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.key")); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - parser.value().data(), static_cast(parser.value().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.value")); - } else { - goto handle_unusual; - } - break; - } - - // .google.protobuf.Timestamp time_partition_value = 5; - case 5: { - if (static_cast< ::google::protobuf::uint8>(tag) == (42 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_time_partition_value())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.ArtifactSource source = 6; - case 6: { - if (static_cast< ::google::protobuf::uint8>(tag) == (50 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_source())); - } 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.artifact.CreateArtifactRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.CreateArtifactRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void CreateArtifactRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.CreateArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact_key(this), output); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::spec(this), output); - } - - // string version = 3; - if (this->version().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.version"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 3, this->version(), output); - } - - // map partitions = 4; - if (!this->partitions().empty()) { - typedef ::google::protobuf::Map< ::std::string, ::std::string >::const_pointer - ConstPtr; - typedef ConstPtr SortItem; - typedef ::google::protobuf::internal::CompareByDerefFirst Less; - struct Utf8Check { - static void Check(ConstPtr p) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - p->first.data(), static_cast(p->first.length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.key"); - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - p->second.data(), static_cast(p->second.length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.value"); - } - }; - - if (output->IsSerializationDeterministic() && - this->partitions().size() > 1) { - ::std::unique_ptr items( - new SortItem[this->partitions().size()]); - typedef ::google::protobuf::Map< ::std::string, ::std::string >::size_type size_type; - size_type n = 0; - for (::google::protobuf::Map< ::std::string, ::std::string >::const_iterator - it = this->partitions().begin(); - it != this->partitions().end(); ++it, ++n) { - items[static_cast(n)] = SortItem(&*it); - } - ::std::sort(&items[0], &items[static_cast(n)], Less()); - ::std::unique_ptr entry; - for (size_type i = 0; i < n; i++) { - entry.reset(partitions_.NewEntryWrapper(items[static_cast(i)]->first, items[static_cast(i)]->second)); - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(4, *entry, output); - Utf8Check::Check(&(*items[static_cast(i)])); - } - } else { - ::std::unique_ptr entry; - for (::google::protobuf::Map< ::std::string, ::std::string >::const_iterator - it = this->partitions().begin(); - it != this->partitions().end(); ++it) { - entry.reset(partitions_.NewEntryWrapper(it->first, it->second)); - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(4, *entry, output); - Utf8Check::Check(&(*it)); - } - } - } - - // .google.protobuf.Timestamp time_partition_value = 5; - if (this->has_time_partition_value()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 5, HasBitSetters::time_partition_value(this), output); - } - - // .flyteidl.artifact.ArtifactSource source = 6; - if (this->has_source()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 6, HasBitSetters::source(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.CreateArtifactRequest) -} - -::google::protobuf::uint8* CreateArtifactRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.CreateArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact_key(this), target); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::spec(this), target); - } - - // string version = 3; - if (this->version().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.version"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 3, this->version(), target); - } - - // map partitions = 4; - if (!this->partitions().empty()) { - typedef ::google::protobuf::Map< ::std::string, ::std::string >::const_pointer - ConstPtr; - typedef ConstPtr SortItem; - typedef ::google::protobuf::internal::CompareByDerefFirst Less; - struct Utf8Check { - static void Check(ConstPtr p) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - p->first.data(), static_cast(p->first.length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.key"); - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - p->second.data(), static_cast(p->second.length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry.value"); - } - }; - - if (false && - this->partitions().size() > 1) { - ::std::unique_ptr items( - new SortItem[this->partitions().size()]); - typedef ::google::protobuf::Map< ::std::string, ::std::string >::size_type size_type; - size_type n = 0; - for (::google::protobuf::Map< ::std::string, ::std::string >::const_iterator - it = this->partitions().begin(); - it != this->partitions().end(); ++it, ++n) { - items[static_cast(n)] = SortItem(&*it); - } - ::std::sort(&items[0], &items[static_cast(n)], Less()); - ::std::unique_ptr entry; - for (size_type i = 0; i < n; i++) { - entry.reset(partitions_.NewEntryWrapper(items[static_cast(i)]->first, items[static_cast(i)]->second)); - target = ::google::protobuf::internal::WireFormatLite::InternalWriteMessageNoVirtualToArray(4, *entry, target); - Utf8Check::Check(&(*items[static_cast(i)])); - } - } else { - ::std::unique_ptr entry; - for (::google::protobuf::Map< ::std::string, ::std::string >::const_iterator - it = this->partitions().begin(); - it != this->partitions().end(); ++it) { - entry.reset(partitions_.NewEntryWrapper(it->first, it->second)); - target = ::google::protobuf::internal::WireFormatLite::InternalWriteMessageNoVirtualToArray(4, *entry, target); - Utf8Check::Check(&(*it)); - } - } - } - - // .google.protobuf.Timestamp time_partition_value = 5; - if (this->has_time_partition_value()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 5, HasBitSetters::time_partition_value(this), target); - } - - // .flyteidl.artifact.ArtifactSource source = 6; - if (this->has_source()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 6, HasBitSetters::source(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.artifact.CreateArtifactRequest) - return target; -} - -size_t CreateArtifactRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.CreateArtifactRequest) - 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; - - // map partitions = 4; - total_size += 1 * - ::google::protobuf::internal::FromIntSize(this->partitions_size()); - { - ::std::unique_ptr entry; - for (::google::protobuf::Map< ::std::string, ::std::string >::const_iterator - it = this->partitions().begin(); - it != this->partitions().end(); ++it) { - entry.reset(partitions_.NewEntryWrapper(it->first, it->second)); - total_size += ::google::protobuf::internal::WireFormatLite:: - MessageSizeNoVirtual(*entry); - } - } - - // string version = 3; - if (this->version().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->version()); - } - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_key_); - } - - // .flyteidl.artifact.ArtifactSpec spec = 2; - if (this->has_spec()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *spec_); - } - - // .google.protobuf.Timestamp time_partition_value = 5; - if (this->has_time_partition_value()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *time_partition_value_); - } - - // .flyteidl.artifact.ArtifactSource source = 6; - if (this->has_source()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *source_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void CreateArtifactRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.CreateArtifactRequest) - GOOGLE_DCHECK_NE(&from, this); - const CreateArtifactRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.CreateArtifactRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.CreateArtifactRequest) - MergeFrom(*source); - } -} - -void CreateArtifactRequest::MergeFrom(const CreateArtifactRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.CreateArtifactRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - partitions_.MergeFrom(from.partitions_); - if (from.version().size() > 0) { - - version_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.version_); - } - if (from.has_artifact_key()) { - mutable_artifact_key()->::flyteidl::core::ArtifactKey::MergeFrom(from.artifact_key()); - } - if (from.has_spec()) { - mutable_spec()->::flyteidl::artifact::ArtifactSpec::MergeFrom(from.spec()); - } - if (from.has_time_partition_value()) { - mutable_time_partition_value()->::google::protobuf::Timestamp::MergeFrom(from.time_partition_value()); - } - if (from.has_source()) { - mutable_source()->::flyteidl::artifact::ArtifactSource::MergeFrom(from.source()); - } -} - -void CreateArtifactRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.CreateArtifactRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void CreateArtifactRequest::CopyFrom(const CreateArtifactRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.CreateArtifactRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool CreateArtifactRequest::IsInitialized() const { - return true; -} - -void CreateArtifactRequest::Swap(CreateArtifactRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void CreateArtifactRequest::InternalSwap(CreateArtifactRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - partitions_.Swap(&other->partitions_); - version_.Swap(&other->version_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - swap(artifact_key_, other->artifact_key_); - swap(spec_, other->spec_); - swap(time_partition_value_, other->time_partition_value_); - swap(source_, other->source_); -} - -::google::protobuf::Metadata CreateArtifactRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ArtifactSource::InitAsDefaultInstance() { - ::flyteidl::artifact::_ArtifactSource_default_instance_._instance.get_mutable()->workflow_execution_ = const_cast< ::flyteidl::core::WorkflowExecutionIdentifier*>( - ::flyteidl::core::WorkflowExecutionIdentifier::internal_default_instance()); - ::flyteidl::artifact::_ArtifactSource_default_instance_._instance.get_mutable()->task_id_ = const_cast< ::flyteidl::core::Identifier*>( - ::flyteidl::core::Identifier::internal_default_instance()); -} -class ArtifactSource::HasBitSetters { - public: - static const ::flyteidl::core::WorkflowExecutionIdentifier& workflow_execution(const ArtifactSource* msg); - static const ::flyteidl::core::Identifier& task_id(const ArtifactSource* msg); -}; - -const ::flyteidl::core::WorkflowExecutionIdentifier& -ArtifactSource::HasBitSetters::workflow_execution(const ArtifactSource* msg) { - return *msg->workflow_execution_; -} -const ::flyteidl::core::Identifier& -ArtifactSource::HasBitSetters::task_id(const ArtifactSource* msg) { - return *msg->task_id_; -} -void ArtifactSource::clear_workflow_execution() { - if (GetArenaNoVirtual() == nullptr && workflow_execution_ != nullptr) { - delete workflow_execution_; - } - workflow_execution_ = nullptr; -} -void ArtifactSource::clear_task_id() { - if (GetArenaNoVirtual() == nullptr && task_id_ != nullptr) { - delete task_id_; - } - task_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ArtifactSource::kWorkflowExecutionFieldNumber; -const int ArtifactSource::kNodeIdFieldNumber; -const int ArtifactSource::kTaskIdFieldNumber; -const int ArtifactSource::kRetryAttemptFieldNumber; -const int ArtifactSource::kPrincipalFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ArtifactSource::ArtifactSource() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ArtifactSource) -} -ArtifactSource::ArtifactSource(const ArtifactSource& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - node_id_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.node_id().size() > 0) { - node_id_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.node_id_); - } - principal_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.principal().size() > 0) { - principal_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.principal_); - } - if (from.has_workflow_execution()) { - workflow_execution_ = new ::flyteidl::core::WorkflowExecutionIdentifier(*from.workflow_execution_); - } else { - workflow_execution_ = nullptr; - } - if (from.has_task_id()) { - task_id_ = new ::flyteidl::core::Identifier(*from.task_id_); - } else { - task_id_ = nullptr; - } - retry_attempt_ = from.retry_attempt_; - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ArtifactSource) -} - -void ArtifactSource::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto.base); - node_id_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - principal_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - ::memset(&workflow_execution_, 0, static_cast( - reinterpret_cast(&retry_attempt_) - - reinterpret_cast(&workflow_execution_)) + sizeof(retry_attempt_)); -} - -ArtifactSource::~ArtifactSource() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ArtifactSource) - SharedDtor(); -} - -void ArtifactSource::SharedDtor() { - node_id_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - principal_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (this != internal_default_instance()) delete workflow_execution_; - if (this != internal_default_instance()) delete task_id_; -} - -void ArtifactSource::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ArtifactSource& ArtifactSource::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ArtifactSource_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ArtifactSource::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ArtifactSource) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - node_id_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - principal_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (GetArenaNoVirtual() == nullptr && workflow_execution_ != nullptr) { - delete workflow_execution_; - } - workflow_execution_ = nullptr; - if (GetArenaNoVirtual() == nullptr && task_id_ != nullptr) { - delete task_id_; - } - task_id_ = nullptr; - retry_attempt_ = 0u; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ArtifactSource::_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.WorkflowExecutionIdentifier workflow_execution = 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::WorkflowExecutionIdentifier::_InternalParse; - object = msg->mutable_workflow_execution(); - 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; - } - // string node_id = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.ArtifactSource.node_id"); - object = msg->mutable_node_id(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // .flyteidl.core.Identifier task_id = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 26) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::Identifier::_InternalParse; - object = msg->mutable_task_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; - } - // uint32 retry_attempt = 4; - case 4: { - if (static_cast<::google::protobuf::uint8>(tag) != 32) goto handle_unusual; - msg->set_retry_attempt(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - break; - } - // string principal = 5; - case 5: { - if (static_cast<::google::protobuf::uint8>(tag) != 42) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.ArtifactSource.principal"); - object = msg->mutable_principal(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - 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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool ArtifactSource::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.artifact.ArtifactSource) - 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.WorkflowExecutionIdentifier workflow_execution = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_workflow_execution())); - } else { - goto handle_unusual; - } - break; - } - - // string node_id = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_node_id())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->node_id().data(), static_cast(this->node_id().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.ArtifactSource.node_id")); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.core.Identifier task_id = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (26 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_task_id())); - } else { - goto handle_unusual; - } - break; - } - - // uint32 retry_attempt = 4; - case 4: { - if (static_cast< ::google::protobuf::uint8>(tag) == (32 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - ::google::protobuf::uint32, ::google::protobuf::internal::WireFormatLite::TYPE_UINT32>( - input, &retry_attempt_))); - } else { - goto handle_unusual; - } - break; - } - - // string principal = 5; - case 5: { - if (static_cast< ::google::protobuf::uint8>(tag) == (42 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_principal())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.ArtifactSource.principal")); - } 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.artifact.ArtifactSource) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ArtifactSource) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ArtifactSource::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ArtifactSource) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - if (this->has_workflow_execution()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::workflow_execution(this), output); - } - - // string node_id = 2; - if (this->node_id().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->node_id().data(), static_cast(this->node_id().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSource.node_id"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 2, this->node_id(), output); - } - - // .flyteidl.core.Identifier task_id = 3; - if (this->has_task_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 3, HasBitSetters::task_id(this), output); - } - - // uint32 retry_attempt = 4; - if (this->retry_attempt() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteUInt32(4, this->retry_attempt(), output); - } - - // string principal = 5; - if (this->principal().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSource.principal"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 5, this->principal(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ArtifactSource) -} - -::google::protobuf::uint8* ArtifactSource::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ArtifactSource) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - if (this->has_workflow_execution()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::workflow_execution(this), target); - } - - // string node_id = 2; - if (this->node_id().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->node_id().data(), static_cast(this->node_id().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSource.node_id"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 2, this->node_id(), target); - } - - // .flyteidl.core.Identifier task_id = 3; - if (this->has_task_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 3, HasBitSetters::task_id(this), target); - } - - // uint32 retry_attempt = 4; - if (this->retry_attempt() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteUInt32ToArray(4, this->retry_attempt(), target); - } - - // string principal = 5; - if (this->principal().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSource.principal"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 5, this->principal(), 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.artifact.ArtifactSource) - return target; -} - -size_t ArtifactSource::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ArtifactSource) - 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; - - // string node_id = 2; - if (this->node_id().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->node_id()); - } - - // string principal = 5; - if (this->principal().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->principal()); - } - - // .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - if (this->has_workflow_execution()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *workflow_execution_); - } - - // .flyteidl.core.Identifier task_id = 3; - if (this->has_task_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *task_id_); - } - - // uint32 retry_attempt = 4; - if (this->retry_attempt() != 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::UInt32Size( - this->retry_attempt()); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ArtifactSource::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ArtifactSource) - GOOGLE_DCHECK_NE(&from, this); - const ArtifactSource* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ArtifactSource) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ArtifactSource) - MergeFrom(*source); - } -} - -void ArtifactSource::MergeFrom(const ArtifactSource& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ArtifactSource) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (from.node_id().size() > 0) { - - node_id_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.node_id_); - } - if (from.principal().size() > 0) { - - principal_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.principal_); - } - if (from.has_workflow_execution()) { - mutable_workflow_execution()->::flyteidl::core::WorkflowExecutionIdentifier::MergeFrom(from.workflow_execution()); - } - if (from.has_task_id()) { - mutable_task_id()->::flyteidl::core::Identifier::MergeFrom(from.task_id()); - } - if (from.retry_attempt() != 0) { - set_retry_attempt(from.retry_attempt()); - } -} - -void ArtifactSource::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ArtifactSource) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ArtifactSource::CopyFrom(const ArtifactSource& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ArtifactSource) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ArtifactSource::IsInitialized() const { - return true; -} - -void ArtifactSource::Swap(ArtifactSource* other) { - if (other == this) return; - InternalSwap(other); -} -void ArtifactSource::InternalSwap(ArtifactSource* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - node_id_.Swap(&other->node_id_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - principal_.Swap(&other->principal_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - swap(workflow_execution_, other->workflow_execution_); - swap(task_id_, other->task_id_); - swap(retry_attempt_, other->retry_attempt_); -} - -::google::protobuf::Metadata ArtifactSource::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ArtifactSpec::InitAsDefaultInstance() { - ::flyteidl::artifact::_ArtifactSpec_default_instance_._instance.get_mutable()->value_ = const_cast< ::flyteidl::core::Literal*>( - ::flyteidl::core::Literal::internal_default_instance()); - ::flyteidl::artifact::_ArtifactSpec_default_instance_._instance.get_mutable()->type_ = const_cast< ::flyteidl::core::LiteralType*>( - ::flyteidl::core::LiteralType::internal_default_instance()); - ::flyteidl::artifact::_ArtifactSpec_default_instance_._instance.get_mutable()->user_metadata_ = const_cast< ::google::protobuf::Any*>( - ::google::protobuf::Any::internal_default_instance()); - ::flyteidl::artifact::_ArtifactSpec_default_instance_._instance.get_mutable()->created_at_ = const_cast< ::google::protobuf::Timestamp*>( - ::google::protobuf::Timestamp::internal_default_instance()); -} -class ArtifactSpec::HasBitSetters { - public: - static const ::flyteidl::core::Literal& value(const ArtifactSpec* msg); - static const ::flyteidl::core::LiteralType& type(const ArtifactSpec* msg); - static const ::google::protobuf::Any& user_metadata(const ArtifactSpec* msg); - static const ::google::protobuf::Timestamp& created_at(const ArtifactSpec* msg); -}; - -const ::flyteidl::core::Literal& -ArtifactSpec::HasBitSetters::value(const ArtifactSpec* msg) { - return *msg->value_; -} -const ::flyteidl::core::LiteralType& -ArtifactSpec::HasBitSetters::type(const ArtifactSpec* msg) { - return *msg->type_; -} -const ::google::protobuf::Any& -ArtifactSpec::HasBitSetters::user_metadata(const ArtifactSpec* msg) { - return *msg->user_metadata_; -} -const ::google::protobuf::Timestamp& -ArtifactSpec::HasBitSetters::created_at(const ArtifactSpec* msg) { - return *msg->created_at_; -} -void ArtifactSpec::clear_value() { - if (GetArenaNoVirtual() == nullptr && value_ != nullptr) { - delete value_; - } - value_ = nullptr; -} -void ArtifactSpec::clear_type() { - if (GetArenaNoVirtual() == nullptr && type_ != nullptr) { - delete type_; - } - type_ = nullptr; -} -void ArtifactSpec::clear_user_metadata() { - if (GetArenaNoVirtual() == nullptr && user_metadata_ != nullptr) { - delete user_metadata_; - } - user_metadata_ = nullptr; -} -void ArtifactSpec::clear_created_at() { - if (GetArenaNoVirtual() == nullptr && created_at_ != nullptr) { - delete created_at_; - } - created_at_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ArtifactSpec::kValueFieldNumber; -const int ArtifactSpec::kTypeFieldNumber; -const int ArtifactSpec::kShortDescriptionFieldNumber; -const int ArtifactSpec::kUserMetadataFieldNumber; -const int ArtifactSpec::kMetadataTypeFieldNumber; -const int ArtifactSpec::kCreatedAtFieldNumber; -const int ArtifactSpec::kFileFormatFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ArtifactSpec::ArtifactSpec() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ArtifactSpec) -} -ArtifactSpec::ArtifactSpec(const ArtifactSpec& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - short_description_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.short_description().size() > 0) { - short_description_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.short_description_); - } - metadata_type_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.metadata_type().size() > 0) { - metadata_type_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.metadata_type_); - } - file_format_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.file_format().size() > 0) { - file_format_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.file_format_); - } - if (from.has_value()) { - value_ = new ::flyteidl::core::Literal(*from.value_); - } else { - value_ = nullptr; - } - if (from.has_type()) { - type_ = new ::flyteidl::core::LiteralType(*from.type_); - } else { - type_ = nullptr; - } - if (from.has_user_metadata()) { - user_metadata_ = new ::google::protobuf::Any(*from.user_metadata_); - } else { - user_metadata_ = nullptr; - } - if (from.has_created_at()) { - created_at_ = new ::google::protobuf::Timestamp(*from.created_at_); - } else { - created_at_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ArtifactSpec) -} - -void ArtifactSpec::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto.base); - short_description_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - metadata_type_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - file_format_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - ::memset(&value_, 0, static_cast( - reinterpret_cast(&created_at_) - - reinterpret_cast(&value_)) + sizeof(created_at_)); -} - -ArtifactSpec::~ArtifactSpec() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ArtifactSpec) - SharedDtor(); -} - -void ArtifactSpec::SharedDtor() { - short_description_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - metadata_type_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - file_format_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (this != internal_default_instance()) delete value_; - if (this != internal_default_instance()) delete type_; - if (this != internal_default_instance()) delete user_metadata_; - if (this != internal_default_instance()) delete created_at_; -} - -void ArtifactSpec::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ArtifactSpec& ArtifactSpec::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ArtifactSpec_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ArtifactSpec::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ArtifactSpec) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - short_description_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - metadata_type_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - file_format_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (GetArenaNoVirtual() == nullptr && value_ != nullptr) { - delete value_; - } - value_ = nullptr; - if (GetArenaNoVirtual() == nullptr && type_ != nullptr) { - delete type_; - } - type_ = nullptr; - if (GetArenaNoVirtual() == nullptr && user_metadata_ != nullptr) { - delete user_metadata_; - } - user_metadata_ = nullptr; - if (GetArenaNoVirtual() == nullptr && created_at_ != nullptr) { - delete created_at_; - } - created_at_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ArtifactSpec::_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.Literal value = 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::Literal::_InternalParse; - object = msg->mutable_value(); - 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; - } - // .flyteidl.core.LiteralType type = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::LiteralType::_InternalParse; - object = msg->mutable_type(); - 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; - } - // string short_description = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 26) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.ArtifactSpec.short_description"); - object = msg->mutable_short_description(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // .google.protobuf.Any user_metadata = 4; - case 4: { - if (static_cast<::google::protobuf::uint8>(tag) != 34) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::google::protobuf::Any::_InternalParse; - object = msg->mutable_user_metadata(); - 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; - } - // string metadata_type = 5; - case 5: { - if (static_cast<::google::protobuf::uint8>(tag) != 42) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.ArtifactSpec.metadata_type"); - object = msg->mutable_metadata_type(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // .google.protobuf.Timestamp created_at = 6; - case 6: { - if (static_cast<::google::protobuf::uint8>(tag) != 50) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::google::protobuf::Timestamp::_InternalParse; - object = msg->mutable_created_at(); - 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; - } - // string file_format = 7; - case 7: { - if (static_cast<::google::protobuf::uint8>(tag) != 58) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.ArtifactSpec.file_format"); - object = msg->mutable_file_format(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - 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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool ArtifactSpec::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.artifact.ArtifactSpec) - 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.Literal value = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_value())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.core.LiteralType type = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_type())); - } else { - goto handle_unusual; - } - break; - } - - // string short_description = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (26 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_short_description())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->short_description().data(), static_cast(this->short_description().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.ArtifactSpec.short_description")); - } else { - goto handle_unusual; - } - break; - } - - // .google.protobuf.Any user_metadata = 4; - case 4: { - if (static_cast< ::google::protobuf::uint8>(tag) == (34 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_user_metadata())); - } else { - goto handle_unusual; - } - break; - } - - // string metadata_type = 5; - case 5: { - if (static_cast< ::google::protobuf::uint8>(tag) == (42 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_metadata_type())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->metadata_type().data(), static_cast(this->metadata_type().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.ArtifactSpec.metadata_type")); - } else { - goto handle_unusual; - } - break; - } - - // .google.protobuf.Timestamp created_at = 6; - case 6: { - if (static_cast< ::google::protobuf::uint8>(tag) == (50 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_created_at())); - } else { - goto handle_unusual; - } - break; - } - - // string file_format = 7; - case 7: { - if (static_cast< ::google::protobuf::uint8>(tag) == (58 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_file_format())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->file_format().data(), static_cast(this->file_format().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.ArtifactSpec.file_format")); - } 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.artifact.ArtifactSpec) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ArtifactSpec) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ArtifactSpec::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ArtifactSpec) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Literal value = 1; - if (this->has_value()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::value(this), output); - } - - // .flyteidl.core.LiteralType type = 2; - if (this->has_type()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::type(this), output); - } - - // string short_description = 3; - if (this->short_description().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->short_description().data(), static_cast(this->short_description().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.short_description"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 3, this->short_description(), output); - } - - // .google.protobuf.Any user_metadata = 4; - if (this->has_user_metadata()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 4, HasBitSetters::user_metadata(this), output); - } - - // string metadata_type = 5; - if (this->metadata_type().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->metadata_type().data(), static_cast(this->metadata_type().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.metadata_type"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 5, this->metadata_type(), output); - } - - // .google.protobuf.Timestamp created_at = 6; - if (this->has_created_at()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 6, HasBitSetters::created_at(this), output); - } - - // string file_format = 7; - if (this->file_format().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->file_format().data(), static_cast(this->file_format().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.file_format"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 7, this->file_format(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ArtifactSpec) -} - -::google::protobuf::uint8* ArtifactSpec::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ArtifactSpec) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Literal value = 1; - if (this->has_value()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::value(this), target); - } - - // .flyteidl.core.LiteralType type = 2; - if (this->has_type()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::type(this), target); - } - - // string short_description = 3; - if (this->short_description().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->short_description().data(), static_cast(this->short_description().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.short_description"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 3, this->short_description(), target); - } - - // .google.protobuf.Any user_metadata = 4; - if (this->has_user_metadata()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 4, HasBitSetters::user_metadata(this), target); - } - - // string metadata_type = 5; - if (this->metadata_type().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->metadata_type().data(), static_cast(this->metadata_type().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.metadata_type"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 5, this->metadata_type(), target); - } - - // .google.protobuf.Timestamp created_at = 6; - if (this->has_created_at()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 6, HasBitSetters::created_at(this), target); - } - - // string file_format = 7; - if (this->file_format().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->file_format().data(), static_cast(this->file_format().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.ArtifactSpec.file_format"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 7, this->file_format(), 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.artifact.ArtifactSpec) - return target; -} - -size_t ArtifactSpec::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ArtifactSpec) - 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; - - // string short_description = 3; - if (this->short_description().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->short_description()); - } - - // string metadata_type = 5; - if (this->metadata_type().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->metadata_type()); - } - - // string file_format = 7; - if (this->file_format().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->file_format()); - } - - // .flyteidl.core.Literal value = 1; - if (this->has_value()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *value_); - } - - // .flyteidl.core.LiteralType type = 2; - if (this->has_type()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *type_); - } - - // .google.protobuf.Any user_metadata = 4; - if (this->has_user_metadata()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *user_metadata_); - } - - // .google.protobuf.Timestamp created_at = 6; - if (this->has_created_at()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *created_at_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ArtifactSpec::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ArtifactSpec) - GOOGLE_DCHECK_NE(&from, this); - const ArtifactSpec* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ArtifactSpec) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ArtifactSpec) - MergeFrom(*source); - } -} - -void ArtifactSpec::MergeFrom(const ArtifactSpec& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ArtifactSpec) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (from.short_description().size() > 0) { - - short_description_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.short_description_); - } - if (from.metadata_type().size() > 0) { - - metadata_type_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.metadata_type_); - } - if (from.file_format().size() > 0) { - - file_format_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.file_format_); - } - if (from.has_value()) { - mutable_value()->::flyteidl::core::Literal::MergeFrom(from.value()); - } - if (from.has_type()) { - mutable_type()->::flyteidl::core::LiteralType::MergeFrom(from.type()); - } - if (from.has_user_metadata()) { - mutable_user_metadata()->::google::protobuf::Any::MergeFrom(from.user_metadata()); - } - if (from.has_created_at()) { - mutable_created_at()->::google::protobuf::Timestamp::MergeFrom(from.created_at()); - } -} - -void ArtifactSpec::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ArtifactSpec) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ArtifactSpec::CopyFrom(const ArtifactSpec& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ArtifactSpec) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ArtifactSpec::IsInitialized() const { - return true; -} - -void ArtifactSpec::Swap(ArtifactSpec* other) { - if (other == this) return; - InternalSwap(other); -} -void ArtifactSpec::InternalSwap(ArtifactSpec* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - short_description_.Swap(&other->short_description_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - metadata_type_.Swap(&other->metadata_type_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - file_format_.Swap(&other->file_format_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - swap(value_, other->value_); - swap(type_, other->type_); - swap(user_metadata_, other->user_metadata_); - swap(created_at_, other->created_at_); -} - -::google::protobuf::Metadata ArtifactSpec::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void CreateArtifactResponse::InitAsDefaultInstance() { - ::flyteidl::artifact::_CreateArtifactResponse_default_instance_._instance.get_mutable()->artifact_ = const_cast< ::flyteidl::artifact::Artifact*>( - ::flyteidl::artifact::Artifact::internal_default_instance()); -} -class CreateArtifactResponse::HasBitSetters { - public: - static const ::flyteidl::artifact::Artifact& artifact(const CreateArtifactResponse* msg); -}; - -const ::flyteidl::artifact::Artifact& -CreateArtifactResponse::HasBitSetters::artifact(const CreateArtifactResponse* msg) { - return *msg->artifact_; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int CreateArtifactResponse::kArtifactFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -CreateArtifactResponse::CreateArtifactResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.CreateArtifactResponse) -} -CreateArtifactResponse::CreateArtifactResponse(const CreateArtifactResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_artifact()) { - artifact_ = new ::flyteidl::artifact::Artifact(*from.artifact_); - } else { - artifact_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.CreateArtifactResponse) -} - -void CreateArtifactResponse::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_CreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - artifact_ = nullptr; -} - -CreateArtifactResponse::~CreateArtifactResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.CreateArtifactResponse) - SharedDtor(); -} - -void CreateArtifactResponse::SharedDtor() { - if (this != internal_default_instance()) delete artifact_; -} - -void CreateArtifactResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const CreateArtifactResponse& CreateArtifactResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_CreateArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void CreateArtifactResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.CreateArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && artifact_ != nullptr) { - delete artifact_; - } - artifact_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* CreateArtifactResponse::_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.artifact.Artifact artifact = 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::artifact::Artifact::_InternalParse; - object = msg->mutable_artifact(); - 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 CreateArtifactResponse::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.artifact.CreateArtifactResponse) - 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.artifact.Artifact artifact = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact())); - } 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.artifact.CreateArtifactResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.CreateArtifactResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void CreateArtifactResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.CreateArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.CreateArtifactResponse) -} - -::google::protobuf::uint8* CreateArtifactResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.CreateArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact(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.artifact.CreateArtifactResponse) - return target; -} - -size_t CreateArtifactResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.CreateArtifactResponse) - 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.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void CreateArtifactResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.CreateArtifactResponse) - GOOGLE_DCHECK_NE(&from, this); - const CreateArtifactResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.CreateArtifactResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.CreateArtifactResponse) - MergeFrom(*source); - } -} - -void CreateArtifactResponse::MergeFrom(const CreateArtifactResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.CreateArtifactResponse) - 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_artifact()) { - mutable_artifact()->::flyteidl::artifact::Artifact::MergeFrom(from.artifact()); - } -} - -void CreateArtifactResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.CreateArtifactResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void CreateArtifactResponse::CopyFrom(const CreateArtifactResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.CreateArtifactResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool CreateArtifactResponse::IsInitialized() const { - return true; -} - -void CreateArtifactResponse::Swap(CreateArtifactResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void CreateArtifactResponse::InternalSwap(CreateArtifactResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(artifact_, other->artifact_); -} - -::google::protobuf::Metadata CreateArtifactResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void GetArtifactRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_GetArtifactRequest_default_instance_._instance.get_mutable()->query_ = const_cast< ::flyteidl::core::ArtifactQuery*>( - ::flyteidl::core::ArtifactQuery::internal_default_instance()); -} -class GetArtifactRequest::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactQuery& query(const GetArtifactRequest* msg); -}; - -const ::flyteidl::core::ArtifactQuery& -GetArtifactRequest::HasBitSetters::query(const GetArtifactRequest* msg) { - return *msg->query_; -} -void GetArtifactRequest::clear_query() { - if (GetArenaNoVirtual() == nullptr && query_ != nullptr) { - delete query_; - } - query_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int GetArtifactRequest::kQueryFieldNumber; -const int GetArtifactRequest::kDetailsFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -GetArtifactRequest::GetArtifactRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.GetArtifactRequest) -} -GetArtifactRequest::GetArtifactRequest(const GetArtifactRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_query()) { - query_ = new ::flyteidl::core::ArtifactQuery(*from.query_); - } else { - query_ = nullptr; - } - details_ = from.details_; - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.GetArtifactRequest) -} - -void GetArtifactRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_GetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::memset(&query_, 0, static_cast( - reinterpret_cast(&details_) - - reinterpret_cast(&query_)) + sizeof(details_)); -} - -GetArtifactRequest::~GetArtifactRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.GetArtifactRequest) - SharedDtor(); -} - -void GetArtifactRequest::SharedDtor() { - if (this != internal_default_instance()) delete query_; -} - -void GetArtifactRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const GetArtifactRequest& GetArtifactRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_GetArtifactRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void GetArtifactRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.GetArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && query_ != nullptr) { - delete query_; - } - query_ = nullptr; - details_ = false; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* GetArtifactRequest::_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.ArtifactQuery query = 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::ArtifactQuery::_InternalParse; - object = msg->mutable_query(); - 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; - } - // bool details = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 16) goto handle_unusual; - msg->set_details(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(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 GetArtifactRequest::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.artifact.GetArtifactRequest) - 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.ArtifactQuery query = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_query())); - } else { - goto handle_unusual; - } - break; - } - - // bool details = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (16 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &details_))); - } 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.artifact.GetArtifactRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.GetArtifactRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void GetArtifactRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.GetArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactQuery query = 1; - if (this->has_query()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::query(this), output); - } - - // bool details = 2; - if (this->details() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteBool(2, this->details(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.GetArtifactRequest) -} - -::google::protobuf::uint8* GetArtifactRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.GetArtifactRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactQuery query = 1; - if (this->has_query()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::query(this), target); - } - - // bool details = 2; - if (this->details() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(2, this->details(), 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.artifact.GetArtifactRequest) - return target; -} - -size_t GetArtifactRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.GetArtifactRequest) - 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.ArtifactQuery query = 1; - if (this->has_query()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *query_); - } - - // bool details = 2; - if (this->details() != 0) { - total_size += 1 + 1; - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void GetArtifactRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.GetArtifactRequest) - GOOGLE_DCHECK_NE(&from, this); - const GetArtifactRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.GetArtifactRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.GetArtifactRequest) - MergeFrom(*source); - } -} - -void GetArtifactRequest::MergeFrom(const GetArtifactRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.GetArtifactRequest) - 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_query()) { - mutable_query()->::flyteidl::core::ArtifactQuery::MergeFrom(from.query()); - } - if (from.details() != 0) { - set_details(from.details()); - } -} - -void GetArtifactRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.GetArtifactRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void GetArtifactRequest::CopyFrom(const GetArtifactRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.GetArtifactRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool GetArtifactRequest::IsInitialized() const { - return true; -} - -void GetArtifactRequest::Swap(GetArtifactRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void GetArtifactRequest::InternalSwap(GetArtifactRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(query_, other->query_); - swap(details_, other->details_); -} - -::google::protobuf::Metadata GetArtifactRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void GetArtifactResponse::InitAsDefaultInstance() { - ::flyteidl::artifact::_GetArtifactResponse_default_instance_._instance.get_mutable()->artifact_ = const_cast< ::flyteidl::artifact::Artifact*>( - ::flyteidl::artifact::Artifact::internal_default_instance()); -} -class GetArtifactResponse::HasBitSetters { - public: - static const ::flyteidl::artifact::Artifact& artifact(const GetArtifactResponse* msg); -}; - -const ::flyteidl::artifact::Artifact& -GetArtifactResponse::HasBitSetters::artifact(const GetArtifactResponse* msg) { - return *msg->artifact_; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int GetArtifactResponse::kArtifactFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -GetArtifactResponse::GetArtifactResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.GetArtifactResponse) -} -GetArtifactResponse::GetArtifactResponse(const GetArtifactResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_artifact()) { - artifact_ = new ::flyteidl::artifact::Artifact(*from.artifact_); - } else { - artifact_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.GetArtifactResponse) -} - -void GetArtifactResponse::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_GetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - artifact_ = nullptr; -} - -GetArtifactResponse::~GetArtifactResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.GetArtifactResponse) - SharedDtor(); -} - -void GetArtifactResponse::SharedDtor() { - if (this != internal_default_instance()) delete artifact_; -} - -void GetArtifactResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const GetArtifactResponse& GetArtifactResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_GetArtifactResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void GetArtifactResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.GetArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && artifact_ != nullptr) { - delete artifact_; - } - artifact_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* GetArtifactResponse::_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.artifact.Artifact artifact = 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::artifact::Artifact::_InternalParse; - object = msg->mutable_artifact(); - 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 GetArtifactResponse::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.artifact.GetArtifactResponse) - 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.artifact.Artifact artifact = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact())); - } 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.artifact.GetArtifactResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.GetArtifactResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void GetArtifactResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.GetArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.GetArtifactResponse) -} - -::google::protobuf::uint8* GetArtifactResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.GetArtifactResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact(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.artifact.GetArtifactResponse) - return target; -} - -size_t GetArtifactResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.GetArtifactResponse) - 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.artifact.Artifact artifact = 1; - if (this->has_artifact()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void GetArtifactResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.GetArtifactResponse) - GOOGLE_DCHECK_NE(&from, this); - const GetArtifactResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.GetArtifactResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.GetArtifactResponse) - MergeFrom(*source); - } -} - -void GetArtifactResponse::MergeFrom(const GetArtifactResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.GetArtifactResponse) - 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_artifact()) { - mutable_artifact()->::flyteidl::artifact::Artifact::MergeFrom(from.artifact()); - } -} - -void GetArtifactResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.GetArtifactResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void GetArtifactResponse::CopyFrom(const GetArtifactResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.GetArtifactResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool GetArtifactResponse::IsInitialized() const { - return true; -} - -void GetArtifactResponse::Swap(GetArtifactResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void GetArtifactResponse::InternalSwap(GetArtifactResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(artifact_, other->artifact_); -} - -::google::protobuf::Metadata GetArtifactResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void SearchOptions::InitAsDefaultInstance() { -} -class SearchOptions::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int SearchOptions::kStrictPartitionsFieldNumber; -const int SearchOptions::kLatestByKeyFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -SearchOptions::SearchOptions() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.SearchOptions) -} -SearchOptions::SearchOptions(const SearchOptions& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::memcpy(&strict_partitions_, &from.strict_partitions_, - static_cast(reinterpret_cast(&latest_by_key_) - - reinterpret_cast(&strict_partitions_)) + sizeof(latest_by_key_)); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.SearchOptions) -} - -void SearchOptions::SharedCtor() { - ::memset(&strict_partitions_, 0, static_cast( - reinterpret_cast(&latest_by_key_) - - reinterpret_cast(&strict_partitions_)) + sizeof(latest_by_key_)); -} - -SearchOptions::~SearchOptions() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.SearchOptions) - SharedDtor(); -} - -void SearchOptions::SharedDtor() { -} - -void SearchOptions::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const SearchOptions& SearchOptions::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_SearchOptions_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void SearchOptions::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.SearchOptions) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - ::memset(&strict_partitions_, 0, static_cast( - reinterpret_cast(&latest_by_key_) - - reinterpret_cast(&strict_partitions_)) + sizeof(latest_by_key_)); - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* SearchOptions::_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) { - // bool strict_partitions = 1; - case 1: { - if (static_cast<::google::protobuf::uint8>(tag) != 8) goto handle_unusual; - msg->set_strict_partitions(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - break; - } - // bool latest_by_key = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 16) goto handle_unusual; - msg->set_latest_by_key(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool SearchOptions::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.artifact.SearchOptions) - 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)) { - // bool strict_partitions = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (8 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &strict_partitions_))); - } else { - goto handle_unusual; - } - break; - } - - // bool latest_by_key = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (16 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &latest_by_key_))); - } 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.artifact.SearchOptions) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.SearchOptions) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void SearchOptions::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.SearchOptions) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // bool strict_partitions = 1; - if (this->strict_partitions() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteBool(1, this->strict_partitions(), output); - } - - // bool latest_by_key = 2; - if (this->latest_by_key() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteBool(2, this->latest_by_key(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.SearchOptions) -} - -::google::protobuf::uint8* SearchOptions::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.SearchOptions) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // bool strict_partitions = 1; - if (this->strict_partitions() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(1, this->strict_partitions(), target); - } - - // bool latest_by_key = 2; - if (this->latest_by_key() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(2, this->latest_by_key(), 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.artifact.SearchOptions) - return target; -} - -size_t SearchOptions::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.SearchOptions) - 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; - - // bool strict_partitions = 1; - if (this->strict_partitions() != 0) { - total_size += 1 + 1; - } - - // bool latest_by_key = 2; - if (this->latest_by_key() != 0) { - total_size += 1 + 1; - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void SearchOptions::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.SearchOptions) - GOOGLE_DCHECK_NE(&from, this); - const SearchOptions* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.SearchOptions) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.SearchOptions) - MergeFrom(*source); - } -} - -void SearchOptions::MergeFrom(const SearchOptions& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.SearchOptions) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (from.strict_partitions() != 0) { - set_strict_partitions(from.strict_partitions()); - } - if (from.latest_by_key() != 0) { - set_latest_by_key(from.latest_by_key()); - } -} - -void SearchOptions::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.SearchOptions) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void SearchOptions::CopyFrom(const SearchOptions& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.SearchOptions) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool SearchOptions::IsInitialized() const { - return true; -} - -void SearchOptions::Swap(SearchOptions* other) { - if (other == this) return; - InternalSwap(other); -} -void SearchOptions::InternalSwap(SearchOptions* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(strict_partitions_, other->strict_partitions_); - swap(latest_by_key_, other->latest_by_key_); -} - -::google::protobuf::Metadata SearchOptions::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void SearchArtifactsRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_SearchArtifactsRequest_default_instance_._instance.get_mutable()->artifact_key_ = const_cast< ::flyteidl::core::ArtifactKey*>( - ::flyteidl::core::ArtifactKey::internal_default_instance()); - ::flyteidl::artifact::_SearchArtifactsRequest_default_instance_._instance.get_mutable()->partitions_ = const_cast< ::flyteidl::core::Partitions*>( - ::flyteidl::core::Partitions::internal_default_instance()); - ::flyteidl::artifact::_SearchArtifactsRequest_default_instance_._instance.get_mutable()->time_partition_value_ = const_cast< ::google::protobuf::Timestamp*>( - ::google::protobuf::Timestamp::internal_default_instance()); - ::flyteidl::artifact::_SearchArtifactsRequest_default_instance_._instance.get_mutable()->options_ = const_cast< ::flyteidl::artifact::SearchOptions*>( - ::flyteidl::artifact::SearchOptions::internal_default_instance()); -} -class SearchArtifactsRequest::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactKey& artifact_key(const SearchArtifactsRequest* msg); - static const ::flyteidl::core::Partitions& partitions(const SearchArtifactsRequest* msg); - static const ::google::protobuf::Timestamp& time_partition_value(const SearchArtifactsRequest* msg); - static const ::flyteidl::artifact::SearchOptions& options(const SearchArtifactsRequest* msg); -}; - -const ::flyteidl::core::ArtifactKey& -SearchArtifactsRequest::HasBitSetters::artifact_key(const SearchArtifactsRequest* msg) { - return *msg->artifact_key_; -} -const ::flyteidl::core::Partitions& -SearchArtifactsRequest::HasBitSetters::partitions(const SearchArtifactsRequest* msg) { - return *msg->partitions_; -} -const ::google::protobuf::Timestamp& -SearchArtifactsRequest::HasBitSetters::time_partition_value(const SearchArtifactsRequest* msg) { - return *msg->time_partition_value_; -} -const ::flyteidl::artifact::SearchOptions& -SearchArtifactsRequest::HasBitSetters::options(const SearchArtifactsRequest* msg) { - return *msg->options_; -} -void SearchArtifactsRequest::clear_artifact_key() { - if (GetArenaNoVirtual() == nullptr && artifact_key_ != nullptr) { - delete artifact_key_; - } - artifact_key_ = nullptr; -} -void SearchArtifactsRequest::clear_partitions() { - if (GetArenaNoVirtual() == nullptr && partitions_ != nullptr) { - delete partitions_; - } - partitions_ = nullptr; -} -void SearchArtifactsRequest::clear_time_partition_value() { - if (GetArenaNoVirtual() == nullptr && time_partition_value_ != nullptr) { - delete time_partition_value_; - } - time_partition_value_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int SearchArtifactsRequest::kArtifactKeyFieldNumber; -const int SearchArtifactsRequest::kPartitionsFieldNumber; -const int SearchArtifactsRequest::kTimePartitionValueFieldNumber; -const int SearchArtifactsRequest::kPrincipalFieldNumber; -const int SearchArtifactsRequest::kVersionFieldNumber; -const int SearchArtifactsRequest::kOptionsFieldNumber; -const int SearchArtifactsRequest::kTokenFieldNumber; -const int SearchArtifactsRequest::kLimitFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -SearchArtifactsRequest::SearchArtifactsRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.SearchArtifactsRequest) -} -SearchArtifactsRequest::SearchArtifactsRequest(const SearchArtifactsRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - principal_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.principal().size() > 0) { - principal_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.principal_); - } - version_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.version().size() > 0) { - version_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.version_); - } - token_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.token().size() > 0) { - token_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.token_); - } - if (from.has_artifact_key()) { - artifact_key_ = new ::flyteidl::core::ArtifactKey(*from.artifact_key_); - } else { - artifact_key_ = nullptr; - } - if (from.has_partitions()) { - partitions_ = new ::flyteidl::core::Partitions(*from.partitions_); - } else { - partitions_ = nullptr; - } - if (from.has_time_partition_value()) { - time_partition_value_ = new ::google::protobuf::Timestamp(*from.time_partition_value_); - } else { - time_partition_value_ = nullptr; - } - if (from.has_options()) { - options_ = new ::flyteidl::artifact::SearchOptions(*from.options_); - } else { - options_ = nullptr; - } - limit_ = from.limit_; - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.SearchArtifactsRequest) -} - -void SearchArtifactsRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_SearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - principal_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - version_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - token_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - ::memset(&artifact_key_, 0, static_cast( - reinterpret_cast(&limit_) - - reinterpret_cast(&artifact_key_)) + sizeof(limit_)); -} - -SearchArtifactsRequest::~SearchArtifactsRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.SearchArtifactsRequest) - SharedDtor(); -} - -void SearchArtifactsRequest::SharedDtor() { - principal_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - version_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - token_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (this != internal_default_instance()) delete artifact_key_; - if (this != internal_default_instance()) delete partitions_; - if (this != internal_default_instance()) delete time_partition_value_; - if (this != internal_default_instance()) delete options_; -} - -void SearchArtifactsRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const SearchArtifactsRequest& SearchArtifactsRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_SearchArtifactsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void SearchArtifactsRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.SearchArtifactsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - principal_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - version_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - token_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (GetArenaNoVirtual() == nullptr && artifact_key_ != nullptr) { - delete artifact_key_; - } - artifact_key_ = nullptr; - if (GetArenaNoVirtual() == nullptr && partitions_ != nullptr) { - delete partitions_; - } - partitions_ = nullptr; - if (GetArenaNoVirtual() == nullptr && time_partition_value_ != nullptr) { - delete time_partition_value_; - } - time_partition_value_ = nullptr; - if (GetArenaNoVirtual() == nullptr && options_ != nullptr) { - delete options_; - } - options_ = nullptr; - limit_ = 0; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* SearchArtifactsRequest::_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.ArtifactKey artifact_key = 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::ArtifactKey::_InternalParse; - object = msg->mutable_artifact_key(); - 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; - } - // .flyteidl.core.Partitions partitions = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::Partitions::_InternalParse; - object = msg->mutable_partitions(); - 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; - } - // .google.protobuf.Timestamp time_partition_value = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 26) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::google::protobuf::Timestamp::_InternalParse; - object = msg->mutable_time_partition_value(); - 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; - } - // string principal = 4; - case 4: { - if (static_cast<::google::protobuf::uint8>(tag) != 34) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.SearchArtifactsRequest.principal"); - object = msg->mutable_principal(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // string version = 5; - case 5: { - if (static_cast<::google::protobuf::uint8>(tag) != 42) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.SearchArtifactsRequest.version"); - object = msg->mutable_version(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // .flyteidl.artifact.SearchOptions options = 6; - case 6: { - if (static_cast<::google::protobuf::uint8>(tag) != 50) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::SearchOptions::_InternalParse; - object = msg->mutable_options(); - 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; - } - // string token = 7; - case 7: { - if (static_cast<::google::protobuf::uint8>(tag) != 58) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.SearchArtifactsRequest.token"); - object = msg->mutable_token(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // int32 limit = 8; - case 8: { - if (static_cast<::google::protobuf::uint8>(tag) != 64) goto handle_unusual; - msg->set_limit(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool SearchArtifactsRequest::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.artifact.SearchArtifactsRequest) - 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.ArtifactKey artifact_key = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact_key())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.core.Partitions partitions = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_partitions())); - } else { - goto handle_unusual; - } - break; - } - - // .google.protobuf.Timestamp time_partition_value = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (26 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_time_partition_value())); - } else { - goto handle_unusual; - } - break; - } - - // string principal = 4; - case 4: { - if (static_cast< ::google::protobuf::uint8>(tag) == (34 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_principal())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.SearchArtifactsRequest.principal")); - } else { - goto handle_unusual; - } - break; - } - - // string version = 5; - case 5: { - if (static_cast< ::google::protobuf::uint8>(tag) == (42 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_version())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.SearchArtifactsRequest.version")); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.SearchOptions options = 6; - case 6: { - if (static_cast< ::google::protobuf::uint8>(tag) == (50 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_options())); - } else { - goto handle_unusual; - } - break; - } - - // string token = 7; - case 7: { - if (static_cast< ::google::protobuf::uint8>(tag) == (58 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_token())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.SearchArtifactsRequest.token")); - } else { - goto handle_unusual; - } - break; - } - - // int32 limit = 8; - case 8: { - if (static_cast< ::google::protobuf::uint8>(tag) == (64 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>( - input, &limit_))); - } 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.artifact.SearchArtifactsRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.SearchArtifactsRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void SearchArtifactsRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.SearchArtifactsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact_key(this), output); - } - - // .flyteidl.core.Partitions partitions = 2; - if (this->has_partitions()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::partitions(this), output); - } - - // .google.protobuf.Timestamp time_partition_value = 3; - if (this->has_time_partition_value()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 3, HasBitSetters::time_partition_value(this), output); - } - - // string principal = 4; - if (this->principal().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.principal"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 4, this->principal(), output); - } - - // string version = 5; - if (this->version().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.version"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 5, this->version(), output); - } - - // .flyteidl.artifact.SearchOptions options = 6; - if (this->has_options()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 6, HasBitSetters::options(this), output); - } - - // string token = 7; - if (this->token().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.token"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 7, this->token(), output); - } - - // int32 limit = 8; - if (this->limit() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteInt32(8, this->limit(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.SearchArtifactsRequest) -} - -::google::protobuf::uint8* SearchArtifactsRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.SearchArtifactsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact_key(this), target); - } - - // .flyteidl.core.Partitions partitions = 2; - if (this->has_partitions()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::partitions(this), target); - } - - // .google.protobuf.Timestamp time_partition_value = 3; - if (this->has_time_partition_value()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 3, HasBitSetters::time_partition_value(this), target); - } - - // string principal = 4; - if (this->principal().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->principal().data(), static_cast(this->principal().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.principal"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 4, this->principal(), target); - } - - // string version = 5; - if (this->version().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->version().data(), static_cast(this->version().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.version"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 5, this->version(), target); - } - - // .flyteidl.artifact.SearchOptions options = 6; - if (this->has_options()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 6, HasBitSetters::options(this), target); - } - - // string token = 7; - if (this->token().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsRequest.token"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 7, this->token(), target); - } - - // int32 limit = 8; - if (this->limit() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(8, this->limit(), 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.artifact.SearchArtifactsRequest) - return target; -} - -size_t SearchArtifactsRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.SearchArtifactsRequest) - 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; - - // string principal = 4; - if (this->principal().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->principal()); - } - - // string version = 5; - if (this->version().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->version()); - } - - // string token = 7; - if (this->token().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->token()); - } - - // .flyteidl.core.ArtifactKey artifact_key = 1; - if (this->has_artifact_key()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_key_); - } - - // .flyteidl.core.Partitions partitions = 2; - if (this->has_partitions()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *partitions_); - } - - // .google.protobuf.Timestamp time_partition_value = 3; - if (this->has_time_partition_value()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *time_partition_value_); - } - - // .flyteidl.artifact.SearchOptions options = 6; - if (this->has_options()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *options_); - } - - // int32 limit = 8; - if (this->limit() != 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::Int32Size( - this->limit()); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void SearchArtifactsRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.SearchArtifactsRequest) - GOOGLE_DCHECK_NE(&from, this); - const SearchArtifactsRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.SearchArtifactsRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.SearchArtifactsRequest) - MergeFrom(*source); - } -} - -void SearchArtifactsRequest::MergeFrom(const SearchArtifactsRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.SearchArtifactsRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (from.principal().size() > 0) { - - principal_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.principal_); - } - if (from.version().size() > 0) { - - version_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.version_); - } - if (from.token().size() > 0) { - - token_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.token_); - } - if (from.has_artifact_key()) { - mutable_artifact_key()->::flyteidl::core::ArtifactKey::MergeFrom(from.artifact_key()); - } - if (from.has_partitions()) { - mutable_partitions()->::flyteidl::core::Partitions::MergeFrom(from.partitions()); - } - if (from.has_time_partition_value()) { - mutable_time_partition_value()->::google::protobuf::Timestamp::MergeFrom(from.time_partition_value()); - } - if (from.has_options()) { - mutable_options()->::flyteidl::artifact::SearchOptions::MergeFrom(from.options()); - } - if (from.limit() != 0) { - set_limit(from.limit()); - } -} - -void SearchArtifactsRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.SearchArtifactsRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void SearchArtifactsRequest::CopyFrom(const SearchArtifactsRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.SearchArtifactsRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool SearchArtifactsRequest::IsInitialized() const { - return true; -} - -void SearchArtifactsRequest::Swap(SearchArtifactsRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void SearchArtifactsRequest::InternalSwap(SearchArtifactsRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - principal_.Swap(&other->principal_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - version_.Swap(&other->version_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - token_.Swap(&other->token_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - swap(artifact_key_, other->artifact_key_); - swap(partitions_, other->partitions_); - swap(time_partition_value_, other->time_partition_value_); - swap(options_, other->options_); - swap(limit_, other->limit_); -} - -::google::protobuf::Metadata SearchArtifactsRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void SearchArtifactsResponse::InitAsDefaultInstance() { -} -class SearchArtifactsResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int SearchArtifactsResponse::kArtifactsFieldNumber; -const int SearchArtifactsResponse::kTokenFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -SearchArtifactsResponse::SearchArtifactsResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.SearchArtifactsResponse) -} -SearchArtifactsResponse::SearchArtifactsResponse(const SearchArtifactsResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - artifacts_(from.artifacts_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - token_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.token().size() > 0) { - token_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.token_); - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.SearchArtifactsResponse) -} - -void SearchArtifactsResponse::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_SearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - token_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} - -SearchArtifactsResponse::~SearchArtifactsResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.SearchArtifactsResponse) - SharedDtor(); -} - -void SearchArtifactsResponse::SharedDtor() { - token_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} - -void SearchArtifactsResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const SearchArtifactsResponse& SearchArtifactsResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_SearchArtifactsResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void SearchArtifactsResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.SearchArtifactsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - artifacts_.Clear(); - token_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* SearchArtifactsResponse::_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) { - // repeated .flyteidl.artifact.Artifact artifacts = 1; - case 1: { - if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::Artifact::_InternalParse; - object = msg->add_artifacts(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 10 && (ptr += 1)); - break; - } - // string token = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.SearchArtifactsResponse.token"); - object = msg->mutable_token(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - 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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool SearchArtifactsResponse::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.artifact.SearchArtifactsResponse) - 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)) { - // repeated .flyteidl.artifact.Artifact artifacts = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_artifacts())); - } else { - goto handle_unusual; - } - break; - } - - // string token = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_token())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.SearchArtifactsResponse.token")); - } 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.artifact.SearchArtifactsResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.SearchArtifactsResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void SearchArtifactsResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.SearchArtifactsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.Artifact artifacts = 1; - for (unsigned int i = 0, - n = static_cast(this->artifacts_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, - this->artifacts(static_cast(i)), - output); - } - - // string token = 2; - if (this->token().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsResponse.token"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 2, this->token(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.SearchArtifactsResponse) -} - -::google::protobuf::uint8* SearchArtifactsResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.SearchArtifactsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.Artifact artifacts = 1; - for (unsigned int i = 0, - n = static_cast(this->artifacts_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, this->artifacts(static_cast(i)), target); - } - - // string token = 2; - if (this->token().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->token().data(), static_cast(this->token().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.SearchArtifactsResponse.token"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 2, this->token(), 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.artifact.SearchArtifactsResponse) - return target; -} - -size_t SearchArtifactsResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.SearchArtifactsResponse) - 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; - - // repeated .flyteidl.artifact.Artifact artifacts = 1; - { - unsigned int count = static_cast(this->artifacts_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->artifacts(static_cast(i))); - } - } - - // string token = 2; - if (this->token().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->token()); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void SearchArtifactsResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.SearchArtifactsResponse) - GOOGLE_DCHECK_NE(&from, this); - const SearchArtifactsResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.SearchArtifactsResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.SearchArtifactsResponse) - MergeFrom(*source); - } -} - -void SearchArtifactsResponse::MergeFrom(const SearchArtifactsResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.SearchArtifactsResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - artifacts_.MergeFrom(from.artifacts_); - if (from.token().size() > 0) { - - token_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.token_); - } -} - -void SearchArtifactsResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.SearchArtifactsResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void SearchArtifactsResponse::CopyFrom(const SearchArtifactsResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.SearchArtifactsResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool SearchArtifactsResponse::IsInitialized() const { - return true; -} - -void SearchArtifactsResponse::Swap(SearchArtifactsResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void SearchArtifactsResponse::InternalSwap(SearchArtifactsResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&artifacts_)->InternalSwap(CastToBase(&other->artifacts_)); - token_.Swap(&other->token_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); -} - -::google::protobuf::Metadata SearchArtifactsResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void FindByWorkflowExecRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_FindByWorkflowExecRequest_default_instance_._instance.get_mutable()->exec_id_ = const_cast< ::flyteidl::core::WorkflowExecutionIdentifier*>( - ::flyteidl::core::WorkflowExecutionIdentifier::internal_default_instance()); -} -class FindByWorkflowExecRequest::HasBitSetters { - public: - static const ::flyteidl::core::WorkflowExecutionIdentifier& exec_id(const FindByWorkflowExecRequest* msg); -}; - -const ::flyteidl::core::WorkflowExecutionIdentifier& -FindByWorkflowExecRequest::HasBitSetters::exec_id(const FindByWorkflowExecRequest* msg) { - return *msg->exec_id_; -} -void FindByWorkflowExecRequest::clear_exec_id() { - if (GetArenaNoVirtual() == nullptr && exec_id_ != nullptr) { - delete exec_id_; - } - exec_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int FindByWorkflowExecRequest::kExecIdFieldNumber; -const int FindByWorkflowExecRequest::kDirectionFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -FindByWorkflowExecRequest::FindByWorkflowExecRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.FindByWorkflowExecRequest) -} -FindByWorkflowExecRequest::FindByWorkflowExecRequest(const FindByWorkflowExecRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_exec_id()) { - exec_id_ = new ::flyteidl::core::WorkflowExecutionIdentifier(*from.exec_id_); - } else { - exec_id_ = nullptr; - } - direction_ = from.direction_; - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.FindByWorkflowExecRequest) -} - -void FindByWorkflowExecRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_FindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::memset(&exec_id_, 0, static_cast( - reinterpret_cast(&direction_) - - reinterpret_cast(&exec_id_)) + sizeof(direction_)); -} - -FindByWorkflowExecRequest::~FindByWorkflowExecRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.FindByWorkflowExecRequest) - SharedDtor(); -} - -void FindByWorkflowExecRequest::SharedDtor() { - if (this != internal_default_instance()) delete exec_id_; -} - -void FindByWorkflowExecRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const FindByWorkflowExecRequest& FindByWorkflowExecRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_FindByWorkflowExecRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void FindByWorkflowExecRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.FindByWorkflowExecRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && exec_id_ != nullptr) { - delete exec_id_; - } - exec_id_ = nullptr; - direction_ = 0; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* FindByWorkflowExecRequest::_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.WorkflowExecutionIdentifier exec_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::WorkflowExecutionIdentifier::_InternalParse; - object = msg->mutable_exec_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; - } - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 16) goto handle_unusual; - ::google::protobuf::uint64 val = ::google::protobuf::internal::ReadVarint(&ptr); - msg->set_direction(static_cast<::flyteidl::artifact::FindByWorkflowExecRequest_Direction>(val)); - GOOGLE_PROTOBUF_PARSER_ASSERT(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 FindByWorkflowExecRequest::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.artifact.FindByWorkflowExecRequest) - 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.WorkflowExecutionIdentifier exec_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_exec_id())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (16 & 0xFF)) { - int value = 0; - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - int, ::google::protobuf::internal::WireFormatLite::TYPE_ENUM>( - input, &value))); - set_direction(static_cast< ::flyteidl::artifact::FindByWorkflowExecRequest_Direction >(value)); - } 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.artifact.FindByWorkflowExecRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.FindByWorkflowExecRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void FindByWorkflowExecRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.FindByWorkflowExecRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - if (this->has_exec_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::exec_id(this), output); - } - - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - if (this->direction() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteEnum( - 2, this->direction(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.FindByWorkflowExecRequest) -} - -::google::protobuf::uint8* FindByWorkflowExecRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.FindByWorkflowExecRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - if (this->has_exec_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::exec_id(this), target); - } - - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - if (this->direction() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteEnumToArray( - 2, this->direction(), 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.artifact.FindByWorkflowExecRequest) - return target; -} - -size_t FindByWorkflowExecRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.FindByWorkflowExecRequest) - 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.WorkflowExecutionIdentifier exec_id = 1; - if (this->has_exec_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *exec_id_); - } - - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - if (this->direction() != 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::EnumSize(this->direction()); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void FindByWorkflowExecRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.FindByWorkflowExecRequest) - GOOGLE_DCHECK_NE(&from, this); - const FindByWorkflowExecRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.FindByWorkflowExecRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.FindByWorkflowExecRequest) - MergeFrom(*source); - } -} - -void FindByWorkflowExecRequest::MergeFrom(const FindByWorkflowExecRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.FindByWorkflowExecRequest) - 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_exec_id()) { - mutable_exec_id()->::flyteidl::core::WorkflowExecutionIdentifier::MergeFrom(from.exec_id()); - } - if (from.direction() != 0) { - set_direction(from.direction()); - } -} - -void FindByWorkflowExecRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.FindByWorkflowExecRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void FindByWorkflowExecRequest::CopyFrom(const FindByWorkflowExecRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.FindByWorkflowExecRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool FindByWorkflowExecRequest::IsInitialized() const { - return true; -} - -void FindByWorkflowExecRequest::Swap(FindByWorkflowExecRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void FindByWorkflowExecRequest::InternalSwap(FindByWorkflowExecRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(exec_id_, other->exec_id_); - swap(direction_, other->direction_); -} - -::google::protobuf::Metadata FindByWorkflowExecRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void AddTagRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_AddTagRequest_default_instance_._instance.get_mutable()->artifact_id_ = const_cast< ::flyteidl::core::ArtifactID*>( - ::flyteidl::core::ArtifactID::internal_default_instance()); -} -class AddTagRequest::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactID& artifact_id(const AddTagRequest* msg); -}; - -const ::flyteidl::core::ArtifactID& -AddTagRequest::HasBitSetters::artifact_id(const AddTagRequest* msg) { - return *msg->artifact_id_; -} -void AddTagRequest::clear_artifact_id() { - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int AddTagRequest::kArtifactIdFieldNumber; -const int AddTagRequest::kValueFieldNumber; -const int AddTagRequest::kOverwriteFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -AddTagRequest::AddTagRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.AddTagRequest) -} -AddTagRequest::AddTagRequest(const AddTagRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - value_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.value().size() > 0) { - value_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.value_); - } - if (from.has_artifact_id()) { - artifact_id_ = new ::flyteidl::core::ArtifactID(*from.artifact_id_); - } else { - artifact_id_ = nullptr; - } - overwrite_ = from.overwrite_; - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.AddTagRequest) -} - -void AddTagRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_AddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - value_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - ::memset(&artifact_id_, 0, static_cast( - reinterpret_cast(&overwrite_) - - reinterpret_cast(&artifact_id_)) + sizeof(overwrite_)); -} - -AddTagRequest::~AddTagRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.AddTagRequest) - SharedDtor(); -} - -void AddTagRequest::SharedDtor() { - value_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (this != internal_default_instance()) delete artifact_id_; -} - -void AddTagRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const AddTagRequest& AddTagRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_AddTagRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void AddTagRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.AddTagRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - value_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; - overwrite_ = false; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* AddTagRequest::_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.ArtifactID artifact_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::ArtifactID::_InternalParse; - object = msg->mutable_artifact_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; - } - // string value = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - ctx->extra_parse_data().SetFieldName("flyteidl.artifact.AddTagRequest.value"); - object = msg->mutable_value(); - if (size > end - ptr + ::google::protobuf::internal::ParseContext::kSlopBytes) { - parser_till_end = ::google::protobuf::internal::GreedyStringParserUTF8; - goto string_till_end; - } - GOOGLE_PROTOBUF_PARSER_ASSERT(::google::protobuf::internal::StringCheckUTF8(ptr, size, ctx)); - ::google::protobuf::internal::InlineGreedyStringParser(object, ptr, size, ctx); - ptr += size; - break; - } - // bool overwrite = 3; - case 3: { - if (static_cast<::google::protobuf::uint8>(tag) != 24) goto handle_unusual; - msg->set_overwrite(::google::protobuf::internal::ReadVarint(&ptr)); - GOOGLE_PROTOBUF_PARSER_ASSERT(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; -string_till_end: - static_cast<::std::string*>(object)->clear(); - static_cast<::std::string*>(object)->reserve(size); - goto len_delim_till_end; -len_delim_till_end: - return ctx->StoreAndTailCall(ptr, end, {_InternalParse, msg}, - {parser_till_end, object}, size); -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool AddTagRequest::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.artifact.AddTagRequest) - 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.ArtifactID artifact_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact_id())); - } else { - goto handle_unusual; - } - break; - } - - // string value = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_value())); - DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->value().data(), static_cast(this->value().length()), - ::google::protobuf::internal::WireFormatLite::PARSE, - "flyteidl.artifact.AddTagRequest.value")); - } else { - goto handle_unusual; - } - break; - } - - // bool overwrite = 3; - case 3: { - if (static_cast< ::google::protobuf::uint8>(tag) == (24 & 0xFF)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &overwrite_))); - } 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.artifact.AddTagRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.AddTagRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void AddTagRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.AddTagRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact_id(this), output); - } - - // string value = 2; - if (this->value().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->value().data(), static_cast(this->value().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.AddTagRequest.value"); - ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 2, this->value(), output); - } - - // bool overwrite = 3; - if (this->overwrite() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteBool(3, this->overwrite(), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.AddTagRequest) -} - -::google::protobuf::uint8* AddTagRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.AddTagRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact_id(this), target); - } - - // string value = 2; - if (this->value().size() > 0) { - ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( - this->value().data(), static_cast(this->value().length()), - ::google::protobuf::internal::WireFormatLite::SERIALIZE, - "flyteidl.artifact.AddTagRequest.value"); - target = - ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 2, this->value(), target); - } - - // bool overwrite = 3; - if (this->overwrite() != 0) { - target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(3, this->overwrite(), 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.artifact.AddTagRequest) - return target; -} - -size_t AddTagRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.AddTagRequest) - 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; - - // string value = 2; - if (this->value().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::StringSize( - this->value()); - } - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_id_); - } - - // bool overwrite = 3; - if (this->overwrite() != 0) { - total_size += 1 + 1; - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void AddTagRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.AddTagRequest) - GOOGLE_DCHECK_NE(&from, this); - const AddTagRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.AddTagRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.AddTagRequest) - MergeFrom(*source); - } -} - -void AddTagRequest::MergeFrom(const AddTagRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.AddTagRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (from.value().size() > 0) { - - value_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.value_); - } - if (from.has_artifact_id()) { - mutable_artifact_id()->::flyteidl::core::ArtifactID::MergeFrom(from.artifact_id()); - } - if (from.overwrite() != 0) { - set_overwrite(from.overwrite()); - } -} - -void AddTagRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.AddTagRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void AddTagRequest::CopyFrom(const AddTagRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.AddTagRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool AddTagRequest::IsInitialized() const { - return true; -} - -void AddTagRequest::Swap(AddTagRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void AddTagRequest::InternalSwap(AddTagRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - value_.Swap(&other->value_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); - swap(artifact_id_, other->artifact_id_); - swap(overwrite_, other->overwrite_); -} - -::google::protobuf::Metadata AddTagRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void AddTagResponse::InitAsDefaultInstance() { -} -class AddTagResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -AddTagResponse::AddTagResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.AddTagResponse) -} -AddTagResponse::AddTagResponse(const AddTagResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.AddTagResponse) -} - -void AddTagResponse::SharedCtor() { -} - -AddTagResponse::~AddTagResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.AddTagResponse) - SharedDtor(); -} - -void AddTagResponse::SharedDtor() { -} - -void AddTagResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const AddTagResponse& AddTagResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_AddTagResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void AddTagResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.AddTagResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* AddTagResponse::_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) { - default: { - 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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool AddTagResponse::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.artifact.AddTagResponse) - for (;;) { - ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); - tag = p.first; - if (!p.second) goto handle_unusual; - handle_unusual: - if (tag == 0) { - goto success; - } - DO_(::google::protobuf::internal::WireFormat::SkipField( - input, tag, _internal_metadata_.mutable_unknown_fields())); - } -success: - // @@protoc_insertion_point(parse_success:flyteidl.artifact.AddTagResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.AddTagResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void AddTagResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.AddTagResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.AddTagResponse) -} - -::google::protobuf::uint8* AddTagResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.AddTagResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - 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.artifact.AddTagResponse) - return target; -} - -size_t AddTagResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.AddTagResponse) - 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; - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void AddTagResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.AddTagResponse) - GOOGLE_DCHECK_NE(&from, this); - const AddTagResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.AddTagResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.AddTagResponse) - MergeFrom(*source); - } -} - -void AddTagResponse::MergeFrom(const AddTagResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.AddTagResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - -} - -void AddTagResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.AddTagResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void AddTagResponse::CopyFrom(const AddTagResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.AddTagResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool AddTagResponse::IsInitialized() const { - return true; -} - -void AddTagResponse::Swap(AddTagResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void AddTagResponse::InternalSwap(AddTagResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); -} - -::google::protobuf::Metadata AddTagResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void CreateTriggerRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_CreateTriggerRequest_default_instance_._instance.get_mutable()->trigger_launch_plan_ = const_cast< ::flyteidl::admin::LaunchPlan*>( - ::flyteidl::admin::LaunchPlan::internal_default_instance()); -} -class CreateTriggerRequest::HasBitSetters { - public: - static const ::flyteidl::admin::LaunchPlan& trigger_launch_plan(const CreateTriggerRequest* msg); -}; - -const ::flyteidl::admin::LaunchPlan& -CreateTriggerRequest::HasBitSetters::trigger_launch_plan(const CreateTriggerRequest* msg) { - return *msg->trigger_launch_plan_; -} -void CreateTriggerRequest::clear_trigger_launch_plan() { - if (GetArenaNoVirtual() == nullptr && trigger_launch_plan_ != nullptr) { - delete trigger_launch_plan_; - } - trigger_launch_plan_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int CreateTriggerRequest::kTriggerLaunchPlanFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -CreateTriggerRequest::CreateTriggerRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.CreateTriggerRequest) -} -CreateTriggerRequest::CreateTriggerRequest(const CreateTriggerRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_trigger_launch_plan()) { - trigger_launch_plan_ = new ::flyteidl::admin::LaunchPlan(*from.trigger_launch_plan_); - } else { - trigger_launch_plan_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.CreateTriggerRequest) -} - -void CreateTriggerRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_CreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - trigger_launch_plan_ = nullptr; -} - -CreateTriggerRequest::~CreateTriggerRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.CreateTriggerRequest) - SharedDtor(); -} - -void CreateTriggerRequest::SharedDtor() { - if (this != internal_default_instance()) delete trigger_launch_plan_; -} - -void CreateTriggerRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const CreateTriggerRequest& CreateTriggerRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_CreateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void CreateTriggerRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.CreateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && trigger_launch_plan_ != nullptr) { - delete trigger_launch_plan_; - } - trigger_launch_plan_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* CreateTriggerRequest::_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.admin.LaunchPlan trigger_launch_plan = 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::admin::LaunchPlan::_InternalParse; - object = msg->mutable_trigger_launch_plan(); - 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 CreateTriggerRequest::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.artifact.CreateTriggerRequest) - 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.admin.LaunchPlan trigger_launch_plan = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_trigger_launch_plan())); - } 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.artifact.CreateTriggerRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.CreateTriggerRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void CreateTriggerRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.CreateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - if (this->has_trigger_launch_plan()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::trigger_launch_plan(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.CreateTriggerRequest) -} - -::google::protobuf::uint8* CreateTriggerRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.CreateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - if (this->has_trigger_launch_plan()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::trigger_launch_plan(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.artifact.CreateTriggerRequest) - return target; -} - -size_t CreateTriggerRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.CreateTriggerRequest) - 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.admin.LaunchPlan trigger_launch_plan = 1; - if (this->has_trigger_launch_plan()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *trigger_launch_plan_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void CreateTriggerRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.CreateTriggerRequest) - GOOGLE_DCHECK_NE(&from, this); - const CreateTriggerRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.CreateTriggerRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.CreateTriggerRequest) - MergeFrom(*source); - } -} - -void CreateTriggerRequest::MergeFrom(const CreateTriggerRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.CreateTriggerRequest) - 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_trigger_launch_plan()) { - mutable_trigger_launch_plan()->::flyteidl::admin::LaunchPlan::MergeFrom(from.trigger_launch_plan()); - } -} - -void CreateTriggerRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.CreateTriggerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void CreateTriggerRequest::CopyFrom(const CreateTriggerRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.CreateTriggerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool CreateTriggerRequest::IsInitialized() const { - return true; -} - -void CreateTriggerRequest::Swap(CreateTriggerRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void CreateTriggerRequest::InternalSwap(CreateTriggerRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(trigger_launch_plan_, other->trigger_launch_plan_); -} - -::google::protobuf::Metadata CreateTriggerRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void CreateTriggerResponse::InitAsDefaultInstance() { -} -class CreateTriggerResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -CreateTriggerResponse::CreateTriggerResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.CreateTriggerResponse) -} -CreateTriggerResponse::CreateTriggerResponse(const CreateTriggerResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.CreateTriggerResponse) -} - -void CreateTriggerResponse::SharedCtor() { -} - -CreateTriggerResponse::~CreateTriggerResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.CreateTriggerResponse) - SharedDtor(); -} - -void CreateTriggerResponse::SharedDtor() { -} - -void CreateTriggerResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const CreateTriggerResponse& CreateTriggerResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_CreateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void CreateTriggerResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.CreateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* CreateTriggerResponse::_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) { - default: { - 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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool CreateTriggerResponse::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.artifact.CreateTriggerResponse) - for (;;) { - ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); - tag = p.first; - if (!p.second) goto handle_unusual; - handle_unusual: - if (tag == 0) { - goto success; - } - DO_(::google::protobuf::internal::WireFormat::SkipField( - input, tag, _internal_metadata_.mutable_unknown_fields())); - } -success: - // @@protoc_insertion_point(parse_success:flyteidl.artifact.CreateTriggerResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.CreateTriggerResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void CreateTriggerResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.CreateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.CreateTriggerResponse) -} - -::google::protobuf::uint8* CreateTriggerResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.CreateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - 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.artifact.CreateTriggerResponse) - return target; -} - -size_t CreateTriggerResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.CreateTriggerResponse) - 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; - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void CreateTriggerResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.CreateTriggerResponse) - GOOGLE_DCHECK_NE(&from, this); - const CreateTriggerResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.CreateTriggerResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.CreateTriggerResponse) - MergeFrom(*source); - } -} - -void CreateTriggerResponse::MergeFrom(const CreateTriggerResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.CreateTriggerResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - -} - -void CreateTriggerResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.CreateTriggerResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void CreateTriggerResponse::CopyFrom(const CreateTriggerResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.CreateTriggerResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool CreateTriggerResponse::IsInitialized() const { - return true; -} - -void CreateTriggerResponse::Swap(CreateTriggerResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void CreateTriggerResponse::InternalSwap(CreateTriggerResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); -} - -::google::protobuf::Metadata CreateTriggerResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void DeactivateTriggerRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_DeactivateTriggerRequest_default_instance_._instance.get_mutable()->trigger_id_ = const_cast< ::flyteidl::core::Identifier*>( - ::flyteidl::core::Identifier::internal_default_instance()); -} -class DeactivateTriggerRequest::HasBitSetters { - public: - static const ::flyteidl::core::Identifier& trigger_id(const DeactivateTriggerRequest* msg); -}; - -const ::flyteidl::core::Identifier& -DeactivateTriggerRequest::HasBitSetters::trigger_id(const DeactivateTriggerRequest* msg) { - return *msg->trigger_id_; -} -void DeactivateTriggerRequest::clear_trigger_id() { - if (GetArenaNoVirtual() == nullptr && trigger_id_ != nullptr) { - delete trigger_id_; - } - trigger_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int DeactivateTriggerRequest::kTriggerIdFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -DeactivateTriggerRequest::DeactivateTriggerRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.DeactivateTriggerRequest) -} -DeactivateTriggerRequest::DeactivateTriggerRequest(const DeactivateTriggerRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_trigger_id()) { - trigger_id_ = new ::flyteidl::core::Identifier(*from.trigger_id_); - } else { - trigger_id_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.DeactivateTriggerRequest) -} - -void DeactivateTriggerRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_DeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - trigger_id_ = nullptr; -} - -DeactivateTriggerRequest::~DeactivateTriggerRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.DeactivateTriggerRequest) - SharedDtor(); -} - -void DeactivateTriggerRequest::SharedDtor() { - if (this != internal_default_instance()) delete trigger_id_; -} - -void DeactivateTriggerRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const DeactivateTriggerRequest& DeactivateTriggerRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_DeactivateTriggerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void DeactivateTriggerRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.DeactivateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && trigger_id_ != nullptr) { - delete trigger_id_; - } - trigger_id_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* DeactivateTriggerRequest::_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.Identifier trigger_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::Identifier::_InternalParse; - object = msg->mutable_trigger_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 DeactivateTriggerRequest::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.artifact.DeactivateTriggerRequest) - 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.Identifier trigger_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_trigger_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.artifact.DeactivateTriggerRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.DeactivateTriggerRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void DeactivateTriggerRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.DeactivateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::trigger_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.artifact.DeactivateTriggerRequest) -} - -::google::protobuf::uint8* DeactivateTriggerRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.DeactivateTriggerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::trigger_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.artifact.DeactivateTriggerRequest) - return target; -} - -size_t DeactivateTriggerRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.DeactivateTriggerRequest) - 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.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *trigger_id_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void DeactivateTriggerRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.DeactivateTriggerRequest) - GOOGLE_DCHECK_NE(&from, this); - const DeactivateTriggerRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.DeactivateTriggerRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.DeactivateTriggerRequest) - MergeFrom(*source); - } -} - -void DeactivateTriggerRequest::MergeFrom(const DeactivateTriggerRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.DeactivateTriggerRequest) - 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_trigger_id()) { - mutable_trigger_id()->::flyteidl::core::Identifier::MergeFrom(from.trigger_id()); - } -} - -void DeactivateTriggerRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.DeactivateTriggerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void DeactivateTriggerRequest::CopyFrom(const DeactivateTriggerRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.DeactivateTriggerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool DeactivateTriggerRequest::IsInitialized() const { - return true; -} - -void DeactivateTriggerRequest::Swap(DeactivateTriggerRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void DeactivateTriggerRequest::InternalSwap(DeactivateTriggerRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(trigger_id_, other->trigger_id_); -} - -::google::protobuf::Metadata DeactivateTriggerRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void DeactivateTriggerResponse::InitAsDefaultInstance() { -} -class DeactivateTriggerResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -DeactivateTriggerResponse::DeactivateTriggerResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.DeactivateTriggerResponse) -} -DeactivateTriggerResponse::DeactivateTriggerResponse(const DeactivateTriggerResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.DeactivateTriggerResponse) -} - -void DeactivateTriggerResponse::SharedCtor() { -} - -DeactivateTriggerResponse::~DeactivateTriggerResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.DeactivateTriggerResponse) - SharedDtor(); -} - -void DeactivateTriggerResponse::SharedDtor() { -} - -void DeactivateTriggerResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const DeactivateTriggerResponse& DeactivateTriggerResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_DeactivateTriggerResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void DeactivateTriggerResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.DeactivateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* DeactivateTriggerResponse::_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) { - default: { - 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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool DeactivateTriggerResponse::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.artifact.DeactivateTriggerResponse) - for (;;) { - ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); - tag = p.first; - if (!p.second) goto handle_unusual; - handle_unusual: - if (tag == 0) { - goto success; - } - DO_(::google::protobuf::internal::WireFormat::SkipField( - input, tag, _internal_metadata_.mutable_unknown_fields())); - } -success: - // @@protoc_insertion_point(parse_success:flyteidl.artifact.DeactivateTriggerResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.DeactivateTriggerResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void DeactivateTriggerResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.DeactivateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.DeactivateTriggerResponse) -} - -::google::protobuf::uint8* DeactivateTriggerResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.DeactivateTriggerResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - 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.artifact.DeactivateTriggerResponse) - return target; -} - -size_t DeactivateTriggerResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.DeactivateTriggerResponse) - 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; - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void DeactivateTriggerResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.DeactivateTriggerResponse) - GOOGLE_DCHECK_NE(&from, this); - const DeactivateTriggerResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.DeactivateTriggerResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.DeactivateTriggerResponse) - MergeFrom(*source); - } -} - -void DeactivateTriggerResponse::MergeFrom(const DeactivateTriggerResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.DeactivateTriggerResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - -} - -void DeactivateTriggerResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.DeactivateTriggerResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void DeactivateTriggerResponse::CopyFrom(const DeactivateTriggerResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.DeactivateTriggerResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool DeactivateTriggerResponse::IsInitialized() const { - return true; -} - -void DeactivateTriggerResponse::Swap(DeactivateTriggerResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void DeactivateTriggerResponse::InternalSwap(DeactivateTriggerResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); -} - -::google::protobuf::Metadata DeactivateTriggerResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ArtifactProducer::InitAsDefaultInstance() { - ::flyteidl::artifact::_ArtifactProducer_default_instance_._instance.get_mutable()->entity_id_ = const_cast< ::flyteidl::core::Identifier*>( - ::flyteidl::core::Identifier::internal_default_instance()); - ::flyteidl::artifact::_ArtifactProducer_default_instance_._instance.get_mutable()->outputs_ = const_cast< ::flyteidl::core::VariableMap*>( - ::flyteidl::core::VariableMap::internal_default_instance()); -} -class ArtifactProducer::HasBitSetters { - public: - static const ::flyteidl::core::Identifier& entity_id(const ArtifactProducer* msg); - static const ::flyteidl::core::VariableMap& outputs(const ArtifactProducer* msg); -}; - -const ::flyteidl::core::Identifier& -ArtifactProducer::HasBitSetters::entity_id(const ArtifactProducer* msg) { - return *msg->entity_id_; -} -const ::flyteidl::core::VariableMap& -ArtifactProducer::HasBitSetters::outputs(const ArtifactProducer* msg) { - return *msg->outputs_; -} -void ArtifactProducer::clear_entity_id() { - if (GetArenaNoVirtual() == nullptr && entity_id_ != nullptr) { - delete entity_id_; - } - entity_id_ = nullptr; -} -void ArtifactProducer::clear_outputs() { - if (GetArenaNoVirtual() == nullptr && outputs_ != nullptr) { - delete outputs_; - } - outputs_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ArtifactProducer::kEntityIdFieldNumber; -const int ArtifactProducer::kOutputsFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ArtifactProducer::ArtifactProducer() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ArtifactProducer) -} -ArtifactProducer::ArtifactProducer(const ArtifactProducer& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_entity_id()) { - entity_id_ = new ::flyteidl::core::Identifier(*from.entity_id_); - } else { - entity_id_ = nullptr; - } - if (from.has_outputs()) { - outputs_ = new ::flyteidl::core::VariableMap(*from.outputs_); - } else { - outputs_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ArtifactProducer) -} - -void ArtifactProducer::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::memset(&entity_id_, 0, static_cast( - reinterpret_cast(&outputs_) - - reinterpret_cast(&entity_id_)) + sizeof(outputs_)); -} - -ArtifactProducer::~ArtifactProducer() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ArtifactProducer) - SharedDtor(); -} - -void ArtifactProducer::SharedDtor() { - if (this != internal_default_instance()) delete entity_id_; - if (this != internal_default_instance()) delete outputs_; -} - -void ArtifactProducer::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ArtifactProducer& ArtifactProducer::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ArtifactProducer_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ArtifactProducer::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ArtifactProducer) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && entity_id_ != nullptr) { - delete entity_id_; - } - entity_id_ = nullptr; - if (GetArenaNoVirtual() == nullptr && outputs_ != nullptr) { - delete outputs_; - } - outputs_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ArtifactProducer::_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.Identifier entity_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::Identifier::_InternalParse; - object = msg->mutable_entity_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; - } - // .flyteidl.core.VariableMap outputs = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::VariableMap::_InternalParse; - object = msg->mutable_outputs(); - 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 ArtifactProducer::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.artifact.ArtifactProducer) - 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.Identifier entity_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_entity_id())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.core.VariableMap outputs = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_outputs())); - } 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.artifact.ArtifactProducer) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ArtifactProducer) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ArtifactProducer::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ArtifactProducer) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier entity_id = 1; - if (this->has_entity_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::entity_id(this), output); - } - - // .flyteidl.core.VariableMap outputs = 2; - if (this->has_outputs()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::outputs(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ArtifactProducer) -} - -::google::protobuf::uint8* ArtifactProducer::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ArtifactProducer) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier entity_id = 1; - if (this->has_entity_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::entity_id(this), target); - } - - // .flyteidl.core.VariableMap outputs = 2; - if (this->has_outputs()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::outputs(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.artifact.ArtifactProducer) - return target; -} - -size_t ArtifactProducer::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ArtifactProducer) - 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.Identifier entity_id = 1; - if (this->has_entity_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *entity_id_); - } - - // .flyteidl.core.VariableMap outputs = 2; - if (this->has_outputs()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *outputs_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ArtifactProducer::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ArtifactProducer) - GOOGLE_DCHECK_NE(&from, this); - const ArtifactProducer* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ArtifactProducer) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ArtifactProducer) - MergeFrom(*source); - } -} - -void ArtifactProducer::MergeFrom(const ArtifactProducer& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ArtifactProducer) - 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_entity_id()) { - mutable_entity_id()->::flyteidl::core::Identifier::MergeFrom(from.entity_id()); - } - if (from.has_outputs()) { - mutable_outputs()->::flyteidl::core::VariableMap::MergeFrom(from.outputs()); - } -} - -void ArtifactProducer::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ArtifactProducer) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ArtifactProducer::CopyFrom(const ArtifactProducer& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ArtifactProducer) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ArtifactProducer::IsInitialized() const { - return true; -} - -void ArtifactProducer::Swap(ArtifactProducer* other) { - if (other == this) return; - InternalSwap(other); -} -void ArtifactProducer::InternalSwap(ArtifactProducer* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(entity_id_, other->entity_id_); - swap(outputs_, other->outputs_); -} - -::google::protobuf::Metadata ArtifactProducer::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void RegisterProducerRequest::InitAsDefaultInstance() { -} -class RegisterProducerRequest::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int RegisterProducerRequest::kProducersFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -RegisterProducerRequest::RegisterProducerRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.RegisterProducerRequest) -} -RegisterProducerRequest::RegisterProducerRequest(const RegisterProducerRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - producers_(from.producers_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.RegisterProducerRequest) -} - -void RegisterProducerRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_RegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); -} - -RegisterProducerRequest::~RegisterProducerRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.RegisterProducerRequest) - SharedDtor(); -} - -void RegisterProducerRequest::SharedDtor() { -} - -void RegisterProducerRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const RegisterProducerRequest& RegisterProducerRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_RegisterProducerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void RegisterProducerRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.RegisterProducerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - producers_.Clear(); - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* RegisterProducerRequest::_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) { - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - case 1: { - if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactProducer::_InternalParse; - object = msg->add_producers(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 10 && (ptr += 1)); - 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 RegisterProducerRequest::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.artifact.RegisterProducerRequest) - 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)) { - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_producers())); - } 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.artifact.RegisterProducerRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.RegisterProducerRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void RegisterProducerRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.RegisterProducerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - for (unsigned int i = 0, - n = static_cast(this->producers_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, - this->producers(static_cast(i)), - output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.RegisterProducerRequest) -} - -::google::protobuf::uint8* RegisterProducerRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.RegisterProducerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - for (unsigned int i = 0, - n = static_cast(this->producers_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, this->producers(static_cast(i)), 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.artifact.RegisterProducerRequest) - return target; -} - -size_t RegisterProducerRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.RegisterProducerRequest) - 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; - - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - { - unsigned int count = static_cast(this->producers_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->producers(static_cast(i))); - } - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void RegisterProducerRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.RegisterProducerRequest) - GOOGLE_DCHECK_NE(&from, this); - const RegisterProducerRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.RegisterProducerRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.RegisterProducerRequest) - MergeFrom(*source); - } -} - -void RegisterProducerRequest::MergeFrom(const RegisterProducerRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.RegisterProducerRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - producers_.MergeFrom(from.producers_); -} - -void RegisterProducerRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.RegisterProducerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void RegisterProducerRequest::CopyFrom(const RegisterProducerRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.RegisterProducerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool RegisterProducerRequest::IsInitialized() const { - return true; -} - -void RegisterProducerRequest::Swap(RegisterProducerRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void RegisterProducerRequest::InternalSwap(RegisterProducerRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&producers_)->InternalSwap(CastToBase(&other->producers_)); -} - -::google::protobuf::Metadata RegisterProducerRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ArtifactConsumer::InitAsDefaultInstance() { - ::flyteidl::artifact::_ArtifactConsumer_default_instance_._instance.get_mutable()->entity_id_ = const_cast< ::flyteidl::core::Identifier*>( - ::flyteidl::core::Identifier::internal_default_instance()); - ::flyteidl::artifact::_ArtifactConsumer_default_instance_._instance.get_mutable()->inputs_ = const_cast< ::flyteidl::core::ParameterMap*>( - ::flyteidl::core::ParameterMap::internal_default_instance()); -} -class ArtifactConsumer::HasBitSetters { - public: - static const ::flyteidl::core::Identifier& entity_id(const ArtifactConsumer* msg); - static const ::flyteidl::core::ParameterMap& inputs(const ArtifactConsumer* msg); -}; - -const ::flyteidl::core::Identifier& -ArtifactConsumer::HasBitSetters::entity_id(const ArtifactConsumer* msg) { - return *msg->entity_id_; -} -const ::flyteidl::core::ParameterMap& -ArtifactConsumer::HasBitSetters::inputs(const ArtifactConsumer* msg) { - return *msg->inputs_; -} -void ArtifactConsumer::clear_entity_id() { - if (GetArenaNoVirtual() == nullptr && entity_id_ != nullptr) { - delete entity_id_; - } - entity_id_ = nullptr; -} -void ArtifactConsumer::clear_inputs() { - if (GetArenaNoVirtual() == nullptr && inputs_ != nullptr) { - delete inputs_; - } - inputs_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ArtifactConsumer::kEntityIdFieldNumber; -const int ArtifactConsumer::kInputsFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ArtifactConsumer::ArtifactConsumer() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ArtifactConsumer) -} -ArtifactConsumer::ArtifactConsumer(const ArtifactConsumer& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_entity_id()) { - entity_id_ = new ::flyteidl::core::Identifier(*from.entity_id_); - } else { - entity_id_ = nullptr; - } - if (from.has_inputs()) { - inputs_ = new ::flyteidl::core::ParameterMap(*from.inputs_); - } else { - inputs_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ArtifactConsumer) -} - -void ArtifactConsumer::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto.base); - ::memset(&entity_id_, 0, static_cast( - reinterpret_cast(&inputs_) - - reinterpret_cast(&entity_id_)) + sizeof(inputs_)); -} - -ArtifactConsumer::~ArtifactConsumer() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ArtifactConsumer) - SharedDtor(); -} - -void ArtifactConsumer::SharedDtor() { - if (this != internal_default_instance()) delete entity_id_; - if (this != internal_default_instance()) delete inputs_; -} - -void ArtifactConsumer::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ArtifactConsumer& ArtifactConsumer::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ArtifactConsumer_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ArtifactConsumer::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ArtifactConsumer) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && entity_id_ != nullptr) { - delete entity_id_; - } - entity_id_ = nullptr; - if (GetArenaNoVirtual() == nullptr && inputs_ != nullptr) { - delete inputs_; - } - inputs_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ArtifactConsumer::_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.Identifier entity_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::Identifier::_InternalParse; - object = msg->mutable_entity_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; - } - // .flyteidl.core.ParameterMap inputs = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::ParameterMap::_InternalParse; - object = msg->mutable_inputs(); - 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 ArtifactConsumer::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.artifact.ArtifactConsumer) - 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.Identifier entity_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_entity_id())); - } else { - goto handle_unusual; - } - break; - } - - // .flyteidl.core.ParameterMap inputs = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_inputs())); - } 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.artifact.ArtifactConsumer) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ArtifactConsumer) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ArtifactConsumer::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ArtifactConsumer) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier entity_id = 1; - if (this->has_entity_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::entity_id(this), output); - } - - // .flyteidl.core.ParameterMap inputs = 2; - if (this->has_inputs()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, HasBitSetters::inputs(this), output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ArtifactConsumer) -} - -::google::protobuf::uint8* ArtifactConsumer::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ArtifactConsumer) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier entity_id = 1; - if (this->has_entity_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::entity_id(this), target); - } - - // .flyteidl.core.ParameterMap inputs = 2; - if (this->has_inputs()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, HasBitSetters::inputs(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.artifact.ArtifactConsumer) - return target; -} - -size_t ArtifactConsumer::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ArtifactConsumer) - 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.Identifier entity_id = 1; - if (this->has_entity_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *entity_id_); - } - - // .flyteidl.core.ParameterMap inputs = 2; - if (this->has_inputs()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *inputs_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ArtifactConsumer::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ArtifactConsumer) - GOOGLE_DCHECK_NE(&from, this); - const ArtifactConsumer* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ArtifactConsumer) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ArtifactConsumer) - MergeFrom(*source); - } -} - -void ArtifactConsumer::MergeFrom(const ArtifactConsumer& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ArtifactConsumer) - 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_entity_id()) { - mutable_entity_id()->::flyteidl::core::Identifier::MergeFrom(from.entity_id()); - } - if (from.has_inputs()) { - mutable_inputs()->::flyteidl::core::ParameterMap::MergeFrom(from.inputs()); - } -} - -void ArtifactConsumer::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ArtifactConsumer) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ArtifactConsumer::CopyFrom(const ArtifactConsumer& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ArtifactConsumer) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ArtifactConsumer::IsInitialized() const { - return true; -} - -void ArtifactConsumer::Swap(ArtifactConsumer* other) { - if (other == this) return; - InternalSwap(other); -} -void ArtifactConsumer::InternalSwap(ArtifactConsumer* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(entity_id_, other->entity_id_); - swap(inputs_, other->inputs_); -} - -::google::protobuf::Metadata ArtifactConsumer::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void RegisterConsumerRequest::InitAsDefaultInstance() { -} -class RegisterConsumerRequest::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int RegisterConsumerRequest::kConsumersFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -RegisterConsumerRequest::RegisterConsumerRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.RegisterConsumerRequest) -} -RegisterConsumerRequest::RegisterConsumerRequest(const RegisterConsumerRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - consumers_(from.consumers_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.RegisterConsumerRequest) -} - -void RegisterConsumerRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_RegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); -} - -RegisterConsumerRequest::~RegisterConsumerRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.RegisterConsumerRequest) - SharedDtor(); -} - -void RegisterConsumerRequest::SharedDtor() { -} - -void RegisterConsumerRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const RegisterConsumerRequest& RegisterConsumerRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_RegisterConsumerRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void RegisterConsumerRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.RegisterConsumerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - consumers_.Clear(); - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* RegisterConsumerRequest::_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) { - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - case 1: { - if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::artifact::ArtifactConsumer::_InternalParse; - object = msg->add_consumers(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 10 && (ptr += 1)); - 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 RegisterConsumerRequest::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.artifact.RegisterConsumerRequest) - 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)) { - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_consumers())); - } 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.artifact.RegisterConsumerRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.RegisterConsumerRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void RegisterConsumerRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.RegisterConsumerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - for (unsigned int i = 0, - n = static_cast(this->consumers_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, - this->consumers(static_cast(i)), - output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.RegisterConsumerRequest) -} - -::google::protobuf::uint8* RegisterConsumerRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.RegisterConsumerRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - for (unsigned int i = 0, - n = static_cast(this->consumers_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, this->consumers(static_cast(i)), 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.artifact.RegisterConsumerRequest) - return target; -} - -size_t RegisterConsumerRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.RegisterConsumerRequest) - 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; - - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - { - unsigned int count = static_cast(this->consumers_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->consumers(static_cast(i))); - } - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void RegisterConsumerRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.RegisterConsumerRequest) - GOOGLE_DCHECK_NE(&from, this); - const RegisterConsumerRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.RegisterConsumerRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.RegisterConsumerRequest) - MergeFrom(*source); - } -} - -void RegisterConsumerRequest::MergeFrom(const RegisterConsumerRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.RegisterConsumerRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - consumers_.MergeFrom(from.consumers_); -} - -void RegisterConsumerRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.RegisterConsumerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void RegisterConsumerRequest::CopyFrom(const RegisterConsumerRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.RegisterConsumerRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool RegisterConsumerRequest::IsInitialized() const { - return true; -} - -void RegisterConsumerRequest::Swap(RegisterConsumerRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void RegisterConsumerRequest::InternalSwap(RegisterConsumerRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&consumers_)->InternalSwap(CastToBase(&other->consumers_)); -} - -::google::protobuf::Metadata RegisterConsumerRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void RegisterResponse::InitAsDefaultInstance() { -} -class RegisterResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -RegisterResponse::RegisterResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.RegisterResponse) -} -RegisterResponse::RegisterResponse(const RegisterResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.RegisterResponse) -} - -void RegisterResponse::SharedCtor() { -} - -RegisterResponse::~RegisterResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.RegisterResponse) - SharedDtor(); -} - -void RegisterResponse::SharedDtor() { -} - -void RegisterResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const RegisterResponse& RegisterResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_RegisterResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void RegisterResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.RegisterResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* RegisterResponse::_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) { - default: { - 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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool RegisterResponse::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.artifact.RegisterResponse) - for (;;) { - ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); - tag = p.first; - if (!p.second) goto handle_unusual; - handle_unusual: - if (tag == 0) { - goto success; - } - DO_(::google::protobuf::internal::WireFormat::SkipField( - input, tag, _internal_metadata_.mutable_unknown_fields())); - } -success: - // @@protoc_insertion_point(parse_success:flyteidl.artifact.RegisterResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.RegisterResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void RegisterResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.RegisterResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.RegisterResponse) -} - -::google::protobuf::uint8* RegisterResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.RegisterResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - 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.artifact.RegisterResponse) - return target; -} - -size_t RegisterResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.RegisterResponse) - 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; - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void RegisterResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.RegisterResponse) - GOOGLE_DCHECK_NE(&from, this); - const RegisterResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.RegisterResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.RegisterResponse) - MergeFrom(*source); - } -} - -void RegisterResponse::MergeFrom(const RegisterResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.RegisterResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - -} - -void RegisterResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.RegisterResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void RegisterResponse::CopyFrom(const RegisterResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.RegisterResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool RegisterResponse::IsInitialized() const { - return true; -} - -void RegisterResponse::Swap(RegisterResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void RegisterResponse::InternalSwap(RegisterResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); -} - -::google::protobuf::Metadata RegisterResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ExecutionInputsRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_ExecutionInputsRequest_default_instance_._instance.get_mutable()->execution_id_ = const_cast< ::flyteidl::core::WorkflowExecutionIdentifier*>( - ::flyteidl::core::WorkflowExecutionIdentifier::internal_default_instance()); -} -class ExecutionInputsRequest::HasBitSetters { - public: - static const ::flyteidl::core::WorkflowExecutionIdentifier& execution_id(const ExecutionInputsRequest* msg); -}; - -const ::flyteidl::core::WorkflowExecutionIdentifier& -ExecutionInputsRequest::HasBitSetters::execution_id(const ExecutionInputsRequest* msg) { - return *msg->execution_id_; -} -void ExecutionInputsRequest::clear_execution_id() { - if (GetArenaNoVirtual() == nullptr && execution_id_ != nullptr) { - delete execution_id_; - } - execution_id_ = nullptr; -} -void ExecutionInputsRequest::clear_inputs() { - inputs_.Clear(); -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ExecutionInputsRequest::kExecutionIdFieldNumber; -const int ExecutionInputsRequest::kInputsFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ExecutionInputsRequest::ExecutionInputsRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ExecutionInputsRequest) -} -ExecutionInputsRequest::ExecutionInputsRequest(const ExecutionInputsRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - inputs_(from.inputs_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_execution_id()) { - execution_id_ = new ::flyteidl::core::WorkflowExecutionIdentifier(*from.execution_id_); - } else { - execution_id_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ExecutionInputsRequest) -} - -void ExecutionInputsRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - execution_id_ = nullptr; -} - -ExecutionInputsRequest::~ExecutionInputsRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ExecutionInputsRequest) - SharedDtor(); -} - -void ExecutionInputsRequest::SharedDtor() { - if (this != internal_default_instance()) delete execution_id_; -} - -void ExecutionInputsRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ExecutionInputsRequest& ExecutionInputsRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ExecutionInputsRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ExecutionInputsRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ExecutionInputsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - inputs_.Clear(); - if (GetArenaNoVirtual() == nullptr && execution_id_ != nullptr) { - delete execution_id_; - } - execution_id_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ExecutionInputsRequest::_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.WorkflowExecutionIdentifier execution_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::WorkflowExecutionIdentifier::_InternalParse; - object = msg->mutable_execution_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; - } - // repeated .flyteidl.core.ArtifactID inputs = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::ArtifactID::_InternalParse; - object = msg->add_inputs(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 18 && (ptr += 1)); - 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 ExecutionInputsRequest::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.artifact.ExecutionInputsRequest) - 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.WorkflowExecutionIdentifier execution_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_execution_id())); - } else { - goto handle_unusual; - } - break; - } - - // repeated .flyteidl.core.ArtifactID inputs = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_inputs())); - } 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.artifact.ExecutionInputsRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ExecutionInputsRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ExecutionInputsRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ExecutionInputsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - if (this->has_execution_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::execution_id(this), output); - } - - // repeated .flyteidl.core.ArtifactID inputs = 2; - for (unsigned int i = 0, - n = static_cast(this->inputs_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, - this->inputs(static_cast(i)), - output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ExecutionInputsRequest) -} - -::google::protobuf::uint8* ExecutionInputsRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ExecutionInputsRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - if (this->has_execution_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::execution_id(this), target); - } - - // repeated .flyteidl.core.ArtifactID inputs = 2; - for (unsigned int i = 0, - n = static_cast(this->inputs_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, this->inputs(static_cast(i)), 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.artifact.ExecutionInputsRequest) - return target; -} - -size_t ExecutionInputsRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ExecutionInputsRequest) - 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; - - // repeated .flyteidl.core.ArtifactID inputs = 2; - { - unsigned int count = static_cast(this->inputs_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->inputs(static_cast(i))); - } - } - - // .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - if (this->has_execution_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *execution_id_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ExecutionInputsRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ExecutionInputsRequest) - GOOGLE_DCHECK_NE(&from, this); - const ExecutionInputsRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ExecutionInputsRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ExecutionInputsRequest) - MergeFrom(*source); - } -} - -void ExecutionInputsRequest::MergeFrom(const ExecutionInputsRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ExecutionInputsRequest) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - inputs_.MergeFrom(from.inputs_); - if (from.has_execution_id()) { - mutable_execution_id()->::flyteidl::core::WorkflowExecutionIdentifier::MergeFrom(from.execution_id()); - } -} - -void ExecutionInputsRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ExecutionInputsRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ExecutionInputsRequest::CopyFrom(const ExecutionInputsRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ExecutionInputsRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ExecutionInputsRequest::IsInitialized() const { - return true; -} - -void ExecutionInputsRequest::Swap(ExecutionInputsRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void ExecutionInputsRequest::InternalSwap(ExecutionInputsRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&inputs_)->InternalSwap(CastToBase(&other->inputs_)); - swap(execution_id_, other->execution_id_); -} - -::google::protobuf::Metadata ExecutionInputsRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ExecutionInputsResponse::InitAsDefaultInstance() { -} -class ExecutionInputsResponse::HasBitSetters { - public: -}; - -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ExecutionInputsResponse::ExecutionInputsResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ExecutionInputsResponse) -} -ExecutionInputsResponse::ExecutionInputsResponse(const ExecutionInputsResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ExecutionInputsResponse) -} - -void ExecutionInputsResponse::SharedCtor() { -} - -ExecutionInputsResponse::~ExecutionInputsResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ExecutionInputsResponse) - SharedDtor(); -} - -void ExecutionInputsResponse::SharedDtor() { -} - -void ExecutionInputsResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ExecutionInputsResponse& ExecutionInputsResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ExecutionInputsResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ExecutionInputsResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ExecutionInputsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ExecutionInputsResponse::_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) { - default: { - 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; -} -#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -bool ExecutionInputsResponse::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.artifact.ExecutionInputsResponse) - for (;;) { - ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); - tag = p.first; - if (!p.second) goto handle_unusual; - handle_unusual: - if (tag == 0) { - goto success; - } - DO_(::google::protobuf::internal::WireFormat::SkipField( - input, tag, _internal_metadata_.mutable_unknown_fields())); - } -success: - // @@protoc_insertion_point(parse_success:flyteidl.artifact.ExecutionInputsResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ExecutionInputsResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ExecutionInputsResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ExecutionInputsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ExecutionInputsResponse) -} - -::google::protobuf::uint8* ExecutionInputsResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ExecutionInputsResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - 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.artifact.ExecutionInputsResponse) - return target; -} - -size_t ExecutionInputsResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ExecutionInputsResponse) - 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; - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ExecutionInputsResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ExecutionInputsResponse) - GOOGLE_DCHECK_NE(&from, this); - const ExecutionInputsResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ExecutionInputsResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ExecutionInputsResponse) - MergeFrom(*source); - } -} - -void ExecutionInputsResponse::MergeFrom(const ExecutionInputsResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ExecutionInputsResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - -} - -void ExecutionInputsResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ExecutionInputsResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ExecutionInputsResponse::CopyFrom(const ExecutionInputsResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ExecutionInputsResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ExecutionInputsResponse::IsInitialized() const { - return true; -} - -void ExecutionInputsResponse::Swap(ExecutionInputsResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void ExecutionInputsResponse::InternalSwap(ExecutionInputsResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); -} - -::google::protobuf::Metadata ExecutionInputsResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ListUsageRequest::InitAsDefaultInstance() { - ::flyteidl::artifact::_ListUsageRequest_default_instance_._instance.get_mutable()->artifact_id_ = const_cast< ::flyteidl::core::ArtifactID*>( - ::flyteidl::core::ArtifactID::internal_default_instance()); -} -class ListUsageRequest::HasBitSetters { - public: - static const ::flyteidl::core::ArtifactID& artifact_id(const ListUsageRequest* msg); -}; - -const ::flyteidl::core::ArtifactID& -ListUsageRequest::HasBitSetters::artifact_id(const ListUsageRequest* msg) { - return *msg->artifact_id_; -} -void ListUsageRequest::clear_artifact_id() { - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ListUsageRequest::kArtifactIdFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ListUsageRequest::ListUsageRequest() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ListUsageRequest) -} -ListUsageRequest::ListUsageRequest(const ListUsageRequest& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_artifact_id()) { - artifact_id_ = new ::flyteidl::core::ArtifactID(*from.artifact_id_); - } else { - artifact_id_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ListUsageRequest) -} - -void ListUsageRequest::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - artifact_id_ = nullptr; -} - -ListUsageRequest::~ListUsageRequest() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ListUsageRequest) - SharedDtor(); -} - -void ListUsageRequest::SharedDtor() { - if (this != internal_default_instance()) delete artifact_id_; -} - -void ListUsageRequest::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ListUsageRequest& ListUsageRequest::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ListUsageRequest_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ListUsageRequest::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ListUsageRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - if (GetArenaNoVirtual() == nullptr && artifact_id_ != nullptr) { - delete artifact_id_; - } - artifact_id_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ListUsageRequest::_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.ArtifactID artifact_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::ArtifactID::_InternalParse; - object = msg->mutable_artifact_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 ListUsageRequest::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.artifact.ListUsageRequest) - 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.ArtifactID artifact_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_artifact_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.artifact.ListUsageRequest) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ListUsageRequest) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ListUsageRequest::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ListUsageRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::artifact_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.artifact.ListUsageRequest) -} - -::google::protobuf::uint8* ListUsageRequest::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ListUsageRequest) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::artifact_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.artifact.ListUsageRequest) - return target; -} - -size_t ListUsageRequest::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ListUsageRequest) - 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.ArtifactID artifact_id = 1; - if (this->has_artifact_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *artifact_id_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ListUsageRequest::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ListUsageRequest) - GOOGLE_DCHECK_NE(&from, this); - const ListUsageRequest* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ListUsageRequest) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ListUsageRequest) - MergeFrom(*source); - } -} - -void ListUsageRequest::MergeFrom(const ListUsageRequest& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ListUsageRequest) - 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_artifact_id()) { - mutable_artifact_id()->::flyteidl::core::ArtifactID::MergeFrom(from.artifact_id()); - } -} - -void ListUsageRequest::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ListUsageRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ListUsageRequest::CopyFrom(const ListUsageRequest& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ListUsageRequest) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ListUsageRequest::IsInitialized() const { - return true; -} - -void ListUsageRequest::Swap(ListUsageRequest* other) { - if (other == this) return; - InternalSwap(other); -} -void ListUsageRequest::InternalSwap(ListUsageRequest* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - swap(artifact_id_, other->artifact_id_); -} - -::google::protobuf::Metadata ListUsageRequest::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// =================================================================== - -void ListUsageResponse::InitAsDefaultInstance() { -} -class ListUsageResponse::HasBitSetters { - public: -}; - -void ListUsageResponse::clear_executions() { - executions_.Clear(); -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int ListUsageResponse::kExecutionsFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -ListUsageResponse::ListUsageResponse() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.artifact.ListUsageResponse) -} -ListUsageResponse::ListUsageResponse(const ListUsageResponse& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - executions_(from.executions_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - // @@protoc_insertion_point(copy_constructor:flyteidl.artifact.ListUsageResponse) -} - -void ListUsageResponse::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_ListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); -} - -ListUsageResponse::~ListUsageResponse() { - // @@protoc_insertion_point(destructor:flyteidl.artifact.ListUsageResponse) - SharedDtor(); -} - -void ListUsageResponse::SharedDtor() { -} - -void ListUsageResponse::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const ListUsageResponse& ListUsageResponse::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_ListUsageResponse_flyteidl_2fartifact_2fartifacts_2eproto.base); - return *internal_default_instance(); -} - - -void ListUsageResponse::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.artifact.ListUsageResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - executions_.Clear(); - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* ListUsageResponse::_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) { - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - case 1: { - if (static_cast<::google::protobuf::uint8>(tag) != 10) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::WorkflowExecutionIdentifier::_InternalParse; - object = msg->add_executions(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 10 && (ptr += 1)); - 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 ListUsageResponse::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.artifact.ListUsageResponse) - 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)) { - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_executions())); - } 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.artifact.ListUsageResponse) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.artifact.ListUsageResponse) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void ListUsageResponse::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.artifact.ListUsageResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - for (unsigned int i = 0, - n = static_cast(this->executions_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, - this->executions(static_cast(i)), - output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.artifact.ListUsageResponse) -} - -::google::protobuf::uint8* ListUsageResponse::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.artifact.ListUsageResponse) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - for (unsigned int i = 0, - n = static_cast(this->executions_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, this->executions(static_cast(i)), 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.artifact.ListUsageResponse) - return target; -} - -size_t ListUsageResponse::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.artifact.ListUsageResponse) - 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; - - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - { - unsigned int count = static_cast(this->executions_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->executions(static_cast(i))); - } - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void ListUsageResponse::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.artifact.ListUsageResponse) - GOOGLE_DCHECK_NE(&from, this); - const ListUsageResponse* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.artifact.ListUsageResponse) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.artifact.ListUsageResponse) - MergeFrom(*source); - } -} - -void ListUsageResponse::MergeFrom(const ListUsageResponse& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.artifact.ListUsageResponse) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - executions_.MergeFrom(from.executions_); -} - -void ListUsageResponse::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.artifact.ListUsageResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void ListUsageResponse::CopyFrom(const ListUsageResponse& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.artifact.ListUsageResponse) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool ListUsageResponse::IsInitialized() const { - return true; -} - -void ListUsageResponse::Swap(ListUsageResponse* other) { - if (other == this) return; - InternalSwap(other); -} -void ListUsageResponse::InternalSwap(ListUsageResponse* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&executions_)->InternalSwap(CastToBase(&other->executions_)); -} - -::google::protobuf::Metadata ListUsageResponse::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fartifact_2fartifacts_2eproto); - return ::file_level_metadata_flyteidl_2fartifact_2fartifacts_2eproto[kIndexInFileMessages]; -} - - -// @@protoc_insertion_point(namespace_scope) -} // namespace artifact -} // namespace flyteidl -namespace google { -namespace protobuf { -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::Artifact* Arena::CreateMaybeMessage< ::flyteidl::artifact::Artifact >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::Artifact >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse* Arena::CreateMaybeMessage< ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::CreateArtifactRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::CreateArtifactRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::CreateArtifactRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ArtifactSource* Arena::CreateMaybeMessage< ::flyteidl::artifact::ArtifactSource >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ArtifactSource >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ArtifactSpec* Arena::CreateMaybeMessage< ::flyteidl::artifact::ArtifactSpec >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ArtifactSpec >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::CreateArtifactResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::CreateArtifactResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::CreateArtifactResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::GetArtifactRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::GetArtifactRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::GetArtifactRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::GetArtifactResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::GetArtifactResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::GetArtifactResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::SearchOptions* Arena::CreateMaybeMessage< ::flyteidl::artifact::SearchOptions >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::SearchOptions >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::SearchArtifactsRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::SearchArtifactsRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::SearchArtifactsRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::SearchArtifactsResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::SearchArtifactsResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::SearchArtifactsResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::FindByWorkflowExecRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::FindByWorkflowExecRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::FindByWorkflowExecRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::AddTagRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::AddTagRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::AddTagRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::AddTagResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::AddTagResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::AddTagResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::CreateTriggerRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::CreateTriggerRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::CreateTriggerRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::CreateTriggerResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::CreateTriggerResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::CreateTriggerResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::DeactivateTriggerRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::DeactivateTriggerRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::DeactivateTriggerRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::DeactivateTriggerResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::DeactivateTriggerResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::DeactivateTriggerResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ArtifactProducer* Arena::CreateMaybeMessage< ::flyteidl::artifact::ArtifactProducer >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ArtifactProducer >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::RegisterProducerRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::RegisterProducerRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::RegisterProducerRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ArtifactConsumer* Arena::CreateMaybeMessage< ::flyteidl::artifact::ArtifactConsumer >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ArtifactConsumer >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::RegisterConsumerRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::RegisterConsumerRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::RegisterConsumerRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::RegisterResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::RegisterResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::RegisterResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ExecutionInputsRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::ExecutionInputsRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ExecutionInputsRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ExecutionInputsResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::ExecutionInputsResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ExecutionInputsResponse >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ListUsageRequest* Arena::CreateMaybeMessage< ::flyteidl::artifact::ListUsageRequest >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ListUsageRequest >(arena); -} -template<> PROTOBUF_NOINLINE ::flyteidl::artifact::ListUsageResponse* Arena::CreateMaybeMessage< ::flyteidl::artifact::ListUsageResponse >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::artifact::ListUsageResponse >(arena); -} -} // namespace protobuf -} // namespace google - -// @@protoc_insertion_point(global_scope) -#include diff --git a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.h b/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.h deleted file mode 100644 index d8b585822e..0000000000 --- a/flyteidl/gen/pb-cpp/flyteidl/artifact/artifacts.pb.h +++ /dev/null @@ -1,6099 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: flyteidl/artifact/artifacts.proto - -#ifndef PROTOBUF_INCLUDED_flyteidl_2fartifact_2fartifacts_2eproto -#define PROTOBUF_INCLUDED_flyteidl_2fartifact_2fartifacts_2eproto - -#include -#include - -#include -#if PROTOBUF_VERSION < 3007000 -#error This file was generated by a newer version of protoc which is -#error incompatible with your Protocol Buffer headers. Please update -#error your headers. -#endif -#if 3007000 < PROTOBUF_MIN_PROTOC_VERSION -#error This file was generated by an older version of protoc which is -#error incompatible with your Protocol Buffer headers. Please -#error regenerate this file with a newer version of protoc. -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include // IWYU pragma: export -#include // IWYU pragma: export -#include // IWYU pragma: export -#include -#include -#include -#include -#include -#include "google/api/annotations.pb.h" -#include -#include "flyteidl/admin/launch_plan.pb.h" -#include "flyteidl/core/literals.pb.h" -#include "flyteidl/core/types.pb.h" -#include "flyteidl/core/identifier.pb.h" -#include "flyteidl/core/artifact_id.pb.h" -#include "flyteidl/core/interface.pb.h" -#include "flyteidl/event/cloudevents.pb.h" -// @@protoc_insertion_point(includes) -#include -#define PROTOBUF_INTERNAL_EXPORT_flyteidl_2fartifact_2fartifacts_2eproto - -// Internal implementation detail -- do not use these members. -struct TableStruct_flyteidl_2fartifact_2fartifacts_2eproto { - static const ::google::protobuf::internal::ParseTableField entries[] - PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::google::protobuf::internal::AuxillaryParseTableField aux[] - PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::google::protobuf::internal::ParseTable schema[27] - PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::google::protobuf::internal::FieldMetadata field_metadata[]; - static const ::google::protobuf::internal::SerializationTable serialization_table[]; - static const ::google::protobuf::uint32 offsets[]; -}; -void AddDescriptors_flyteidl_2fartifact_2fartifacts_2eproto(); -namespace flyteidl { -namespace artifact { -class AddTagRequest; -class AddTagRequestDefaultTypeInternal; -extern AddTagRequestDefaultTypeInternal _AddTagRequest_default_instance_; -class AddTagResponse; -class AddTagResponseDefaultTypeInternal; -extern AddTagResponseDefaultTypeInternal _AddTagResponse_default_instance_; -class Artifact; -class ArtifactDefaultTypeInternal; -extern ArtifactDefaultTypeInternal _Artifact_default_instance_; -class ArtifactConsumer; -class ArtifactConsumerDefaultTypeInternal; -extern ArtifactConsumerDefaultTypeInternal _ArtifactConsumer_default_instance_; -class ArtifactProducer; -class ArtifactProducerDefaultTypeInternal; -extern ArtifactProducerDefaultTypeInternal _ArtifactProducer_default_instance_; -class ArtifactSource; -class ArtifactSourceDefaultTypeInternal; -extern ArtifactSourceDefaultTypeInternal _ArtifactSource_default_instance_; -class ArtifactSpec; -class ArtifactSpecDefaultTypeInternal; -extern ArtifactSpecDefaultTypeInternal _ArtifactSpec_default_instance_; -class CreateArtifactRequest; -class CreateArtifactRequestDefaultTypeInternal; -extern CreateArtifactRequestDefaultTypeInternal _CreateArtifactRequest_default_instance_; -class CreateArtifactRequest_PartitionsEntry_DoNotUse; -class CreateArtifactRequest_PartitionsEntry_DoNotUseDefaultTypeInternal; -extern CreateArtifactRequest_PartitionsEntry_DoNotUseDefaultTypeInternal _CreateArtifactRequest_PartitionsEntry_DoNotUse_default_instance_; -class CreateArtifactResponse; -class CreateArtifactResponseDefaultTypeInternal; -extern CreateArtifactResponseDefaultTypeInternal _CreateArtifactResponse_default_instance_; -class CreateTriggerRequest; -class CreateTriggerRequestDefaultTypeInternal; -extern CreateTriggerRequestDefaultTypeInternal _CreateTriggerRequest_default_instance_; -class CreateTriggerResponse; -class CreateTriggerResponseDefaultTypeInternal; -extern CreateTriggerResponseDefaultTypeInternal _CreateTriggerResponse_default_instance_; -class DeactivateTriggerRequest; -class DeactivateTriggerRequestDefaultTypeInternal; -extern DeactivateTriggerRequestDefaultTypeInternal _DeactivateTriggerRequest_default_instance_; -class DeactivateTriggerResponse; -class DeactivateTriggerResponseDefaultTypeInternal; -extern DeactivateTriggerResponseDefaultTypeInternal _DeactivateTriggerResponse_default_instance_; -class ExecutionInputsRequest; -class ExecutionInputsRequestDefaultTypeInternal; -extern ExecutionInputsRequestDefaultTypeInternal _ExecutionInputsRequest_default_instance_; -class ExecutionInputsResponse; -class ExecutionInputsResponseDefaultTypeInternal; -extern ExecutionInputsResponseDefaultTypeInternal _ExecutionInputsResponse_default_instance_; -class FindByWorkflowExecRequest; -class FindByWorkflowExecRequestDefaultTypeInternal; -extern FindByWorkflowExecRequestDefaultTypeInternal _FindByWorkflowExecRequest_default_instance_; -class GetArtifactRequest; -class GetArtifactRequestDefaultTypeInternal; -extern GetArtifactRequestDefaultTypeInternal _GetArtifactRequest_default_instance_; -class GetArtifactResponse; -class GetArtifactResponseDefaultTypeInternal; -extern GetArtifactResponseDefaultTypeInternal _GetArtifactResponse_default_instance_; -class ListUsageRequest; -class ListUsageRequestDefaultTypeInternal; -extern ListUsageRequestDefaultTypeInternal _ListUsageRequest_default_instance_; -class ListUsageResponse; -class ListUsageResponseDefaultTypeInternal; -extern ListUsageResponseDefaultTypeInternal _ListUsageResponse_default_instance_; -class RegisterConsumerRequest; -class RegisterConsumerRequestDefaultTypeInternal; -extern RegisterConsumerRequestDefaultTypeInternal _RegisterConsumerRequest_default_instance_; -class RegisterProducerRequest; -class RegisterProducerRequestDefaultTypeInternal; -extern RegisterProducerRequestDefaultTypeInternal _RegisterProducerRequest_default_instance_; -class RegisterResponse; -class RegisterResponseDefaultTypeInternal; -extern RegisterResponseDefaultTypeInternal _RegisterResponse_default_instance_; -class SearchArtifactsRequest; -class SearchArtifactsRequestDefaultTypeInternal; -extern SearchArtifactsRequestDefaultTypeInternal _SearchArtifactsRequest_default_instance_; -class SearchArtifactsResponse; -class SearchArtifactsResponseDefaultTypeInternal; -extern SearchArtifactsResponseDefaultTypeInternal _SearchArtifactsResponse_default_instance_; -class SearchOptions; -class SearchOptionsDefaultTypeInternal; -extern SearchOptionsDefaultTypeInternal _SearchOptions_default_instance_; -} // namespace artifact -} // namespace flyteidl -namespace google { -namespace protobuf { -template<> ::flyteidl::artifact::AddTagRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::AddTagRequest>(Arena*); -template<> ::flyteidl::artifact::AddTagResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::AddTagResponse>(Arena*); -template<> ::flyteidl::artifact::Artifact* Arena::CreateMaybeMessage<::flyteidl::artifact::Artifact>(Arena*); -template<> ::flyteidl::artifact::ArtifactConsumer* Arena::CreateMaybeMessage<::flyteidl::artifact::ArtifactConsumer>(Arena*); -template<> ::flyteidl::artifact::ArtifactProducer* Arena::CreateMaybeMessage<::flyteidl::artifact::ArtifactProducer>(Arena*); -template<> ::flyteidl::artifact::ArtifactSource* Arena::CreateMaybeMessage<::flyteidl::artifact::ArtifactSource>(Arena*); -template<> ::flyteidl::artifact::ArtifactSpec* Arena::CreateMaybeMessage<::flyteidl::artifact::ArtifactSpec>(Arena*); -template<> ::flyteidl::artifact::CreateArtifactRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::CreateArtifactRequest>(Arena*); -template<> ::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse* Arena::CreateMaybeMessage<::flyteidl::artifact::CreateArtifactRequest_PartitionsEntry_DoNotUse>(Arena*); -template<> ::flyteidl::artifact::CreateArtifactResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::CreateArtifactResponse>(Arena*); -template<> ::flyteidl::artifact::CreateTriggerRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::CreateTriggerRequest>(Arena*); -template<> ::flyteidl::artifact::CreateTriggerResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::CreateTriggerResponse>(Arena*); -template<> ::flyteidl::artifact::DeactivateTriggerRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::DeactivateTriggerRequest>(Arena*); -template<> ::flyteidl::artifact::DeactivateTriggerResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::DeactivateTriggerResponse>(Arena*); -template<> ::flyteidl::artifact::ExecutionInputsRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::ExecutionInputsRequest>(Arena*); -template<> ::flyteidl::artifact::ExecutionInputsResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::ExecutionInputsResponse>(Arena*); -template<> ::flyteidl::artifact::FindByWorkflowExecRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::FindByWorkflowExecRequest>(Arena*); -template<> ::flyteidl::artifact::GetArtifactRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::GetArtifactRequest>(Arena*); -template<> ::flyteidl::artifact::GetArtifactResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::GetArtifactResponse>(Arena*); -template<> ::flyteidl::artifact::ListUsageRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::ListUsageRequest>(Arena*); -template<> ::flyteidl::artifact::ListUsageResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::ListUsageResponse>(Arena*); -template<> ::flyteidl::artifact::RegisterConsumerRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::RegisterConsumerRequest>(Arena*); -template<> ::flyteidl::artifact::RegisterProducerRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::RegisterProducerRequest>(Arena*); -template<> ::flyteidl::artifact::RegisterResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::RegisterResponse>(Arena*); -template<> ::flyteidl::artifact::SearchArtifactsRequest* Arena::CreateMaybeMessage<::flyteidl::artifact::SearchArtifactsRequest>(Arena*); -template<> ::flyteidl::artifact::SearchArtifactsResponse* Arena::CreateMaybeMessage<::flyteidl::artifact::SearchArtifactsResponse>(Arena*); -template<> ::flyteidl::artifact::SearchOptions* Arena::CreateMaybeMessage<::flyteidl::artifact::SearchOptions>(Arena*); -} // namespace protobuf -} // namespace google -namespace flyteidl { -namespace artifact { - -enum FindByWorkflowExecRequest_Direction { - FindByWorkflowExecRequest_Direction_INPUTS = 0, - FindByWorkflowExecRequest_Direction_OUTPUTS = 1, - FindByWorkflowExecRequest_Direction_FindByWorkflowExecRequest_Direction_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::google::protobuf::int32>::min(), - FindByWorkflowExecRequest_Direction_FindByWorkflowExecRequest_Direction_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::google::protobuf::int32>::max() -}; -bool FindByWorkflowExecRequest_Direction_IsValid(int value); -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest_Direction_Direction_MIN = FindByWorkflowExecRequest_Direction_INPUTS; -const FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest_Direction_Direction_MAX = FindByWorkflowExecRequest_Direction_OUTPUTS; -const int FindByWorkflowExecRequest_Direction_Direction_ARRAYSIZE = FindByWorkflowExecRequest_Direction_Direction_MAX + 1; - -const ::google::protobuf::EnumDescriptor* FindByWorkflowExecRequest_Direction_descriptor(); -inline const ::std::string& FindByWorkflowExecRequest_Direction_Name(FindByWorkflowExecRequest_Direction value) { - return ::google::protobuf::internal::NameOfEnum( - FindByWorkflowExecRequest_Direction_descriptor(), value); -} -inline bool FindByWorkflowExecRequest_Direction_Parse( - const ::std::string& name, FindByWorkflowExecRequest_Direction* value) { - return ::google::protobuf::internal::ParseNamedEnum( - FindByWorkflowExecRequest_Direction_descriptor(), name, value); -} -// =================================================================== - -class Artifact final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.Artifact) */ { - public: - Artifact(); - virtual ~Artifact(); - - Artifact(const Artifact& from); - - inline Artifact& operator=(const Artifact& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - Artifact(Artifact&& from) noexcept - : Artifact() { - *this = ::std::move(from); - } - - inline Artifact& operator=(Artifact&& 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 Artifact& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const Artifact* internal_default_instance() { - return reinterpret_cast( - &_Artifact_default_instance_); - } - static constexpr int kIndexInFileMessages = - 0; - - void Swap(Artifact* other); - friend void swap(Artifact& a, Artifact& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline Artifact* New() const final { - return CreateMaybeMessage(nullptr); - } - - Artifact* 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 Artifact& from); - void MergeFrom(const Artifact& 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(Artifact* 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 ------------------------------------------------------- - - // repeated string tags = 3; - int tags_size() const; - void clear_tags(); - static const int kTagsFieldNumber = 3; - const ::std::string& tags(int index) const; - ::std::string* mutable_tags(int index); - void set_tags(int index, const ::std::string& value); - #if LANG_CXX11 - void set_tags(int index, ::std::string&& value); - #endif - void set_tags(int index, const char* value); - void set_tags(int index, const char* value, size_t size); - ::std::string* add_tags(); - void add_tags(const ::std::string& value); - #if LANG_CXX11 - void add_tags(::std::string&& value); - #endif - void add_tags(const char* value); - void add_tags(const char* value, size_t size); - const ::google::protobuf::RepeatedPtrField<::std::string>& tags() const; - ::google::protobuf::RepeatedPtrField<::std::string>* mutable_tags(); - - // .flyteidl.core.ArtifactID artifact_id = 1; - bool has_artifact_id() const; - void clear_artifact_id(); - static const int kArtifactIdFieldNumber = 1; - const ::flyteidl::core::ArtifactID& artifact_id() const; - ::flyteidl::core::ArtifactID* release_artifact_id(); - ::flyteidl::core::ArtifactID* mutable_artifact_id(); - void set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id); - - // .flyteidl.artifact.ArtifactSpec spec = 2; - bool has_spec() const; - void clear_spec(); - static const int kSpecFieldNumber = 2; - const ::flyteidl::artifact::ArtifactSpec& spec() const; - ::flyteidl::artifact::ArtifactSpec* release_spec(); - ::flyteidl::artifact::ArtifactSpec* mutable_spec(); - void set_allocated_spec(::flyteidl::artifact::ArtifactSpec* spec); - - // .flyteidl.artifact.ArtifactSource source = 4; - bool has_source() const; - void clear_source(); - static const int kSourceFieldNumber = 4; - const ::flyteidl::artifact::ArtifactSource& source() const; - ::flyteidl::artifact::ArtifactSource* release_source(); - ::flyteidl::artifact::ArtifactSource* mutable_source(); - void set_allocated_source(::flyteidl::artifact::ArtifactSource* source); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.Artifact) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField<::std::string> tags_; - ::flyteidl::core::ArtifactID* artifact_id_; - ::flyteidl::artifact::ArtifactSpec* spec_; - ::flyteidl::artifact::ArtifactSource* source_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class CreateArtifactRequest_PartitionsEntry_DoNotUse : public ::google::protobuf::internal::MapEntry { -public: -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -static bool _ParseMap(const char* begin, const char* end, void* object, ::google::protobuf::internal::ParseContext* ctx); -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - typedef ::google::protobuf::internal::MapEntry SuperType; - CreateArtifactRequest_PartitionsEntry_DoNotUse(); - CreateArtifactRequest_PartitionsEntry_DoNotUse(::google::protobuf::Arena* arena); - void MergeFrom(const CreateArtifactRequest_PartitionsEntry_DoNotUse& other); - static const CreateArtifactRequest_PartitionsEntry_DoNotUse* internal_default_instance() { return reinterpret_cast(&_CreateArtifactRequest_PartitionsEntry_DoNotUse_default_instance_); } - void MergeFrom(const ::google::protobuf::Message& other) final; - ::google::protobuf::Metadata GetMetadata() const; -}; - -// ------------------------------------------------------------------- - -class CreateArtifactRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.CreateArtifactRequest) */ { - public: - CreateArtifactRequest(); - virtual ~CreateArtifactRequest(); - - CreateArtifactRequest(const CreateArtifactRequest& from); - - inline CreateArtifactRequest& operator=(const CreateArtifactRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - CreateArtifactRequest(CreateArtifactRequest&& from) noexcept - : CreateArtifactRequest() { - *this = ::std::move(from); - } - - inline CreateArtifactRequest& operator=(CreateArtifactRequest&& 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 CreateArtifactRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const CreateArtifactRequest* internal_default_instance() { - return reinterpret_cast( - &_CreateArtifactRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 2; - - void Swap(CreateArtifactRequest* other); - friend void swap(CreateArtifactRequest& a, CreateArtifactRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline CreateArtifactRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - CreateArtifactRequest* 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 CreateArtifactRequest& from); - void MergeFrom(const CreateArtifactRequest& 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(CreateArtifactRequest* 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 ------------------------------------------------------- - - // map partitions = 4; - int partitions_size() const; - void clear_partitions(); - static const int kPartitionsFieldNumber = 4; - const ::google::protobuf::Map< ::std::string, ::std::string >& - partitions() const; - ::google::protobuf::Map< ::std::string, ::std::string >* - mutable_partitions(); - - // string version = 3; - void clear_version(); - static const int kVersionFieldNumber = 3; - const ::std::string& version() const; - void set_version(const ::std::string& value); - #if LANG_CXX11 - void set_version(::std::string&& value); - #endif - void set_version(const char* value); - void set_version(const char* value, size_t size); - ::std::string* mutable_version(); - ::std::string* release_version(); - void set_allocated_version(::std::string* version); - - // .flyteidl.core.ArtifactKey artifact_key = 1; - bool has_artifact_key() const; - void clear_artifact_key(); - static const int kArtifactKeyFieldNumber = 1; - const ::flyteidl::core::ArtifactKey& artifact_key() const; - ::flyteidl::core::ArtifactKey* release_artifact_key(); - ::flyteidl::core::ArtifactKey* mutable_artifact_key(); - void set_allocated_artifact_key(::flyteidl::core::ArtifactKey* artifact_key); - - // .flyteidl.artifact.ArtifactSpec spec = 2; - bool has_spec() const; - void clear_spec(); - static const int kSpecFieldNumber = 2; - const ::flyteidl::artifact::ArtifactSpec& spec() const; - ::flyteidl::artifact::ArtifactSpec* release_spec(); - ::flyteidl::artifact::ArtifactSpec* mutable_spec(); - void set_allocated_spec(::flyteidl::artifact::ArtifactSpec* spec); - - // .google.protobuf.Timestamp time_partition_value = 5; - bool has_time_partition_value() const; - void clear_time_partition_value(); - static const int kTimePartitionValueFieldNumber = 5; - const ::google::protobuf::Timestamp& time_partition_value() const; - ::google::protobuf::Timestamp* release_time_partition_value(); - ::google::protobuf::Timestamp* mutable_time_partition_value(); - void set_allocated_time_partition_value(::google::protobuf::Timestamp* time_partition_value); - - // .flyteidl.artifact.ArtifactSource source = 6; - bool has_source() const; - void clear_source(); - static const int kSourceFieldNumber = 6; - const ::flyteidl::artifact::ArtifactSource& source() const; - ::flyteidl::artifact::ArtifactSource* release_source(); - ::flyteidl::artifact::ArtifactSource* mutable_source(); - void set_allocated_source(::flyteidl::artifact::ArtifactSource* source); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateArtifactRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::internal::MapField< - CreateArtifactRequest_PartitionsEntry_DoNotUse, - ::std::string, ::std::string, - ::google::protobuf::internal::WireFormatLite::TYPE_STRING, - ::google::protobuf::internal::WireFormatLite::TYPE_STRING, - 0 > partitions_; - ::google::protobuf::internal::ArenaStringPtr version_; - ::flyteidl::core::ArtifactKey* artifact_key_; - ::flyteidl::artifact::ArtifactSpec* spec_; - ::google::protobuf::Timestamp* time_partition_value_; - ::flyteidl::artifact::ArtifactSource* source_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ArtifactSource final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ArtifactSource) */ { - public: - ArtifactSource(); - virtual ~ArtifactSource(); - - ArtifactSource(const ArtifactSource& from); - - inline ArtifactSource& operator=(const ArtifactSource& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ArtifactSource(ArtifactSource&& from) noexcept - : ArtifactSource() { - *this = ::std::move(from); - } - - inline ArtifactSource& operator=(ArtifactSource&& 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 ArtifactSource& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ArtifactSource* internal_default_instance() { - return reinterpret_cast( - &_ArtifactSource_default_instance_); - } - static constexpr int kIndexInFileMessages = - 3; - - void Swap(ArtifactSource* other); - friend void swap(ArtifactSource& a, ArtifactSource& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ArtifactSource* New() const final { - return CreateMaybeMessage(nullptr); - } - - ArtifactSource* 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 ArtifactSource& from); - void MergeFrom(const ArtifactSource& 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(ArtifactSource* 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 ------------------------------------------------------- - - // string node_id = 2; - void clear_node_id(); - static const int kNodeIdFieldNumber = 2; - const ::std::string& node_id() const; - void set_node_id(const ::std::string& value); - #if LANG_CXX11 - void set_node_id(::std::string&& value); - #endif - void set_node_id(const char* value); - void set_node_id(const char* value, size_t size); - ::std::string* mutable_node_id(); - ::std::string* release_node_id(); - void set_allocated_node_id(::std::string* node_id); - - // string principal = 5; - void clear_principal(); - static const int kPrincipalFieldNumber = 5; - const ::std::string& principal() const; - void set_principal(const ::std::string& value); - #if LANG_CXX11 - void set_principal(::std::string&& value); - #endif - void set_principal(const char* value); - void set_principal(const char* value, size_t size); - ::std::string* mutable_principal(); - ::std::string* release_principal(); - void set_allocated_principal(::std::string* principal); - - // .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - bool has_workflow_execution() const; - void clear_workflow_execution(); - static const int kWorkflowExecutionFieldNumber = 1; - const ::flyteidl::core::WorkflowExecutionIdentifier& workflow_execution() const; - ::flyteidl::core::WorkflowExecutionIdentifier* release_workflow_execution(); - ::flyteidl::core::WorkflowExecutionIdentifier* mutable_workflow_execution(); - void set_allocated_workflow_execution(::flyteidl::core::WorkflowExecutionIdentifier* workflow_execution); - - // .flyteidl.core.Identifier task_id = 3; - bool has_task_id() const; - void clear_task_id(); - static const int kTaskIdFieldNumber = 3; - const ::flyteidl::core::Identifier& task_id() const; - ::flyteidl::core::Identifier* release_task_id(); - ::flyteidl::core::Identifier* mutable_task_id(); - void set_allocated_task_id(::flyteidl::core::Identifier* task_id); - - // uint32 retry_attempt = 4; - void clear_retry_attempt(); - static const int kRetryAttemptFieldNumber = 4; - ::google::protobuf::uint32 retry_attempt() const; - void set_retry_attempt(::google::protobuf::uint32 value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactSource) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::internal::ArenaStringPtr node_id_; - ::google::protobuf::internal::ArenaStringPtr principal_; - ::flyteidl::core::WorkflowExecutionIdentifier* workflow_execution_; - ::flyteidl::core::Identifier* task_id_; - ::google::protobuf::uint32 retry_attempt_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ArtifactSpec final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ArtifactSpec) */ { - public: - ArtifactSpec(); - virtual ~ArtifactSpec(); - - ArtifactSpec(const ArtifactSpec& from); - - inline ArtifactSpec& operator=(const ArtifactSpec& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ArtifactSpec(ArtifactSpec&& from) noexcept - : ArtifactSpec() { - *this = ::std::move(from); - } - - inline ArtifactSpec& operator=(ArtifactSpec&& 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 ArtifactSpec& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ArtifactSpec* internal_default_instance() { - return reinterpret_cast( - &_ArtifactSpec_default_instance_); - } - static constexpr int kIndexInFileMessages = - 4; - - void Swap(ArtifactSpec* other); - friend void swap(ArtifactSpec& a, ArtifactSpec& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ArtifactSpec* New() const final { - return CreateMaybeMessage(nullptr); - } - - ArtifactSpec* 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 ArtifactSpec& from); - void MergeFrom(const ArtifactSpec& 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(ArtifactSpec* 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 ------------------------------------------------------- - - // string short_description = 3; - void clear_short_description(); - static const int kShortDescriptionFieldNumber = 3; - const ::std::string& short_description() const; - void set_short_description(const ::std::string& value); - #if LANG_CXX11 - void set_short_description(::std::string&& value); - #endif - void set_short_description(const char* value); - void set_short_description(const char* value, size_t size); - ::std::string* mutable_short_description(); - ::std::string* release_short_description(); - void set_allocated_short_description(::std::string* short_description); - - // string metadata_type = 5; - void clear_metadata_type(); - static const int kMetadataTypeFieldNumber = 5; - const ::std::string& metadata_type() const; - void set_metadata_type(const ::std::string& value); - #if LANG_CXX11 - void set_metadata_type(::std::string&& value); - #endif - void set_metadata_type(const char* value); - void set_metadata_type(const char* value, size_t size); - ::std::string* mutable_metadata_type(); - ::std::string* release_metadata_type(); - void set_allocated_metadata_type(::std::string* metadata_type); - - // string file_format = 7; - void clear_file_format(); - static const int kFileFormatFieldNumber = 7; - const ::std::string& file_format() const; - void set_file_format(const ::std::string& value); - #if LANG_CXX11 - void set_file_format(::std::string&& value); - #endif - void set_file_format(const char* value); - void set_file_format(const char* value, size_t size); - ::std::string* mutable_file_format(); - ::std::string* release_file_format(); - void set_allocated_file_format(::std::string* file_format); - - // .flyteidl.core.Literal value = 1; - bool has_value() const; - void clear_value(); - static const int kValueFieldNumber = 1; - const ::flyteidl::core::Literal& value() const; - ::flyteidl::core::Literal* release_value(); - ::flyteidl::core::Literal* mutable_value(); - void set_allocated_value(::flyteidl::core::Literal* value); - - // .flyteidl.core.LiteralType type = 2; - bool has_type() const; - void clear_type(); - static const int kTypeFieldNumber = 2; - const ::flyteidl::core::LiteralType& type() const; - ::flyteidl::core::LiteralType* release_type(); - ::flyteidl::core::LiteralType* mutable_type(); - void set_allocated_type(::flyteidl::core::LiteralType* type); - - // .google.protobuf.Any user_metadata = 4; - bool has_user_metadata() const; - void clear_user_metadata(); - static const int kUserMetadataFieldNumber = 4; - const ::google::protobuf::Any& user_metadata() const; - ::google::protobuf::Any* release_user_metadata(); - ::google::protobuf::Any* mutable_user_metadata(); - void set_allocated_user_metadata(::google::protobuf::Any* user_metadata); - - // .google.protobuf.Timestamp created_at = 6; - bool has_created_at() const; - void clear_created_at(); - static const int kCreatedAtFieldNumber = 6; - const ::google::protobuf::Timestamp& created_at() const; - ::google::protobuf::Timestamp* release_created_at(); - ::google::protobuf::Timestamp* mutable_created_at(); - void set_allocated_created_at(::google::protobuf::Timestamp* created_at); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactSpec) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::internal::ArenaStringPtr short_description_; - ::google::protobuf::internal::ArenaStringPtr metadata_type_; - ::google::protobuf::internal::ArenaStringPtr file_format_; - ::flyteidl::core::Literal* value_; - ::flyteidl::core::LiteralType* type_; - ::google::protobuf::Any* user_metadata_; - ::google::protobuf::Timestamp* created_at_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class CreateArtifactResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.CreateArtifactResponse) */ { - public: - CreateArtifactResponse(); - virtual ~CreateArtifactResponse(); - - CreateArtifactResponse(const CreateArtifactResponse& from); - - inline CreateArtifactResponse& operator=(const CreateArtifactResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - CreateArtifactResponse(CreateArtifactResponse&& from) noexcept - : CreateArtifactResponse() { - *this = ::std::move(from); - } - - inline CreateArtifactResponse& operator=(CreateArtifactResponse&& 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 CreateArtifactResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const CreateArtifactResponse* internal_default_instance() { - return reinterpret_cast( - &_CreateArtifactResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 5; - - void Swap(CreateArtifactResponse* other); - friend void swap(CreateArtifactResponse& a, CreateArtifactResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline CreateArtifactResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - CreateArtifactResponse* 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 CreateArtifactResponse& from); - void MergeFrom(const CreateArtifactResponse& 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(CreateArtifactResponse* 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.artifact.Artifact artifact = 1; - bool has_artifact() const; - void clear_artifact(); - static const int kArtifactFieldNumber = 1; - const ::flyteidl::artifact::Artifact& artifact() const; - ::flyteidl::artifact::Artifact* release_artifact(); - ::flyteidl::artifact::Artifact* mutable_artifact(); - void set_allocated_artifact(::flyteidl::artifact::Artifact* artifact); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateArtifactResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::artifact::Artifact* artifact_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class GetArtifactRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.GetArtifactRequest) */ { - public: - GetArtifactRequest(); - virtual ~GetArtifactRequest(); - - GetArtifactRequest(const GetArtifactRequest& from); - - inline GetArtifactRequest& operator=(const GetArtifactRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - GetArtifactRequest(GetArtifactRequest&& from) noexcept - : GetArtifactRequest() { - *this = ::std::move(from); - } - - inline GetArtifactRequest& operator=(GetArtifactRequest&& 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 GetArtifactRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const GetArtifactRequest* internal_default_instance() { - return reinterpret_cast( - &_GetArtifactRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 6; - - void Swap(GetArtifactRequest* other); - friend void swap(GetArtifactRequest& a, GetArtifactRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline GetArtifactRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - GetArtifactRequest* 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 GetArtifactRequest& from); - void MergeFrom(const GetArtifactRequest& 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(GetArtifactRequest* 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.ArtifactQuery query = 1; - bool has_query() const; - void clear_query(); - static const int kQueryFieldNumber = 1; - const ::flyteidl::core::ArtifactQuery& query() const; - ::flyteidl::core::ArtifactQuery* release_query(); - ::flyteidl::core::ArtifactQuery* mutable_query(); - void set_allocated_query(::flyteidl::core::ArtifactQuery* query); - - // bool details = 2; - void clear_details(); - static const int kDetailsFieldNumber = 2; - bool details() const; - void set_details(bool value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.GetArtifactRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::ArtifactQuery* query_; - bool details_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class GetArtifactResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.GetArtifactResponse) */ { - public: - GetArtifactResponse(); - virtual ~GetArtifactResponse(); - - GetArtifactResponse(const GetArtifactResponse& from); - - inline GetArtifactResponse& operator=(const GetArtifactResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - GetArtifactResponse(GetArtifactResponse&& from) noexcept - : GetArtifactResponse() { - *this = ::std::move(from); - } - - inline GetArtifactResponse& operator=(GetArtifactResponse&& 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 GetArtifactResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const GetArtifactResponse* internal_default_instance() { - return reinterpret_cast( - &_GetArtifactResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 7; - - void Swap(GetArtifactResponse* other); - friend void swap(GetArtifactResponse& a, GetArtifactResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline GetArtifactResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - GetArtifactResponse* 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 GetArtifactResponse& from); - void MergeFrom(const GetArtifactResponse& 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(GetArtifactResponse* 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.artifact.Artifact artifact = 1; - bool has_artifact() const; - void clear_artifact(); - static const int kArtifactFieldNumber = 1; - const ::flyteidl::artifact::Artifact& artifact() const; - ::flyteidl::artifact::Artifact* release_artifact(); - ::flyteidl::artifact::Artifact* mutable_artifact(); - void set_allocated_artifact(::flyteidl::artifact::Artifact* artifact); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.GetArtifactResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::artifact::Artifact* artifact_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class SearchOptions final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.SearchOptions) */ { - public: - SearchOptions(); - virtual ~SearchOptions(); - - SearchOptions(const SearchOptions& from); - - inline SearchOptions& operator=(const SearchOptions& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - SearchOptions(SearchOptions&& from) noexcept - : SearchOptions() { - *this = ::std::move(from); - } - - inline SearchOptions& operator=(SearchOptions&& 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 SearchOptions& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const SearchOptions* internal_default_instance() { - return reinterpret_cast( - &_SearchOptions_default_instance_); - } - static constexpr int kIndexInFileMessages = - 8; - - void Swap(SearchOptions* other); - friend void swap(SearchOptions& a, SearchOptions& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline SearchOptions* New() const final { - return CreateMaybeMessage(nullptr); - } - - SearchOptions* 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 SearchOptions& from); - void MergeFrom(const SearchOptions& 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(SearchOptions* 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 ------------------------------------------------------- - - // bool strict_partitions = 1; - void clear_strict_partitions(); - static const int kStrictPartitionsFieldNumber = 1; - bool strict_partitions() const; - void set_strict_partitions(bool value); - - // bool latest_by_key = 2; - void clear_latest_by_key(); - static const int kLatestByKeyFieldNumber = 2; - bool latest_by_key() const; - void set_latest_by_key(bool value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchOptions) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - bool strict_partitions_; - bool latest_by_key_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class SearchArtifactsRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.SearchArtifactsRequest) */ { - public: - SearchArtifactsRequest(); - virtual ~SearchArtifactsRequest(); - - SearchArtifactsRequest(const SearchArtifactsRequest& from); - - inline SearchArtifactsRequest& operator=(const SearchArtifactsRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - SearchArtifactsRequest(SearchArtifactsRequest&& from) noexcept - : SearchArtifactsRequest() { - *this = ::std::move(from); - } - - inline SearchArtifactsRequest& operator=(SearchArtifactsRequest&& 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 SearchArtifactsRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const SearchArtifactsRequest* internal_default_instance() { - return reinterpret_cast( - &_SearchArtifactsRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 9; - - void Swap(SearchArtifactsRequest* other); - friend void swap(SearchArtifactsRequest& a, SearchArtifactsRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline SearchArtifactsRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - SearchArtifactsRequest* 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 SearchArtifactsRequest& from); - void MergeFrom(const SearchArtifactsRequest& 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(SearchArtifactsRequest* 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 ------------------------------------------------------- - - // string principal = 4; - void clear_principal(); - static const int kPrincipalFieldNumber = 4; - const ::std::string& principal() const; - void set_principal(const ::std::string& value); - #if LANG_CXX11 - void set_principal(::std::string&& value); - #endif - void set_principal(const char* value); - void set_principal(const char* value, size_t size); - ::std::string* mutable_principal(); - ::std::string* release_principal(); - void set_allocated_principal(::std::string* principal); - - // string version = 5; - void clear_version(); - static const int kVersionFieldNumber = 5; - const ::std::string& version() const; - void set_version(const ::std::string& value); - #if LANG_CXX11 - void set_version(::std::string&& value); - #endif - void set_version(const char* value); - void set_version(const char* value, size_t size); - ::std::string* mutable_version(); - ::std::string* release_version(); - void set_allocated_version(::std::string* version); - - // string token = 7; - void clear_token(); - static const int kTokenFieldNumber = 7; - const ::std::string& token() const; - void set_token(const ::std::string& value); - #if LANG_CXX11 - void set_token(::std::string&& value); - #endif - void set_token(const char* value); - void set_token(const char* value, size_t size); - ::std::string* mutable_token(); - ::std::string* release_token(); - void set_allocated_token(::std::string* token); - - // .flyteidl.core.ArtifactKey artifact_key = 1; - bool has_artifact_key() const; - void clear_artifact_key(); - static const int kArtifactKeyFieldNumber = 1; - const ::flyteidl::core::ArtifactKey& artifact_key() const; - ::flyteidl::core::ArtifactKey* release_artifact_key(); - ::flyteidl::core::ArtifactKey* mutable_artifact_key(); - void set_allocated_artifact_key(::flyteidl::core::ArtifactKey* artifact_key); - - // .flyteidl.core.Partitions partitions = 2; - bool has_partitions() const; - void clear_partitions(); - static const int kPartitionsFieldNumber = 2; - const ::flyteidl::core::Partitions& partitions() const; - ::flyteidl::core::Partitions* release_partitions(); - ::flyteidl::core::Partitions* mutable_partitions(); - void set_allocated_partitions(::flyteidl::core::Partitions* partitions); - - // .google.protobuf.Timestamp time_partition_value = 3; - bool has_time_partition_value() const; - void clear_time_partition_value(); - static const int kTimePartitionValueFieldNumber = 3; - const ::google::protobuf::Timestamp& time_partition_value() const; - ::google::protobuf::Timestamp* release_time_partition_value(); - ::google::protobuf::Timestamp* mutable_time_partition_value(); - void set_allocated_time_partition_value(::google::protobuf::Timestamp* time_partition_value); - - // .flyteidl.artifact.SearchOptions options = 6; - bool has_options() const; - void clear_options(); - static const int kOptionsFieldNumber = 6; - const ::flyteidl::artifact::SearchOptions& options() const; - ::flyteidl::artifact::SearchOptions* release_options(); - ::flyteidl::artifact::SearchOptions* mutable_options(); - void set_allocated_options(::flyteidl::artifact::SearchOptions* options); - - // int32 limit = 8; - void clear_limit(); - static const int kLimitFieldNumber = 8; - ::google::protobuf::int32 limit() const; - void set_limit(::google::protobuf::int32 value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchArtifactsRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::internal::ArenaStringPtr principal_; - ::google::protobuf::internal::ArenaStringPtr version_; - ::google::protobuf::internal::ArenaStringPtr token_; - ::flyteidl::core::ArtifactKey* artifact_key_; - ::flyteidl::core::Partitions* partitions_; - ::google::protobuf::Timestamp* time_partition_value_; - ::flyteidl::artifact::SearchOptions* options_; - ::google::protobuf::int32 limit_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class SearchArtifactsResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.SearchArtifactsResponse) */ { - public: - SearchArtifactsResponse(); - virtual ~SearchArtifactsResponse(); - - SearchArtifactsResponse(const SearchArtifactsResponse& from); - - inline SearchArtifactsResponse& operator=(const SearchArtifactsResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - SearchArtifactsResponse(SearchArtifactsResponse&& from) noexcept - : SearchArtifactsResponse() { - *this = ::std::move(from); - } - - inline SearchArtifactsResponse& operator=(SearchArtifactsResponse&& 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 SearchArtifactsResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const SearchArtifactsResponse* internal_default_instance() { - return reinterpret_cast( - &_SearchArtifactsResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 10; - - void Swap(SearchArtifactsResponse* other); - friend void swap(SearchArtifactsResponse& a, SearchArtifactsResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline SearchArtifactsResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - SearchArtifactsResponse* 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 SearchArtifactsResponse& from); - void MergeFrom(const SearchArtifactsResponse& 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(SearchArtifactsResponse* 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 ------------------------------------------------------- - - // repeated .flyteidl.artifact.Artifact artifacts = 1; - int artifacts_size() const; - void clear_artifacts(); - static const int kArtifactsFieldNumber = 1; - ::flyteidl::artifact::Artifact* mutable_artifacts(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::Artifact >* - mutable_artifacts(); - const ::flyteidl::artifact::Artifact& artifacts(int index) const; - ::flyteidl::artifact::Artifact* add_artifacts(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::Artifact >& - artifacts() const; - - // string token = 2; - void clear_token(); - static const int kTokenFieldNumber = 2; - const ::std::string& token() const; - void set_token(const ::std::string& value); - #if LANG_CXX11 - void set_token(::std::string&& value); - #endif - void set_token(const char* value); - void set_token(const char* value, size_t size); - ::std::string* mutable_token(); - ::std::string* release_token(); - void set_allocated_token(::std::string* token); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchArtifactsResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::Artifact > artifacts_; - ::google::protobuf::internal::ArenaStringPtr token_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class FindByWorkflowExecRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.FindByWorkflowExecRequest) */ { - public: - FindByWorkflowExecRequest(); - virtual ~FindByWorkflowExecRequest(); - - FindByWorkflowExecRequest(const FindByWorkflowExecRequest& from); - - inline FindByWorkflowExecRequest& operator=(const FindByWorkflowExecRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - FindByWorkflowExecRequest(FindByWorkflowExecRequest&& from) noexcept - : FindByWorkflowExecRequest() { - *this = ::std::move(from); - } - - inline FindByWorkflowExecRequest& operator=(FindByWorkflowExecRequest&& 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 FindByWorkflowExecRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const FindByWorkflowExecRequest* internal_default_instance() { - return reinterpret_cast( - &_FindByWorkflowExecRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 11; - - void Swap(FindByWorkflowExecRequest* other); - friend void swap(FindByWorkflowExecRequest& a, FindByWorkflowExecRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline FindByWorkflowExecRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - FindByWorkflowExecRequest* 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 FindByWorkflowExecRequest& from); - void MergeFrom(const FindByWorkflowExecRequest& 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(FindByWorkflowExecRequest* 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 ---------------------------------------------------- - - typedef FindByWorkflowExecRequest_Direction Direction; - static const Direction INPUTS = - FindByWorkflowExecRequest_Direction_INPUTS; - static const Direction OUTPUTS = - FindByWorkflowExecRequest_Direction_OUTPUTS; - static inline bool Direction_IsValid(int value) { - return FindByWorkflowExecRequest_Direction_IsValid(value); - } - static const Direction Direction_MIN = - FindByWorkflowExecRequest_Direction_Direction_MIN; - static const Direction Direction_MAX = - FindByWorkflowExecRequest_Direction_Direction_MAX; - static const int Direction_ARRAYSIZE = - FindByWorkflowExecRequest_Direction_Direction_ARRAYSIZE; - static inline const ::google::protobuf::EnumDescriptor* - Direction_descriptor() { - return FindByWorkflowExecRequest_Direction_descriptor(); - } - static inline const ::std::string& Direction_Name(Direction value) { - return FindByWorkflowExecRequest_Direction_Name(value); - } - static inline bool Direction_Parse(const ::std::string& name, - Direction* value) { - return FindByWorkflowExecRequest_Direction_Parse(name, value); - } - - // accessors ------------------------------------------------------- - - // .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - bool has_exec_id() const; - void clear_exec_id(); - static const int kExecIdFieldNumber = 1; - const ::flyteidl::core::WorkflowExecutionIdentifier& exec_id() const; - ::flyteidl::core::WorkflowExecutionIdentifier* release_exec_id(); - ::flyteidl::core::WorkflowExecutionIdentifier* mutable_exec_id(); - void set_allocated_exec_id(::flyteidl::core::WorkflowExecutionIdentifier* exec_id); - - // .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - void clear_direction(); - static const int kDirectionFieldNumber = 2; - ::flyteidl::artifact::FindByWorkflowExecRequest_Direction direction() const; - void set_direction(::flyteidl::artifact::FindByWorkflowExecRequest_Direction value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.FindByWorkflowExecRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::WorkflowExecutionIdentifier* exec_id_; - int direction_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class AddTagRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.AddTagRequest) */ { - public: - AddTagRequest(); - virtual ~AddTagRequest(); - - AddTagRequest(const AddTagRequest& from); - - inline AddTagRequest& operator=(const AddTagRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - AddTagRequest(AddTagRequest&& from) noexcept - : AddTagRequest() { - *this = ::std::move(from); - } - - inline AddTagRequest& operator=(AddTagRequest&& 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 AddTagRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const AddTagRequest* internal_default_instance() { - return reinterpret_cast( - &_AddTagRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 12; - - void Swap(AddTagRequest* other); - friend void swap(AddTagRequest& a, AddTagRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline AddTagRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - AddTagRequest* 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 AddTagRequest& from); - void MergeFrom(const AddTagRequest& 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(AddTagRequest* 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 ------------------------------------------------------- - - // string value = 2; - void clear_value(); - static const int kValueFieldNumber = 2; - const ::std::string& value() const; - void set_value(const ::std::string& value); - #if LANG_CXX11 - void set_value(::std::string&& value); - #endif - void set_value(const char* value); - void set_value(const char* value, size_t size); - ::std::string* mutable_value(); - ::std::string* release_value(); - void set_allocated_value(::std::string* value); - - // .flyteidl.core.ArtifactID artifact_id = 1; - bool has_artifact_id() const; - void clear_artifact_id(); - static const int kArtifactIdFieldNumber = 1; - const ::flyteidl::core::ArtifactID& artifact_id() const; - ::flyteidl::core::ArtifactID* release_artifact_id(); - ::flyteidl::core::ArtifactID* mutable_artifact_id(); - void set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id); - - // bool overwrite = 3; - void clear_overwrite(); - static const int kOverwriteFieldNumber = 3; - bool overwrite() const; - void set_overwrite(bool value); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.AddTagRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::internal::ArenaStringPtr value_; - ::flyteidl::core::ArtifactID* artifact_id_; - bool overwrite_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class AddTagResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.AddTagResponse) */ { - public: - AddTagResponse(); - virtual ~AddTagResponse(); - - AddTagResponse(const AddTagResponse& from); - - inline AddTagResponse& operator=(const AddTagResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - AddTagResponse(AddTagResponse&& from) noexcept - : AddTagResponse() { - *this = ::std::move(from); - } - - inline AddTagResponse& operator=(AddTagResponse&& 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 AddTagResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const AddTagResponse* internal_default_instance() { - return reinterpret_cast( - &_AddTagResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 13; - - void Swap(AddTagResponse* other); - friend void swap(AddTagResponse& a, AddTagResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline AddTagResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - AddTagResponse* 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 AddTagResponse& from); - void MergeFrom(const AddTagResponse& 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(AddTagResponse* 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 ------------------------------------------------------- - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.AddTagResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class CreateTriggerRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.CreateTriggerRequest) */ { - public: - CreateTriggerRequest(); - virtual ~CreateTriggerRequest(); - - CreateTriggerRequest(const CreateTriggerRequest& from); - - inline CreateTriggerRequest& operator=(const CreateTriggerRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - CreateTriggerRequest(CreateTriggerRequest&& from) noexcept - : CreateTriggerRequest() { - *this = ::std::move(from); - } - - inline CreateTriggerRequest& operator=(CreateTriggerRequest&& 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 CreateTriggerRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const CreateTriggerRequest* internal_default_instance() { - return reinterpret_cast( - &_CreateTriggerRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 14; - - void Swap(CreateTriggerRequest* other); - friend void swap(CreateTriggerRequest& a, CreateTriggerRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline CreateTriggerRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - CreateTriggerRequest* 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 CreateTriggerRequest& from); - void MergeFrom(const CreateTriggerRequest& 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(CreateTriggerRequest* 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.admin.LaunchPlan trigger_launch_plan = 1; - bool has_trigger_launch_plan() const; - void clear_trigger_launch_plan(); - static const int kTriggerLaunchPlanFieldNumber = 1; - const ::flyteidl::admin::LaunchPlan& trigger_launch_plan() const; - ::flyteidl::admin::LaunchPlan* release_trigger_launch_plan(); - ::flyteidl::admin::LaunchPlan* mutable_trigger_launch_plan(); - void set_allocated_trigger_launch_plan(::flyteidl::admin::LaunchPlan* trigger_launch_plan); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateTriggerRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::admin::LaunchPlan* trigger_launch_plan_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class CreateTriggerResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.CreateTriggerResponse) */ { - public: - CreateTriggerResponse(); - virtual ~CreateTriggerResponse(); - - CreateTriggerResponse(const CreateTriggerResponse& from); - - inline CreateTriggerResponse& operator=(const CreateTriggerResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - CreateTriggerResponse(CreateTriggerResponse&& from) noexcept - : CreateTriggerResponse() { - *this = ::std::move(from); - } - - inline CreateTriggerResponse& operator=(CreateTriggerResponse&& 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 CreateTriggerResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const CreateTriggerResponse* internal_default_instance() { - return reinterpret_cast( - &_CreateTriggerResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 15; - - void Swap(CreateTriggerResponse* other); - friend void swap(CreateTriggerResponse& a, CreateTriggerResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline CreateTriggerResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - CreateTriggerResponse* 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 CreateTriggerResponse& from); - void MergeFrom(const CreateTriggerResponse& 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(CreateTriggerResponse* 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 ------------------------------------------------------- - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateTriggerResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class DeactivateTriggerRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.DeactivateTriggerRequest) */ { - public: - DeactivateTriggerRequest(); - virtual ~DeactivateTriggerRequest(); - - DeactivateTriggerRequest(const DeactivateTriggerRequest& from); - - inline DeactivateTriggerRequest& operator=(const DeactivateTriggerRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - DeactivateTriggerRequest(DeactivateTriggerRequest&& from) noexcept - : DeactivateTriggerRequest() { - *this = ::std::move(from); - } - - inline DeactivateTriggerRequest& operator=(DeactivateTriggerRequest&& 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 DeactivateTriggerRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const DeactivateTriggerRequest* internal_default_instance() { - return reinterpret_cast( - &_DeactivateTriggerRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 16; - - void Swap(DeactivateTriggerRequest* other); - friend void swap(DeactivateTriggerRequest& a, DeactivateTriggerRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline DeactivateTriggerRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - DeactivateTriggerRequest* 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 DeactivateTriggerRequest& from); - void MergeFrom(const DeactivateTriggerRequest& 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(DeactivateTriggerRequest* 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.Identifier trigger_id = 1; - bool has_trigger_id() const; - void clear_trigger_id(); - static const int kTriggerIdFieldNumber = 1; - const ::flyteidl::core::Identifier& trigger_id() const; - ::flyteidl::core::Identifier* release_trigger_id(); - ::flyteidl::core::Identifier* mutable_trigger_id(); - void set_allocated_trigger_id(::flyteidl::core::Identifier* trigger_id); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.DeactivateTriggerRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::Identifier* trigger_id_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class DeactivateTriggerResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.DeactivateTriggerResponse) */ { - public: - DeactivateTriggerResponse(); - virtual ~DeactivateTriggerResponse(); - - DeactivateTriggerResponse(const DeactivateTriggerResponse& from); - - inline DeactivateTriggerResponse& operator=(const DeactivateTriggerResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - DeactivateTriggerResponse(DeactivateTriggerResponse&& from) noexcept - : DeactivateTriggerResponse() { - *this = ::std::move(from); - } - - inline DeactivateTriggerResponse& operator=(DeactivateTriggerResponse&& 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 DeactivateTriggerResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const DeactivateTriggerResponse* internal_default_instance() { - return reinterpret_cast( - &_DeactivateTriggerResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 17; - - void Swap(DeactivateTriggerResponse* other); - friend void swap(DeactivateTriggerResponse& a, DeactivateTriggerResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline DeactivateTriggerResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - DeactivateTriggerResponse* 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 DeactivateTriggerResponse& from); - void MergeFrom(const DeactivateTriggerResponse& 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(DeactivateTriggerResponse* 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 ------------------------------------------------------- - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.DeactivateTriggerResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ArtifactProducer final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ArtifactProducer) */ { - public: - ArtifactProducer(); - virtual ~ArtifactProducer(); - - ArtifactProducer(const ArtifactProducer& from); - - inline ArtifactProducer& operator=(const ArtifactProducer& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ArtifactProducer(ArtifactProducer&& from) noexcept - : ArtifactProducer() { - *this = ::std::move(from); - } - - inline ArtifactProducer& operator=(ArtifactProducer&& 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 ArtifactProducer& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ArtifactProducer* internal_default_instance() { - return reinterpret_cast( - &_ArtifactProducer_default_instance_); - } - static constexpr int kIndexInFileMessages = - 18; - - void Swap(ArtifactProducer* other); - friend void swap(ArtifactProducer& a, ArtifactProducer& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ArtifactProducer* New() const final { - return CreateMaybeMessage(nullptr); - } - - ArtifactProducer* 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 ArtifactProducer& from); - void MergeFrom(const ArtifactProducer& 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(ArtifactProducer* 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.Identifier entity_id = 1; - bool has_entity_id() const; - void clear_entity_id(); - static const int kEntityIdFieldNumber = 1; - const ::flyteidl::core::Identifier& entity_id() const; - ::flyteidl::core::Identifier* release_entity_id(); - ::flyteidl::core::Identifier* mutable_entity_id(); - void set_allocated_entity_id(::flyteidl::core::Identifier* entity_id); - - // .flyteidl.core.VariableMap outputs = 2; - bool has_outputs() const; - void clear_outputs(); - static const int kOutputsFieldNumber = 2; - const ::flyteidl::core::VariableMap& outputs() const; - ::flyteidl::core::VariableMap* release_outputs(); - ::flyteidl::core::VariableMap* mutable_outputs(); - void set_allocated_outputs(::flyteidl::core::VariableMap* outputs); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactProducer) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::Identifier* entity_id_; - ::flyteidl::core::VariableMap* outputs_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class RegisterProducerRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.RegisterProducerRequest) */ { - public: - RegisterProducerRequest(); - virtual ~RegisterProducerRequest(); - - RegisterProducerRequest(const RegisterProducerRequest& from); - - inline RegisterProducerRequest& operator=(const RegisterProducerRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - RegisterProducerRequest(RegisterProducerRequest&& from) noexcept - : RegisterProducerRequest() { - *this = ::std::move(from); - } - - inline RegisterProducerRequest& operator=(RegisterProducerRequest&& 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 RegisterProducerRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const RegisterProducerRequest* internal_default_instance() { - return reinterpret_cast( - &_RegisterProducerRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 19; - - void Swap(RegisterProducerRequest* other); - friend void swap(RegisterProducerRequest& a, RegisterProducerRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline RegisterProducerRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - RegisterProducerRequest* 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 RegisterProducerRequest& from); - void MergeFrom(const RegisterProducerRequest& 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(RegisterProducerRequest* 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 ------------------------------------------------------- - - // repeated .flyteidl.artifact.ArtifactProducer producers = 1; - int producers_size() const; - void clear_producers(); - static const int kProducersFieldNumber = 1; - ::flyteidl::artifact::ArtifactProducer* mutable_producers(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactProducer >* - mutable_producers(); - const ::flyteidl::artifact::ArtifactProducer& producers(int index) const; - ::flyteidl::artifact::ArtifactProducer* add_producers(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactProducer >& - producers() const; - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterProducerRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactProducer > producers_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ArtifactConsumer final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ArtifactConsumer) */ { - public: - ArtifactConsumer(); - virtual ~ArtifactConsumer(); - - ArtifactConsumer(const ArtifactConsumer& from); - - inline ArtifactConsumer& operator=(const ArtifactConsumer& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ArtifactConsumer(ArtifactConsumer&& from) noexcept - : ArtifactConsumer() { - *this = ::std::move(from); - } - - inline ArtifactConsumer& operator=(ArtifactConsumer&& 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 ArtifactConsumer& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ArtifactConsumer* internal_default_instance() { - return reinterpret_cast( - &_ArtifactConsumer_default_instance_); - } - static constexpr int kIndexInFileMessages = - 20; - - void Swap(ArtifactConsumer* other); - friend void swap(ArtifactConsumer& a, ArtifactConsumer& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ArtifactConsumer* New() const final { - return CreateMaybeMessage(nullptr); - } - - ArtifactConsumer* 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 ArtifactConsumer& from); - void MergeFrom(const ArtifactConsumer& 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(ArtifactConsumer* 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.Identifier entity_id = 1; - bool has_entity_id() const; - void clear_entity_id(); - static const int kEntityIdFieldNumber = 1; - const ::flyteidl::core::Identifier& entity_id() const; - ::flyteidl::core::Identifier* release_entity_id(); - ::flyteidl::core::Identifier* mutable_entity_id(); - void set_allocated_entity_id(::flyteidl::core::Identifier* entity_id); - - // .flyteidl.core.ParameterMap inputs = 2; - bool has_inputs() const; - void clear_inputs(); - static const int kInputsFieldNumber = 2; - const ::flyteidl::core::ParameterMap& inputs() const; - ::flyteidl::core::ParameterMap* release_inputs(); - ::flyteidl::core::ParameterMap* mutable_inputs(); - void set_allocated_inputs(::flyteidl::core::ParameterMap* inputs); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactConsumer) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::Identifier* entity_id_; - ::flyteidl::core::ParameterMap* inputs_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class RegisterConsumerRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.RegisterConsumerRequest) */ { - public: - RegisterConsumerRequest(); - virtual ~RegisterConsumerRequest(); - - RegisterConsumerRequest(const RegisterConsumerRequest& from); - - inline RegisterConsumerRequest& operator=(const RegisterConsumerRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - RegisterConsumerRequest(RegisterConsumerRequest&& from) noexcept - : RegisterConsumerRequest() { - *this = ::std::move(from); - } - - inline RegisterConsumerRequest& operator=(RegisterConsumerRequest&& 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 RegisterConsumerRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const RegisterConsumerRequest* internal_default_instance() { - return reinterpret_cast( - &_RegisterConsumerRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 21; - - void Swap(RegisterConsumerRequest* other); - friend void swap(RegisterConsumerRequest& a, RegisterConsumerRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline RegisterConsumerRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - RegisterConsumerRequest* 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 RegisterConsumerRequest& from); - void MergeFrom(const RegisterConsumerRequest& 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(RegisterConsumerRequest* 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 ------------------------------------------------------- - - // repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - int consumers_size() const; - void clear_consumers(); - static const int kConsumersFieldNumber = 1; - ::flyteidl::artifact::ArtifactConsumer* mutable_consumers(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactConsumer >* - mutable_consumers(); - const ::flyteidl::artifact::ArtifactConsumer& consumers(int index) const; - ::flyteidl::artifact::ArtifactConsumer* add_consumers(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactConsumer >& - consumers() const; - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterConsumerRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactConsumer > consumers_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class RegisterResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.RegisterResponse) */ { - public: - RegisterResponse(); - virtual ~RegisterResponse(); - - RegisterResponse(const RegisterResponse& from); - - inline RegisterResponse& operator=(const RegisterResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - RegisterResponse(RegisterResponse&& from) noexcept - : RegisterResponse() { - *this = ::std::move(from); - } - - inline RegisterResponse& operator=(RegisterResponse&& 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 RegisterResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const RegisterResponse* internal_default_instance() { - return reinterpret_cast( - &_RegisterResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 22; - - void Swap(RegisterResponse* other); - friend void swap(RegisterResponse& a, RegisterResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline RegisterResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - RegisterResponse* 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 RegisterResponse& from); - void MergeFrom(const RegisterResponse& 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(RegisterResponse* 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 ------------------------------------------------------- - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ExecutionInputsRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ExecutionInputsRequest) */ { - public: - ExecutionInputsRequest(); - virtual ~ExecutionInputsRequest(); - - ExecutionInputsRequest(const ExecutionInputsRequest& from); - - inline ExecutionInputsRequest& operator=(const ExecutionInputsRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ExecutionInputsRequest(ExecutionInputsRequest&& from) noexcept - : ExecutionInputsRequest() { - *this = ::std::move(from); - } - - inline ExecutionInputsRequest& operator=(ExecutionInputsRequest&& 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 ExecutionInputsRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ExecutionInputsRequest* internal_default_instance() { - return reinterpret_cast( - &_ExecutionInputsRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 23; - - void Swap(ExecutionInputsRequest* other); - friend void swap(ExecutionInputsRequest& a, ExecutionInputsRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ExecutionInputsRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - ExecutionInputsRequest* 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 ExecutionInputsRequest& from); - void MergeFrom(const ExecutionInputsRequest& 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(ExecutionInputsRequest* 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 ------------------------------------------------------- - - // repeated .flyteidl.core.ArtifactID inputs = 2; - int inputs_size() const; - void clear_inputs(); - static const int kInputsFieldNumber = 2; - ::flyteidl::core::ArtifactID* mutable_inputs(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >* - mutable_inputs(); - const ::flyteidl::core::ArtifactID& inputs(int index) const; - ::flyteidl::core::ArtifactID* add_inputs(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >& - inputs() const; - - // .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - bool has_execution_id() const; - void clear_execution_id(); - static const int kExecutionIdFieldNumber = 1; - const ::flyteidl::core::WorkflowExecutionIdentifier& execution_id() const; - ::flyteidl::core::WorkflowExecutionIdentifier* release_execution_id(); - ::flyteidl::core::WorkflowExecutionIdentifier* mutable_execution_id(); - void set_allocated_execution_id(::flyteidl::core::WorkflowExecutionIdentifier* execution_id); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ExecutionInputsRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID > inputs_; - ::flyteidl::core::WorkflowExecutionIdentifier* execution_id_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ExecutionInputsResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ExecutionInputsResponse) */ { - public: - ExecutionInputsResponse(); - virtual ~ExecutionInputsResponse(); - - ExecutionInputsResponse(const ExecutionInputsResponse& from); - - inline ExecutionInputsResponse& operator=(const ExecutionInputsResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ExecutionInputsResponse(ExecutionInputsResponse&& from) noexcept - : ExecutionInputsResponse() { - *this = ::std::move(from); - } - - inline ExecutionInputsResponse& operator=(ExecutionInputsResponse&& 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 ExecutionInputsResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ExecutionInputsResponse* internal_default_instance() { - return reinterpret_cast( - &_ExecutionInputsResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 24; - - void Swap(ExecutionInputsResponse* other); - friend void swap(ExecutionInputsResponse& a, ExecutionInputsResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ExecutionInputsResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - ExecutionInputsResponse* 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 ExecutionInputsResponse& from); - void MergeFrom(const ExecutionInputsResponse& 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(ExecutionInputsResponse* 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 ------------------------------------------------------- - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ExecutionInputsResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ListUsageRequest final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ListUsageRequest) */ { - public: - ListUsageRequest(); - virtual ~ListUsageRequest(); - - ListUsageRequest(const ListUsageRequest& from); - - inline ListUsageRequest& operator=(const ListUsageRequest& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ListUsageRequest(ListUsageRequest&& from) noexcept - : ListUsageRequest() { - *this = ::std::move(from); - } - - inline ListUsageRequest& operator=(ListUsageRequest&& 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 ListUsageRequest& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ListUsageRequest* internal_default_instance() { - return reinterpret_cast( - &_ListUsageRequest_default_instance_); - } - static constexpr int kIndexInFileMessages = - 25; - - void Swap(ListUsageRequest* other); - friend void swap(ListUsageRequest& a, ListUsageRequest& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ListUsageRequest* New() const final { - return CreateMaybeMessage(nullptr); - } - - ListUsageRequest* 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 ListUsageRequest& from); - void MergeFrom(const ListUsageRequest& 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(ListUsageRequest* 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.ArtifactID artifact_id = 1; - bool has_artifact_id() const; - void clear_artifact_id(); - static const int kArtifactIdFieldNumber = 1; - const ::flyteidl::core::ArtifactID& artifact_id() const; - ::flyteidl::core::ArtifactID* release_artifact_id(); - ::flyteidl::core::ArtifactID* mutable_artifact_id(); - void set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id); - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ListUsageRequest) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::flyteidl::core::ArtifactID* artifact_id_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// ------------------------------------------------------------------- - -class ListUsageResponse final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.artifact.ListUsageResponse) */ { - public: - ListUsageResponse(); - virtual ~ListUsageResponse(); - - ListUsageResponse(const ListUsageResponse& from); - - inline ListUsageResponse& operator=(const ListUsageResponse& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - ListUsageResponse(ListUsageResponse&& from) noexcept - : ListUsageResponse() { - *this = ::std::move(from); - } - - inline ListUsageResponse& operator=(ListUsageResponse&& 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 ListUsageResponse& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const ListUsageResponse* internal_default_instance() { - return reinterpret_cast( - &_ListUsageResponse_default_instance_); - } - static constexpr int kIndexInFileMessages = - 26; - - void Swap(ListUsageResponse* other); - friend void swap(ListUsageResponse& a, ListUsageResponse& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline ListUsageResponse* New() const final { - return CreateMaybeMessage(nullptr); - } - - ListUsageResponse* 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 ListUsageResponse& from); - void MergeFrom(const ListUsageResponse& 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(ListUsageResponse* 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 ------------------------------------------------------- - - // repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - int executions_size() const; - void clear_executions(); - static const int kExecutionsFieldNumber = 1; - ::flyteidl::core::WorkflowExecutionIdentifier* mutable_executions(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::WorkflowExecutionIdentifier >* - mutable_executions(); - const ::flyteidl::core::WorkflowExecutionIdentifier& executions(int index) const; - ::flyteidl::core::WorkflowExecutionIdentifier* add_executions(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::WorkflowExecutionIdentifier >& - executions() const; - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ListUsageResponse) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::WorkflowExecutionIdentifier > executions_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fartifact_2fartifacts_2eproto; -}; -// =================================================================== - - -// =================================================================== - -#ifdef __GNUC__ - #pragma GCC diagnostic push - #pragma GCC diagnostic ignored "-Wstrict-aliasing" -#endif // __GNUC__ -// Artifact - -// .flyteidl.core.ArtifactID artifact_id = 1; -inline bool Artifact::has_artifact_id() const { - return this != internal_default_instance() && artifact_id_ != nullptr; -} -inline const ::flyteidl::core::ArtifactID& Artifact::artifact_id() const { - const ::flyteidl::core::ArtifactID* p = artifact_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.Artifact.artifact_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactID_default_instance_); -} -inline ::flyteidl::core::ArtifactID* Artifact::release_artifact_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.Artifact.artifact_id) - - ::flyteidl::core::ArtifactID* temp = artifact_id_; - artifact_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactID* Artifact::mutable_artifact_id() { - - if (artifact_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactID>(GetArenaNoVirtual()); - artifact_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.Artifact.artifact_id) - return artifact_id_; -} -inline void Artifact::set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(artifact_id_); - } - if (artifact_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact_id, submessage_arena); - } - - } else { - - } - artifact_id_ = artifact_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.Artifact.artifact_id) -} - -// .flyteidl.artifact.ArtifactSpec spec = 2; -inline bool Artifact::has_spec() const { - return this != internal_default_instance() && spec_ != nullptr; -} -inline void Artifact::clear_spec() { - if (GetArenaNoVirtual() == nullptr && spec_ != nullptr) { - delete spec_; - } - spec_ = nullptr; -} -inline const ::flyteidl::artifact::ArtifactSpec& Artifact::spec() const { - const ::flyteidl::artifact::ArtifactSpec* p = spec_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.Artifact.spec) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_ArtifactSpec_default_instance_); -} -inline ::flyteidl::artifact::ArtifactSpec* Artifact::release_spec() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.Artifact.spec) - - ::flyteidl::artifact::ArtifactSpec* temp = spec_; - spec_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::ArtifactSpec* Artifact::mutable_spec() { - - if (spec_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::ArtifactSpec>(GetArenaNoVirtual()); - spec_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.Artifact.spec) - return spec_; -} -inline void Artifact::set_allocated_spec(::flyteidl::artifact::ArtifactSpec* spec) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete spec_; - } - if (spec) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - spec = ::google::protobuf::internal::GetOwnedMessage( - message_arena, spec, submessage_arena); - } - - } else { - - } - spec_ = spec; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.Artifact.spec) -} - -// repeated string tags = 3; -inline int Artifact::tags_size() const { - return tags_.size(); -} -inline void Artifact::clear_tags() { - tags_.Clear(); -} -inline const ::std::string& Artifact::tags(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.Artifact.tags) - return tags_.Get(index); -} -inline ::std::string* Artifact::mutable_tags(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.Artifact.tags) - return tags_.Mutable(index); -} -inline void Artifact::set_tags(int index, const ::std::string& value) { - // @@protoc_insertion_point(field_set:flyteidl.artifact.Artifact.tags) - tags_.Mutable(index)->assign(value); -} -#if LANG_CXX11 -inline void Artifact::set_tags(int index, ::std::string&& value) { - // @@protoc_insertion_point(field_set:flyteidl.artifact.Artifact.tags) - tags_.Mutable(index)->assign(std::move(value)); -} -#endif -inline void Artifact::set_tags(int index, const char* value) { - GOOGLE_DCHECK(value != nullptr); - tags_.Mutable(index)->assign(value); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.Artifact.tags) -} -inline void Artifact::set_tags(int index, const char* value, size_t size) { - tags_.Mutable(index)->assign( - reinterpret_cast(value), size); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.Artifact.tags) -} -inline ::std::string* Artifact::add_tags() { - // @@protoc_insertion_point(field_add_mutable:flyteidl.artifact.Artifact.tags) - return tags_.Add(); -} -inline void Artifact::add_tags(const ::std::string& value) { - tags_.Add()->assign(value); - // @@protoc_insertion_point(field_add:flyteidl.artifact.Artifact.tags) -} -#if LANG_CXX11 -inline void Artifact::add_tags(::std::string&& value) { - tags_.Add(std::move(value)); - // @@protoc_insertion_point(field_add:flyteidl.artifact.Artifact.tags) -} -#endif -inline void Artifact::add_tags(const char* value) { - GOOGLE_DCHECK(value != nullptr); - tags_.Add()->assign(value); - // @@protoc_insertion_point(field_add_char:flyteidl.artifact.Artifact.tags) -} -inline void Artifact::add_tags(const char* value, size_t size) { - tags_.Add()->assign(reinterpret_cast(value), size); - // @@protoc_insertion_point(field_add_pointer:flyteidl.artifact.Artifact.tags) -} -inline const ::google::protobuf::RepeatedPtrField<::std::string>& -Artifact::tags() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.Artifact.tags) - return tags_; -} -inline ::google::protobuf::RepeatedPtrField<::std::string>* -Artifact::mutable_tags() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.Artifact.tags) - return &tags_; -} - -// .flyteidl.artifact.ArtifactSource source = 4; -inline bool Artifact::has_source() const { - return this != internal_default_instance() && source_ != nullptr; -} -inline void Artifact::clear_source() { - if (GetArenaNoVirtual() == nullptr && source_ != nullptr) { - delete source_; - } - source_ = nullptr; -} -inline const ::flyteidl::artifact::ArtifactSource& Artifact::source() const { - const ::flyteidl::artifact::ArtifactSource* p = source_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.Artifact.source) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_ArtifactSource_default_instance_); -} -inline ::flyteidl::artifact::ArtifactSource* Artifact::release_source() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.Artifact.source) - - ::flyteidl::artifact::ArtifactSource* temp = source_; - source_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::ArtifactSource* Artifact::mutable_source() { - - if (source_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::ArtifactSource>(GetArenaNoVirtual()); - source_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.Artifact.source) - return source_; -} -inline void Artifact::set_allocated_source(::flyteidl::artifact::ArtifactSource* source) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete source_; - } - if (source) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - source = ::google::protobuf::internal::GetOwnedMessage( - message_arena, source, submessage_arena); - } - - } else { - - } - source_ = source; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.Artifact.source) -} - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// CreateArtifactRequest - -// .flyteidl.core.ArtifactKey artifact_key = 1; -inline bool CreateArtifactRequest::has_artifact_key() const { - return this != internal_default_instance() && artifact_key_ != nullptr; -} -inline const ::flyteidl::core::ArtifactKey& CreateArtifactRequest::artifact_key() const { - const ::flyteidl::core::ArtifactKey* p = artifact_key_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactRequest.artifact_key) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactKey_default_instance_); -} -inline ::flyteidl::core::ArtifactKey* CreateArtifactRequest::release_artifact_key() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactRequest.artifact_key) - - ::flyteidl::core::ArtifactKey* temp = artifact_key_; - artifact_key_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactKey* CreateArtifactRequest::mutable_artifact_key() { - - if (artifact_key_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactKey>(GetArenaNoVirtual()); - artifact_key_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactRequest.artifact_key) - return artifact_key_; -} -inline void CreateArtifactRequest::set_allocated_artifact_key(::flyteidl::core::ArtifactKey* artifact_key) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(artifact_key_); - } - if (artifact_key) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact_key = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact_key, submessage_arena); - } - - } else { - - } - artifact_key_ = artifact_key; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactRequest.artifact_key) -} - -// string version = 3; -inline void CreateArtifactRequest::clear_version() { - version_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& CreateArtifactRequest::version() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactRequest.version) - return version_.GetNoArena(); -} -inline void CreateArtifactRequest::set_version(const ::std::string& value) { - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.CreateArtifactRequest.version) -} -#if LANG_CXX11 -inline void CreateArtifactRequest::set_version(::std::string&& value) { - - version_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.CreateArtifactRequest.version) -} -#endif -inline void CreateArtifactRequest::set_version(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.CreateArtifactRequest.version) -} -inline void CreateArtifactRequest::set_version(const char* value, size_t size) { - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.CreateArtifactRequest.version) -} -inline ::std::string* CreateArtifactRequest::mutable_version() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactRequest.version) - return version_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* CreateArtifactRequest::release_version() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactRequest.version) - - return version_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void CreateArtifactRequest::set_allocated_version(::std::string* version) { - if (version != nullptr) { - - } else { - - } - version_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), version); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactRequest.version) -} - -// .flyteidl.artifact.ArtifactSpec spec = 2; -inline bool CreateArtifactRequest::has_spec() const { - return this != internal_default_instance() && spec_ != nullptr; -} -inline void CreateArtifactRequest::clear_spec() { - if (GetArenaNoVirtual() == nullptr && spec_ != nullptr) { - delete spec_; - } - spec_ = nullptr; -} -inline const ::flyteidl::artifact::ArtifactSpec& CreateArtifactRequest::spec() const { - const ::flyteidl::artifact::ArtifactSpec* p = spec_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactRequest.spec) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_ArtifactSpec_default_instance_); -} -inline ::flyteidl::artifact::ArtifactSpec* CreateArtifactRequest::release_spec() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactRequest.spec) - - ::flyteidl::artifact::ArtifactSpec* temp = spec_; - spec_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::ArtifactSpec* CreateArtifactRequest::mutable_spec() { - - if (spec_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::ArtifactSpec>(GetArenaNoVirtual()); - spec_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactRequest.spec) - return spec_; -} -inline void CreateArtifactRequest::set_allocated_spec(::flyteidl::artifact::ArtifactSpec* spec) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete spec_; - } - if (spec) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - spec = ::google::protobuf::internal::GetOwnedMessage( - message_arena, spec, submessage_arena); - } - - } else { - - } - spec_ = spec; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactRequest.spec) -} - -// map partitions = 4; -inline int CreateArtifactRequest::partitions_size() const { - return partitions_.size(); -} -inline void CreateArtifactRequest::clear_partitions() { - partitions_.Clear(); -} -inline const ::google::protobuf::Map< ::std::string, ::std::string >& -CreateArtifactRequest::partitions() const { - // @@protoc_insertion_point(field_map:flyteidl.artifact.CreateArtifactRequest.partitions) - return partitions_.GetMap(); -} -inline ::google::protobuf::Map< ::std::string, ::std::string >* -CreateArtifactRequest::mutable_partitions() { - // @@protoc_insertion_point(field_mutable_map:flyteidl.artifact.CreateArtifactRequest.partitions) - return partitions_.MutableMap(); -} - -// .google.protobuf.Timestamp time_partition_value = 5; -inline bool CreateArtifactRequest::has_time_partition_value() const { - return this != internal_default_instance() && time_partition_value_ != nullptr; -} -inline const ::google::protobuf::Timestamp& CreateArtifactRequest::time_partition_value() const { - const ::google::protobuf::Timestamp* p = time_partition_value_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactRequest.time_partition_value) - return p != nullptr ? *p : *reinterpret_cast( - &::google::protobuf::_Timestamp_default_instance_); -} -inline ::google::protobuf::Timestamp* CreateArtifactRequest::release_time_partition_value() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactRequest.time_partition_value) - - ::google::protobuf::Timestamp* temp = time_partition_value_; - time_partition_value_ = nullptr; - return temp; -} -inline ::google::protobuf::Timestamp* CreateArtifactRequest::mutable_time_partition_value() { - - if (time_partition_value_ == nullptr) { - auto* p = CreateMaybeMessage<::google::protobuf::Timestamp>(GetArenaNoVirtual()); - time_partition_value_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactRequest.time_partition_value) - return time_partition_value_; -} -inline void CreateArtifactRequest::set_allocated_time_partition_value(::google::protobuf::Timestamp* time_partition_value) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(time_partition_value_); - } - if (time_partition_value) { - ::google::protobuf::Arena* submessage_arena = - reinterpret_cast<::google::protobuf::MessageLite*>(time_partition_value)->GetArena(); - if (message_arena != submessage_arena) { - time_partition_value = ::google::protobuf::internal::GetOwnedMessage( - message_arena, time_partition_value, submessage_arena); - } - - } else { - - } - time_partition_value_ = time_partition_value; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactRequest.time_partition_value) -} - -// .flyteidl.artifact.ArtifactSource source = 6; -inline bool CreateArtifactRequest::has_source() const { - return this != internal_default_instance() && source_ != nullptr; -} -inline void CreateArtifactRequest::clear_source() { - if (GetArenaNoVirtual() == nullptr && source_ != nullptr) { - delete source_; - } - source_ = nullptr; -} -inline const ::flyteidl::artifact::ArtifactSource& CreateArtifactRequest::source() const { - const ::flyteidl::artifact::ArtifactSource* p = source_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactRequest.source) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_ArtifactSource_default_instance_); -} -inline ::flyteidl::artifact::ArtifactSource* CreateArtifactRequest::release_source() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactRequest.source) - - ::flyteidl::artifact::ArtifactSource* temp = source_; - source_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::ArtifactSource* CreateArtifactRequest::mutable_source() { - - if (source_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::ArtifactSource>(GetArenaNoVirtual()); - source_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactRequest.source) - return source_; -} -inline void CreateArtifactRequest::set_allocated_source(::flyteidl::artifact::ArtifactSource* source) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete source_; - } - if (source) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - source = ::google::protobuf::internal::GetOwnedMessage( - message_arena, source, submessage_arena); - } - - } else { - - } - source_ = source; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactRequest.source) -} - -// ------------------------------------------------------------------- - -// ArtifactSource - -// .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; -inline bool ArtifactSource::has_workflow_execution() const { - return this != internal_default_instance() && workflow_execution_ != nullptr; -} -inline const ::flyteidl::core::WorkflowExecutionIdentifier& ArtifactSource::workflow_execution() const { - const ::flyteidl::core::WorkflowExecutionIdentifier* p = workflow_execution_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSource.workflow_execution) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_WorkflowExecutionIdentifier_default_instance_); -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ArtifactSource::release_workflow_execution() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSource.workflow_execution) - - ::flyteidl::core::WorkflowExecutionIdentifier* temp = workflow_execution_; - workflow_execution_ = nullptr; - return temp; -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ArtifactSource::mutable_workflow_execution() { - - if (workflow_execution_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::WorkflowExecutionIdentifier>(GetArenaNoVirtual()); - workflow_execution_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSource.workflow_execution) - return workflow_execution_; -} -inline void ArtifactSource::set_allocated_workflow_execution(::flyteidl::core::WorkflowExecutionIdentifier* workflow_execution) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(workflow_execution_); - } - if (workflow_execution) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - workflow_execution = ::google::protobuf::internal::GetOwnedMessage( - message_arena, workflow_execution, submessage_arena); - } - - } else { - - } - workflow_execution_ = workflow_execution; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSource.workflow_execution) -} - -// string node_id = 2; -inline void ArtifactSource::clear_node_id() { - node_id_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& ArtifactSource::node_id() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSource.node_id) - return node_id_.GetNoArena(); -} -inline void ArtifactSource::set_node_id(const ::std::string& value) { - - node_id_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSource.node_id) -} -#if LANG_CXX11 -inline void ArtifactSource::set_node_id(::std::string&& value) { - - node_id_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.ArtifactSource.node_id) -} -#endif -inline void ArtifactSource::set_node_id(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - node_id_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.ArtifactSource.node_id) -} -inline void ArtifactSource::set_node_id(const char* value, size_t size) { - - node_id_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.ArtifactSource.node_id) -} -inline ::std::string* ArtifactSource::mutable_node_id() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSource.node_id) - return node_id_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* ArtifactSource::release_node_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSource.node_id) - - return node_id_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void ArtifactSource::set_allocated_node_id(::std::string* node_id) { - if (node_id != nullptr) { - - } else { - - } - node_id_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), node_id); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSource.node_id) -} - -// .flyteidl.core.Identifier task_id = 3; -inline bool ArtifactSource::has_task_id() const { - return this != internal_default_instance() && task_id_ != nullptr; -} -inline const ::flyteidl::core::Identifier& ArtifactSource::task_id() const { - const ::flyteidl::core::Identifier* p = task_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSource.task_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Identifier_default_instance_); -} -inline ::flyteidl::core::Identifier* ArtifactSource::release_task_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSource.task_id) - - ::flyteidl::core::Identifier* temp = task_id_; - task_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::Identifier* ArtifactSource::mutable_task_id() { - - if (task_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Identifier>(GetArenaNoVirtual()); - task_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSource.task_id) - return task_id_; -} -inline void ArtifactSource::set_allocated_task_id(::flyteidl::core::Identifier* task_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(task_id_); - } - if (task_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - task_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, task_id, submessage_arena); - } - - } else { - - } - task_id_ = task_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSource.task_id) -} - -// uint32 retry_attempt = 4; -inline void ArtifactSource::clear_retry_attempt() { - retry_attempt_ = 0u; -} -inline ::google::protobuf::uint32 ArtifactSource::retry_attempt() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSource.retry_attempt) - return retry_attempt_; -} -inline void ArtifactSource::set_retry_attempt(::google::protobuf::uint32 value) { - - retry_attempt_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSource.retry_attempt) -} - -// string principal = 5; -inline void ArtifactSource::clear_principal() { - principal_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& ArtifactSource::principal() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSource.principal) - return principal_.GetNoArena(); -} -inline void ArtifactSource::set_principal(const ::std::string& value) { - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSource.principal) -} -#if LANG_CXX11 -inline void ArtifactSource::set_principal(::std::string&& value) { - - principal_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.ArtifactSource.principal) -} -#endif -inline void ArtifactSource::set_principal(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.ArtifactSource.principal) -} -inline void ArtifactSource::set_principal(const char* value, size_t size) { - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.ArtifactSource.principal) -} -inline ::std::string* ArtifactSource::mutable_principal() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSource.principal) - return principal_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* ArtifactSource::release_principal() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSource.principal) - - return principal_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void ArtifactSource::set_allocated_principal(::std::string* principal) { - if (principal != nullptr) { - - } else { - - } - principal_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), principal); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSource.principal) -} - -// ------------------------------------------------------------------- - -// ArtifactSpec - -// .flyteidl.core.Literal value = 1; -inline bool ArtifactSpec::has_value() const { - return this != internal_default_instance() && value_ != nullptr; -} -inline const ::flyteidl::core::Literal& ArtifactSpec::value() const { - const ::flyteidl::core::Literal* p = value_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.value) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Literal_default_instance_); -} -inline ::flyteidl::core::Literal* ArtifactSpec::release_value() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.value) - - ::flyteidl::core::Literal* temp = value_; - value_ = nullptr; - return temp; -} -inline ::flyteidl::core::Literal* ArtifactSpec::mutable_value() { - - if (value_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Literal>(GetArenaNoVirtual()); - value_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.value) - return value_; -} -inline void ArtifactSpec::set_allocated_value(::flyteidl::core::Literal* value) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(value_); - } - if (value) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - value = ::google::protobuf::internal::GetOwnedMessage( - message_arena, value, submessage_arena); - } - - } else { - - } - value_ = value; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.value) -} - -// .flyteidl.core.LiteralType type = 2; -inline bool ArtifactSpec::has_type() const { - return this != internal_default_instance() && type_ != nullptr; -} -inline const ::flyteidl::core::LiteralType& ArtifactSpec::type() const { - const ::flyteidl::core::LiteralType* p = type_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.type) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_LiteralType_default_instance_); -} -inline ::flyteidl::core::LiteralType* ArtifactSpec::release_type() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.type) - - ::flyteidl::core::LiteralType* temp = type_; - type_ = nullptr; - return temp; -} -inline ::flyteidl::core::LiteralType* ArtifactSpec::mutable_type() { - - if (type_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::LiteralType>(GetArenaNoVirtual()); - type_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.type) - return type_; -} -inline void ArtifactSpec::set_allocated_type(::flyteidl::core::LiteralType* type) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(type_); - } - if (type) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - type = ::google::protobuf::internal::GetOwnedMessage( - message_arena, type, submessage_arena); - } - - } else { - - } - type_ = type; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.type) -} - -// string short_description = 3; -inline void ArtifactSpec::clear_short_description() { - short_description_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& ArtifactSpec::short_description() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.short_description) - return short_description_.GetNoArena(); -} -inline void ArtifactSpec::set_short_description(const ::std::string& value) { - - short_description_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSpec.short_description) -} -#if LANG_CXX11 -inline void ArtifactSpec::set_short_description(::std::string&& value) { - - short_description_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.ArtifactSpec.short_description) -} -#endif -inline void ArtifactSpec::set_short_description(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - short_description_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.ArtifactSpec.short_description) -} -inline void ArtifactSpec::set_short_description(const char* value, size_t size) { - - short_description_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.ArtifactSpec.short_description) -} -inline ::std::string* ArtifactSpec::mutable_short_description() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.short_description) - return short_description_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* ArtifactSpec::release_short_description() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.short_description) - - return short_description_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void ArtifactSpec::set_allocated_short_description(::std::string* short_description) { - if (short_description != nullptr) { - - } else { - - } - short_description_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), short_description); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.short_description) -} - -// .google.protobuf.Any user_metadata = 4; -inline bool ArtifactSpec::has_user_metadata() const { - return this != internal_default_instance() && user_metadata_ != nullptr; -} -inline const ::google::protobuf::Any& ArtifactSpec::user_metadata() const { - const ::google::protobuf::Any* p = user_metadata_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.user_metadata) - return p != nullptr ? *p : *reinterpret_cast( - &::google::protobuf::_Any_default_instance_); -} -inline ::google::protobuf::Any* ArtifactSpec::release_user_metadata() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.user_metadata) - - ::google::protobuf::Any* temp = user_metadata_; - user_metadata_ = nullptr; - return temp; -} -inline ::google::protobuf::Any* ArtifactSpec::mutable_user_metadata() { - - if (user_metadata_ == nullptr) { - auto* p = CreateMaybeMessage<::google::protobuf::Any>(GetArenaNoVirtual()); - user_metadata_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.user_metadata) - return user_metadata_; -} -inline void ArtifactSpec::set_allocated_user_metadata(::google::protobuf::Any* user_metadata) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(user_metadata_); - } - if (user_metadata) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - user_metadata = ::google::protobuf::internal::GetOwnedMessage( - message_arena, user_metadata, submessage_arena); - } - - } else { - - } - user_metadata_ = user_metadata; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.user_metadata) -} - -// string metadata_type = 5; -inline void ArtifactSpec::clear_metadata_type() { - metadata_type_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& ArtifactSpec::metadata_type() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.metadata_type) - return metadata_type_.GetNoArena(); -} -inline void ArtifactSpec::set_metadata_type(const ::std::string& value) { - - metadata_type_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSpec.metadata_type) -} -#if LANG_CXX11 -inline void ArtifactSpec::set_metadata_type(::std::string&& value) { - - metadata_type_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.ArtifactSpec.metadata_type) -} -#endif -inline void ArtifactSpec::set_metadata_type(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - metadata_type_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.ArtifactSpec.metadata_type) -} -inline void ArtifactSpec::set_metadata_type(const char* value, size_t size) { - - metadata_type_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.ArtifactSpec.metadata_type) -} -inline ::std::string* ArtifactSpec::mutable_metadata_type() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.metadata_type) - return metadata_type_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* ArtifactSpec::release_metadata_type() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.metadata_type) - - return metadata_type_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void ArtifactSpec::set_allocated_metadata_type(::std::string* metadata_type) { - if (metadata_type != nullptr) { - - } else { - - } - metadata_type_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), metadata_type); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.metadata_type) -} - -// .google.protobuf.Timestamp created_at = 6; -inline bool ArtifactSpec::has_created_at() const { - return this != internal_default_instance() && created_at_ != nullptr; -} -inline const ::google::protobuf::Timestamp& ArtifactSpec::created_at() const { - const ::google::protobuf::Timestamp* p = created_at_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.created_at) - return p != nullptr ? *p : *reinterpret_cast( - &::google::protobuf::_Timestamp_default_instance_); -} -inline ::google::protobuf::Timestamp* ArtifactSpec::release_created_at() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.created_at) - - ::google::protobuf::Timestamp* temp = created_at_; - created_at_ = nullptr; - return temp; -} -inline ::google::protobuf::Timestamp* ArtifactSpec::mutable_created_at() { - - if (created_at_ == nullptr) { - auto* p = CreateMaybeMessage<::google::protobuf::Timestamp>(GetArenaNoVirtual()); - created_at_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.created_at) - return created_at_; -} -inline void ArtifactSpec::set_allocated_created_at(::google::protobuf::Timestamp* created_at) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(created_at_); - } - if (created_at) { - ::google::protobuf::Arena* submessage_arena = - reinterpret_cast<::google::protobuf::MessageLite*>(created_at)->GetArena(); - if (message_arena != submessage_arena) { - created_at = ::google::protobuf::internal::GetOwnedMessage( - message_arena, created_at, submessage_arena); - } - - } else { - - } - created_at_ = created_at; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.created_at) -} - -// string file_format = 7; -inline void ArtifactSpec::clear_file_format() { - file_format_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& ArtifactSpec::file_format() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactSpec.file_format) - return file_format_.GetNoArena(); -} -inline void ArtifactSpec::set_file_format(const ::std::string& value) { - - file_format_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.ArtifactSpec.file_format) -} -#if LANG_CXX11 -inline void ArtifactSpec::set_file_format(::std::string&& value) { - - file_format_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.ArtifactSpec.file_format) -} -#endif -inline void ArtifactSpec::set_file_format(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - file_format_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.ArtifactSpec.file_format) -} -inline void ArtifactSpec::set_file_format(const char* value, size_t size) { - - file_format_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.ArtifactSpec.file_format) -} -inline ::std::string* ArtifactSpec::mutable_file_format() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactSpec.file_format) - return file_format_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* ArtifactSpec::release_file_format() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactSpec.file_format) - - return file_format_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void ArtifactSpec::set_allocated_file_format(::std::string* file_format) { - if (file_format != nullptr) { - - } else { - - } - file_format_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), file_format); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactSpec.file_format) -} - -// ------------------------------------------------------------------- - -// CreateArtifactResponse - -// .flyteidl.artifact.Artifact artifact = 1; -inline bool CreateArtifactResponse::has_artifact() const { - return this != internal_default_instance() && artifact_ != nullptr; -} -inline void CreateArtifactResponse::clear_artifact() { - if (GetArenaNoVirtual() == nullptr && artifact_ != nullptr) { - delete artifact_; - } - artifact_ = nullptr; -} -inline const ::flyteidl::artifact::Artifact& CreateArtifactResponse::artifact() const { - const ::flyteidl::artifact::Artifact* p = artifact_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateArtifactResponse.artifact) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_Artifact_default_instance_); -} -inline ::flyteidl::artifact::Artifact* CreateArtifactResponse::release_artifact() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateArtifactResponse.artifact) - - ::flyteidl::artifact::Artifact* temp = artifact_; - artifact_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::Artifact* CreateArtifactResponse::mutable_artifact() { - - if (artifact_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::Artifact>(GetArenaNoVirtual()); - artifact_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateArtifactResponse.artifact) - return artifact_; -} -inline void CreateArtifactResponse::set_allocated_artifact(::flyteidl::artifact::Artifact* artifact) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete artifact_; - } - if (artifact) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact, submessage_arena); - } - - } else { - - } - artifact_ = artifact; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateArtifactResponse.artifact) -} - -// ------------------------------------------------------------------- - -// GetArtifactRequest - -// .flyteidl.core.ArtifactQuery query = 1; -inline bool GetArtifactRequest::has_query() const { - return this != internal_default_instance() && query_ != nullptr; -} -inline const ::flyteidl::core::ArtifactQuery& GetArtifactRequest::query() const { - const ::flyteidl::core::ArtifactQuery* p = query_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.GetArtifactRequest.query) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactQuery_default_instance_); -} -inline ::flyteidl::core::ArtifactQuery* GetArtifactRequest::release_query() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.GetArtifactRequest.query) - - ::flyteidl::core::ArtifactQuery* temp = query_; - query_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactQuery* GetArtifactRequest::mutable_query() { - - if (query_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactQuery>(GetArenaNoVirtual()); - query_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.GetArtifactRequest.query) - return query_; -} -inline void GetArtifactRequest::set_allocated_query(::flyteidl::core::ArtifactQuery* query) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(query_); - } - if (query) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - query = ::google::protobuf::internal::GetOwnedMessage( - message_arena, query, submessage_arena); - } - - } else { - - } - query_ = query; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.GetArtifactRequest.query) -} - -// bool details = 2; -inline void GetArtifactRequest::clear_details() { - details_ = false; -} -inline bool GetArtifactRequest::details() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.GetArtifactRequest.details) - return details_; -} -inline void GetArtifactRequest::set_details(bool value) { - - details_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.GetArtifactRequest.details) -} - -// ------------------------------------------------------------------- - -// GetArtifactResponse - -// .flyteidl.artifact.Artifact artifact = 1; -inline bool GetArtifactResponse::has_artifact() const { - return this != internal_default_instance() && artifact_ != nullptr; -} -inline void GetArtifactResponse::clear_artifact() { - if (GetArenaNoVirtual() == nullptr && artifact_ != nullptr) { - delete artifact_; - } - artifact_ = nullptr; -} -inline const ::flyteidl::artifact::Artifact& GetArtifactResponse::artifact() const { - const ::flyteidl::artifact::Artifact* p = artifact_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.GetArtifactResponse.artifact) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_Artifact_default_instance_); -} -inline ::flyteidl::artifact::Artifact* GetArtifactResponse::release_artifact() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.GetArtifactResponse.artifact) - - ::flyteidl::artifact::Artifact* temp = artifact_; - artifact_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::Artifact* GetArtifactResponse::mutable_artifact() { - - if (artifact_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::Artifact>(GetArenaNoVirtual()); - artifact_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.GetArtifactResponse.artifact) - return artifact_; -} -inline void GetArtifactResponse::set_allocated_artifact(::flyteidl::artifact::Artifact* artifact) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete artifact_; - } - if (artifact) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact, submessage_arena); - } - - } else { - - } - artifact_ = artifact; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.GetArtifactResponse.artifact) -} - -// ------------------------------------------------------------------- - -// SearchOptions - -// bool strict_partitions = 1; -inline void SearchOptions::clear_strict_partitions() { - strict_partitions_ = false; -} -inline bool SearchOptions::strict_partitions() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchOptions.strict_partitions) - return strict_partitions_; -} -inline void SearchOptions::set_strict_partitions(bool value) { - - strict_partitions_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchOptions.strict_partitions) -} - -// bool latest_by_key = 2; -inline void SearchOptions::clear_latest_by_key() { - latest_by_key_ = false; -} -inline bool SearchOptions::latest_by_key() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchOptions.latest_by_key) - return latest_by_key_; -} -inline void SearchOptions::set_latest_by_key(bool value) { - - latest_by_key_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchOptions.latest_by_key) -} - -// ------------------------------------------------------------------- - -// SearchArtifactsRequest - -// .flyteidl.core.ArtifactKey artifact_key = 1; -inline bool SearchArtifactsRequest::has_artifact_key() const { - return this != internal_default_instance() && artifact_key_ != nullptr; -} -inline const ::flyteidl::core::ArtifactKey& SearchArtifactsRequest::artifact_key() const { - const ::flyteidl::core::ArtifactKey* p = artifact_key_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.artifact_key) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactKey_default_instance_); -} -inline ::flyteidl::core::ArtifactKey* SearchArtifactsRequest::release_artifact_key() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.artifact_key) - - ::flyteidl::core::ArtifactKey* temp = artifact_key_; - artifact_key_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactKey* SearchArtifactsRequest::mutable_artifact_key() { - - if (artifact_key_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactKey>(GetArenaNoVirtual()); - artifact_key_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.artifact_key) - return artifact_key_; -} -inline void SearchArtifactsRequest::set_allocated_artifact_key(::flyteidl::core::ArtifactKey* artifact_key) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(artifact_key_); - } - if (artifact_key) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact_key = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact_key, submessage_arena); - } - - } else { - - } - artifact_key_ = artifact_key; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.artifact_key) -} - -// .flyteidl.core.Partitions partitions = 2; -inline bool SearchArtifactsRequest::has_partitions() const { - return this != internal_default_instance() && partitions_ != nullptr; -} -inline const ::flyteidl::core::Partitions& SearchArtifactsRequest::partitions() const { - const ::flyteidl::core::Partitions* p = partitions_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.partitions) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Partitions_default_instance_); -} -inline ::flyteidl::core::Partitions* SearchArtifactsRequest::release_partitions() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.partitions) - - ::flyteidl::core::Partitions* temp = partitions_; - partitions_ = nullptr; - return temp; -} -inline ::flyteidl::core::Partitions* SearchArtifactsRequest::mutable_partitions() { - - if (partitions_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Partitions>(GetArenaNoVirtual()); - partitions_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.partitions) - return partitions_; -} -inline void SearchArtifactsRequest::set_allocated_partitions(::flyteidl::core::Partitions* partitions) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(partitions_); - } - if (partitions) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - partitions = ::google::protobuf::internal::GetOwnedMessage( - message_arena, partitions, submessage_arena); - } - - } else { - - } - partitions_ = partitions; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.partitions) -} - -// .google.protobuf.Timestamp time_partition_value = 3; -inline bool SearchArtifactsRequest::has_time_partition_value() const { - return this != internal_default_instance() && time_partition_value_ != nullptr; -} -inline const ::google::protobuf::Timestamp& SearchArtifactsRequest::time_partition_value() const { - const ::google::protobuf::Timestamp* p = time_partition_value_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.time_partition_value) - return p != nullptr ? *p : *reinterpret_cast( - &::google::protobuf::_Timestamp_default_instance_); -} -inline ::google::protobuf::Timestamp* SearchArtifactsRequest::release_time_partition_value() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.time_partition_value) - - ::google::protobuf::Timestamp* temp = time_partition_value_; - time_partition_value_ = nullptr; - return temp; -} -inline ::google::protobuf::Timestamp* SearchArtifactsRequest::mutable_time_partition_value() { - - if (time_partition_value_ == nullptr) { - auto* p = CreateMaybeMessage<::google::protobuf::Timestamp>(GetArenaNoVirtual()); - time_partition_value_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.time_partition_value) - return time_partition_value_; -} -inline void SearchArtifactsRequest::set_allocated_time_partition_value(::google::protobuf::Timestamp* time_partition_value) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(time_partition_value_); - } - if (time_partition_value) { - ::google::protobuf::Arena* submessage_arena = - reinterpret_cast<::google::protobuf::MessageLite*>(time_partition_value)->GetArena(); - if (message_arena != submessage_arena) { - time_partition_value = ::google::protobuf::internal::GetOwnedMessage( - message_arena, time_partition_value, submessage_arena); - } - - } else { - - } - time_partition_value_ = time_partition_value; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.time_partition_value) -} - -// string principal = 4; -inline void SearchArtifactsRequest::clear_principal() { - principal_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& SearchArtifactsRequest::principal() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.principal) - return principal_.GetNoArena(); -} -inline void SearchArtifactsRequest::set_principal(const ::std::string& value) { - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchArtifactsRequest.principal) -} -#if LANG_CXX11 -inline void SearchArtifactsRequest::set_principal(::std::string&& value) { - - principal_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.SearchArtifactsRequest.principal) -} -#endif -inline void SearchArtifactsRequest::set_principal(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.SearchArtifactsRequest.principal) -} -inline void SearchArtifactsRequest::set_principal(const char* value, size_t size) { - - principal_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.SearchArtifactsRequest.principal) -} -inline ::std::string* SearchArtifactsRequest::mutable_principal() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.principal) - return principal_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* SearchArtifactsRequest::release_principal() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.principal) - - return principal_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void SearchArtifactsRequest::set_allocated_principal(::std::string* principal) { - if (principal != nullptr) { - - } else { - - } - principal_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), principal); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.principal) -} - -// string version = 5; -inline void SearchArtifactsRequest::clear_version() { - version_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& SearchArtifactsRequest::version() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.version) - return version_.GetNoArena(); -} -inline void SearchArtifactsRequest::set_version(const ::std::string& value) { - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchArtifactsRequest.version) -} -#if LANG_CXX11 -inline void SearchArtifactsRequest::set_version(::std::string&& value) { - - version_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.SearchArtifactsRequest.version) -} -#endif -inline void SearchArtifactsRequest::set_version(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.SearchArtifactsRequest.version) -} -inline void SearchArtifactsRequest::set_version(const char* value, size_t size) { - - version_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.SearchArtifactsRequest.version) -} -inline ::std::string* SearchArtifactsRequest::mutable_version() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.version) - return version_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* SearchArtifactsRequest::release_version() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.version) - - return version_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void SearchArtifactsRequest::set_allocated_version(::std::string* version) { - if (version != nullptr) { - - } else { - - } - version_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), version); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.version) -} - -// .flyteidl.artifact.SearchOptions options = 6; -inline bool SearchArtifactsRequest::has_options() const { - return this != internal_default_instance() && options_ != nullptr; -} -inline void SearchArtifactsRequest::clear_options() { - if (GetArenaNoVirtual() == nullptr && options_ != nullptr) { - delete options_; - } - options_ = nullptr; -} -inline const ::flyteidl::artifact::SearchOptions& SearchArtifactsRequest::options() const { - const ::flyteidl::artifact::SearchOptions* p = options_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.options) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::artifact::_SearchOptions_default_instance_); -} -inline ::flyteidl::artifact::SearchOptions* SearchArtifactsRequest::release_options() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.options) - - ::flyteidl::artifact::SearchOptions* temp = options_; - options_ = nullptr; - return temp; -} -inline ::flyteidl::artifact::SearchOptions* SearchArtifactsRequest::mutable_options() { - - if (options_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::artifact::SearchOptions>(GetArenaNoVirtual()); - options_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.options) - return options_; -} -inline void SearchArtifactsRequest::set_allocated_options(::flyteidl::artifact::SearchOptions* options) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete options_; - } - if (options) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - options = ::google::protobuf::internal::GetOwnedMessage( - message_arena, options, submessage_arena); - } - - } else { - - } - options_ = options; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.options) -} - -// string token = 7; -inline void SearchArtifactsRequest::clear_token() { - token_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& SearchArtifactsRequest::token() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.token) - return token_.GetNoArena(); -} -inline void SearchArtifactsRequest::set_token(const ::std::string& value) { - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchArtifactsRequest.token) -} -#if LANG_CXX11 -inline void SearchArtifactsRequest::set_token(::std::string&& value) { - - token_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.SearchArtifactsRequest.token) -} -#endif -inline void SearchArtifactsRequest::set_token(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.SearchArtifactsRequest.token) -} -inline void SearchArtifactsRequest::set_token(const char* value, size_t size) { - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.SearchArtifactsRequest.token) -} -inline ::std::string* SearchArtifactsRequest::mutable_token() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsRequest.token) - return token_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* SearchArtifactsRequest::release_token() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsRequest.token) - - return token_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void SearchArtifactsRequest::set_allocated_token(::std::string* token) { - if (token != nullptr) { - - } else { - - } - token_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), token); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsRequest.token) -} - -// int32 limit = 8; -inline void SearchArtifactsRequest::clear_limit() { - limit_ = 0; -} -inline ::google::protobuf::int32 SearchArtifactsRequest::limit() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsRequest.limit) - return limit_; -} -inline void SearchArtifactsRequest::set_limit(::google::protobuf::int32 value) { - - limit_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchArtifactsRequest.limit) -} - -// ------------------------------------------------------------------- - -// SearchArtifactsResponse - -// repeated .flyteidl.artifact.Artifact artifacts = 1; -inline int SearchArtifactsResponse::artifacts_size() const { - return artifacts_.size(); -} -inline void SearchArtifactsResponse::clear_artifacts() { - artifacts_.Clear(); -} -inline ::flyteidl::artifact::Artifact* SearchArtifactsResponse::mutable_artifacts(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsResponse.artifacts) - return artifacts_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::Artifact >* -SearchArtifactsResponse::mutable_artifacts() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.SearchArtifactsResponse.artifacts) - return &artifacts_; -} -inline const ::flyteidl::artifact::Artifact& SearchArtifactsResponse::artifacts(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsResponse.artifacts) - return artifacts_.Get(index); -} -inline ::flyteidl::artifact::Artifact* SearchArtifactsResponse::add_artifacts() { - // @@protoc_insertion_point(field_add:flyteidl.artifact.SearchArtifactsResponse.artifacts) - return artifacts_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::Artifact >& -SearchArtifactsResponse::artifacts() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.SearchArtifactsResponse.artifacts) - return artifacts_; -} - -// string token = 2; -inline void SearchArtifactsResponse::clear_token() { - token_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& SearchArtifactsResponse::token() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.SearchArtifactsResponse.token) - return token_.GetNoArena(); -} -inline void SearchArtifactsResponse::set_token(const ::std::string& value) { - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.SearchArtifactsResponse.token) -} -#if LANG_CXX11 -inline void SearchArtifactsResponse::set_token(::std::string&& value) { - - token_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.SearchArtifactsResponse.token) -} -#endif -inline void SearchArtifactsResponse::set_token(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.SearchArtifactsResponse.token) -} -inline void SearchArtifactsResponse::set_token(const char* value, size_t size) { - - token_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.SearchArtifactsResponse.token) -} -inline ::std::string* SearchArtifactsResponse::mutable_token() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.SearchArtifactsResponse.token) - return token_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* SearchArtifactsResponse::release_token() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.SearchArtifactsResponse.token) - - return token_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void SearchArtifactsResponse::set_allocated_token(::std::string* token) { - if (token != nullptr) { - - } else { - - } - token_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), token); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.SearchArtifactsResponse.token) -} - -// ------------------------------------------------------------------- - -// FindByWorkflowExecRequest - -// .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; -inline bool FindByWorkflowExecRequest::has_exec_id() const { - return this != internal_default_instance() && exec_id_ != nullptr; -} -inline const ::flyteidl::core::WorkflowExecutionIdentifier& FindByWorkflowExecRequest::exec_id() const { - const ::flyteidl::core::WorkflowExecutionIdentifier* p = exec_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.FindByWorkflowExecRequest.exec_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_WorkflowExecutionIdentifier_default_instance_); -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* FindByWorkflowExecRequest::release_exec_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.FindByWorkflowExecRequest.exec_id) - - ::flyteidl::core::WorkflowExecutionIdentifier* temp = exec_id_; - exec_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* FindByWorkflowExecRequest::mutable_exec_id() { - - if (exec_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::WorkflowExecutionIdentifier>(GetArenaNoVirtual()); - exec_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.FindByWorkflowExecRequest.exec_id) - return exec_id_; -} -inline void FindByWorkflowExecRequest::set_allocated_exec_id(::flyteidl::core::WorkflowExecutionIdentifier* exec_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(exec_id_); - } - if (exec_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - exec_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, exec_id, submessage_arena); - } - - } else { - - } - exec_id_ = exec_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.FindByWorkflowExecRequest.exec_id) -} - -// .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; -inline void FindByWorkflowExecRequest::clear_direction() { - direction_ = 0; -} -inline ::flyteidl::artifact::FindByWorkflowExecRequest_Direction FindByWorkflowExecRequest::direction() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.FindByWorkflowExecRequest.direction) - return static_cast< ::flyteidl::artifact::FindByWorkflowExecRequest_Direction >(direction_); -} -inline void FindByWorkflowExecRequest::set_direction(::flyteidl::artifact::FindByWorkflowExecRequest_Direction value) { - - direction_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.FindByWorkflowExecRequest.direction) -} - -// ------------------------------------------------------------------- - -// AddTagRequest - -// .flyteidl.core.ArtifactID artifact_id = 1; -inline bool AddTagRequest::has_artifact_id() const { - return this != internal_default_instance() && artifact_id_ != nullptr; -} -inline const ::flyteidl::core::ArtifactID& AddTagRequest::artifact_id() const { - const ::flyteidl::core::ArtifactID* p = artifact_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.AddTagRequest.artifact_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactID_default_instance_); -} -inline ::flyteidl::core::ArtifactID* AddTagRequest::release_artifact_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.AddTagRequest.artifact_id) - - ::flyteidl::core::ArtifactID* temp = artifact_id_; - artifact_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactID* AddTagRequest::mutable_artifact_id() { - - if (artifact_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactID>(GetArenaNoVirtual()); - artifact_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.AddTagRequest.artifact_id) - return artifact_id_; -} -inline void AddTagRequest::set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(artifact_id_); - } - if (artifact_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact_id, submessage_arena); - } - - } else { - - } - artifact_id_ = artifact_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.AddTagRequest.artifact_id) -} - -// string value = 2; -inline void AddTagRequest::clear_value() { - value_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& AddTagRequest::value() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.AddTagRequest.value) - return value_.GetNoArena(); -} -inline void AddTagRequest::set_value(const ::std::string& value) { - - value_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:flyteidl.artifact.AddTagRequest.value) -} -#if LANG_CXX11 -inline void AddTagRequest::set_value(::std::string&& value) { - - value_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:flyteidl.artifact.AddTagRequest.value) -} -#endif -inline void AddTagRequest::set_value(const char* value) { - GOOGLE_DCHECK(value != nullptr); - - value_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:flyteidl.artifact.AddTagRequest.value) -} -inline void AddTagRequest::set_value(const char* value, size_t size) { - - value_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:flyteidl.artifact.AddTagRequest.value) -} -inline ::std::string* AddTagRequest::mutable_value() { - - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.AddTagRequest.value) - return value_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline ::std::string* AddTagRequest::release_value() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.AddTagRequest.value) - - return value_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline void AddTagRequest::set_allocated_value(::std::string* value) { - if (value != nullptr) { - - } else { - - } - value_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.AddTagRequest.value) -} - -// bool overwrite = 3; -inline void AddTagRequest::clear_overwrite() { - overwrite_ = false; -} -inline bool AddTagRequest::overwrite() const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.AddTagRequest.overwrite) - return overwrite_; -} -inline void AddTagRequest::set_overwrite(bool value) { - - overwrite_ = value; - // @@protoc_insertion_point(field_set:flyteidl.artifact.AddTagRequest.overwrite) -} - -// ------------------------------------------------------------------- - -// AddTagResponse - -// ------------------------------------------------------------------- - -// CreateTriggerRequest - -// .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; -inline bool CreateTriggerRequest::has_trigger_launch_plan() const { - return this != internal_default_instance() && trigger_launch_plan_ != nullptr; -} -inline const ::flyteidl::admin::LaunchPlan& CreateTriggerRequest::trigger_launch_plan() const { - const ::flyteidl::admin::LaunchPlan* p = trigger_launch_plan_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.CreateTriggerRequest.trigger_launch_plan) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::admin::_LaunchPlan_default_instance_); -} -inline ::flyteidl::admin::LaunchPlan* CreateTriggerRequest::release_trigger_launch_plan() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.CreateTriggerRequest.trigger_launch_plan) - - ::flyteidl::admin::LaunchPlan* temp = trigger_launch_plan_; - trigger_launch_plan_ = nullptr; - return temp; -} -inline ::flyteidl::admin::LaunchPlan* CreateTriggerRequest::mutable_trigger_launch_plan() { - - if (trigger_launch_plan_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::admin::LaunchPlan>(GetArenaNoVirtual()); - trigger_launch_plan_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.CreateTriggerRequest.trigger_launch_plan) - return trigger_launch_plan_; -} -inline void CreateTriggerRequest::set_allocated_trigger_launch_plan(::flyteidl::admin::LaunchPlan* trigger_launch_plan) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(trigger_launch_plan_); - } - if (trigger_launch_plan) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - trigger_launch_plan = ::google::protobuf::internal::GetOwnedMessage( - message_arena, trigger_launch_plan, submessage_arena); - } - - } else { - - } - trigger_launch_plan_ = trigger_launch_plan; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.CreateTriggerRequest.trigger_launch_plan) -} - -// ------------------------------------------------------------------- - -// CreateTriggerResponse - -// ------------------------------------------------------------------- - -// DeactivateTriggerRequest - -// .flyteidl.core.Identifier trigger_id = 1; -inline bool DeactivateTriggerRequest::has_trigger_id() const { - return this != internal_default_instance() && trigger_id_ != nullptr; -} -inline const ::flyteidl::core::Identifier& DeactivateTriggerRequest::trigger_id() const { - const ::flyteidl::core::Identifier* p = trigger_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.DeactivateTriggerRequest.trigger_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Identifier_default_instance_); -} -inline ::flyteidl::core::Identifier* DeactivateTriggerRequest::release_trigger_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.DeactivateTriggerRequest.trigger_id) - - ::flyteidl::core::Identifier* temp = trigger_id_; - trigger_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::Identifier* DeactivateTriggerRequest::mutable_trigger_id() { - - if (trigger_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Identifier>(GetArenaNoVirtual()); - trigger_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.DeactivateTriggerRequest.trigger_id) - return trigger_id_; -} -inline void DeactivateTriggerRequest::set_allocated_trigger_id(::flyteidl::core::Identifier* trigger_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(trigger_id_); - } - if (trigger_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - trigger_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, trigger_id, submessage_arena); - } - - } else { - - } - trigger_id_ = trigger_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.DeactivateTriggerRequest.trigger_id) -} - -// ------------------------------------------------------------------- - -// DeactivateTriggerResponse - -// ------------------------------------------------------------------- - -// ArtifactProducer - -// .flyteidl.core.Identifier entity_id = 1; -inline bool ArtifactProducer::has_entity_id() const { - return this != internal_default_instance() && entity_id_ != nullptr; -} -inline const ::flyteidl::core::Identifier& ArtifactProducer::entity_id() const { - const ::flyteidl::core::Identifier* p = entity_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactProducer.entity_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Identifier_default_instance_); -} -inline ::flyteidl::core::Identifier* ArtifactProducer::release_entity_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactProducer.entity_id) - - ::flyteidl::core::Identifier* temp = entity_id_; - entity_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::Identifier* ArtifactProducer::mutable_entity_id() { - - if (entity_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Identifier>(GetArenaNoVirtual()); - entity_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactProducer.entity_id) - return entity_id_; -} -inline void ArtifactProducer::set_allocated_entity_id(::flyteidl::core::Identifier* entity_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(entity_id_); - } - if (entity_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - entity_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, entity_id, submessage_arena); - } - - } else { - - } - entity_id_ = entity_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactProducer.entity_id) -} - -// .flyteidl.core.VariableMap outputs = 2; -inline bool ArtifactProducer::has_outputs() const { - return this != internal_default_instance() && outputs_ != nullptr; -} -inline const ::flyteidl::core::VariableMap& ArtifactProducer::outputs() const { - const ::flyteidl::core::VariableMap* p = outputs_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactProducer.outputs) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_VariableMap_default_instance_); -} -inline ::flyteidl::core::VariableMap* ArtifactProducer::release_outputs() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactProducer.outputs) - - ::flyteidl::core::VariableMap* temp = outputs_; - outputs_ = nullptr; - return temp; -} -inline ::flyteidl::core::VariableMap* ArtifactProducer::mutable_outputs() { - - if (outputs_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::VariableMap>(GetArenaNoVirtual()); - outputs_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactProducer.outputs) - return outputs_; -} -inline void ArtifactProducer::set_allocated_outputs(::flyteidl::core::VariableMap* outputs) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(outputs_); - } - if (outputs) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - outputs = ::google::protobuf::internal::GetOwnedMessage( - message_arena, outputs, submessage_arena); - } - - } else { - - } - outputs_ = outputs; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactProducer.outputs) -} - -// ------------------------------------------------------------------- - -// RegisterProducerRequest - -// repeated .flyteidl.artifact.ArtifactProducer producers = 1; -inline int RegisterProducerRequest::producers_size() const { - return producers_.size(); -} -inline void RegisterProducerRequest::clear_producers() { - producers_.Clear(); -} -inline ::flyteidl::artifact::ArtifactProducer* RegisterProducerRequest::mutable_producers(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.RegisterProducerRequest.producers) - return producers_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactProducer >* -RegisterProducerRequest::mutable_producers() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.RegisterProducerRequest.producers) - return &producers_; -} -inline const ::flyteidl::artifact::ArtifactProducer& RegisterProducerRequest::producers(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.RegisterProducerRequest.producers) - return producers_.Get(index); -} -inline ::flyteidl::artifact::ArtifactProducer* RegisterProducerRequest::add_producers() { - // @@protoc_insertion_point(field_add:flyteidl.artifact.RegisterProducerRequest.producers) - return producers_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactProducer >& -RegisterProducerRequest::producers() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.RegisterProducerRequest.producers) - return producers_; -} - -// ------------------------------------------------------------------- - -// ArtifactConsumer - -// .flyteidl.core.Identifier entity_id = 1; -inline bool ArtifactConsumer::has_entity_id() const { - return this != internal_default_instance() && entity_id_ != nullptr; -} -inline const ::flyteidl::core::Identifier& ArtifactConsumer::entity_id() const { - const ::flyteidl::core::Identifier* p = entity_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactConsumer.entity_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Identifier_default_instance_); -} -inline ::flyteidl::core::Identifier* ArtifactConsumer::release_entity_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactConsumer.entity_id) - - ::flyteidl::core::Identifier* temp = entity_id_; - entity_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::Identifier* ArtifactConsumer::mutable_entity_id() { - - if (entity_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Identifier>(GetArenaNoVirtual()); - entity_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactConsumer.entity_id) - return entity_id_; -} -inline void ArtifactConsumer::set_allocated_entity_id(::flyteidl::core::Identifier* entity_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(entity_id_); - } - if (entity_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - entity_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, entity_id, submessage_arena); - } - - } else { - - } - entity_id_ = entity_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactConsumer.entity_id) -} - -// .flyteidl.core.ParameterMap inputs = 2; -inline bool ArtifactConsumer::has_inputs() const { - return this != internal_default_instance() && inputs_ != nullptr; -} -inline const ::flyteidl::core::ParameterMap& ArtifactConsumer::inputs() const { - const ::flyteidl::core::ParameterMap* p = inputs_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ArtifactConsumer.inputs) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ParameterMap_default_instance_); -} -inline ::flyteidl::core::ParameterMap* ArtifactConsumer::release_inputs() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ArtifactConsumer.inputs) - - ::flyteidl::core::ParameterMap* temp = inputs_; - inputs_ = nullptr; - return temp; -} -inline ::flyteidl::core::ParameterMap* ArtifactConsumer::mutable_inputs() { - - if (inputs_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ParameterMap>(GetArenaNoVirtual()); - inputs_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ArtifactConsumer.inputs) - return inputs_; -} -inline void ArtifactConsumer::set_allocated_inputs(::flyteidl::core::ParameterMap* inputs) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(inputs_); - } - if (inputs) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - inputs = ::google::protobuf::internal::GetOwnedMessage( - message_arena, inputs, submessage_arena); - } - - } else { - - } - inputs_ = inputs; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ArtifactConsumer.inputs) -} - -// ------------------------------------------------------------------- - -// RegisterConsumerRequest - -// repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; -inline int RegisterConsumerRequest::consumers_size() const { - return consumers_.size(); -} -inline void RegisterConsumerRequest::clear_consumers() { - consumers_.Clear(); -} -inline ::flyteidl::artifact::ArtifactConsumer* RegisterConsumerRequest::mutable_consumers(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.RegisterConsumerRequest.consumers) - return consumers_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactConsumer >* -RegisterConsumerRequest::mutable_consumers() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.RegisterConsumerRequest.consumers) - return &consumers_; -} -inline const ::flyteidl::artifact::ArtifactConsumer& RegisterConsumerRequest::consumers(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.RegisterConsumerRequest.consumers) - return consumers_.Get(index); -} -inline ::flyteidl::artifact::ArtifactConsumer* RegisterConsumerRequest::add_consumers() { - // @@protoc_insertion_point(field_add:flyteidl.artifact.RegisterConsumerRequest.consumers) - return consumers_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::artifact::ArtifactConsumer >& -RegisterConsumerRequest::consumers() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.RegisterConsumerRequest.consumers) - return consumers_; -} - -// ------------------------------------------------------------------- - -// RegisterResponse - -// ------------------------------------------------------------------- - -// ExecutionInputsRequest - -// .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; -inline bool ExecutionInputsRequest::has_execution_id() const { - return this != internal_default_instance() && execution_id_ != nullptr; -} -inline const ::flyteidl::core::WorkflowExecutionIdentifier& ExecutionInputsRequest::execution_id() const { - const ::flyteidl::core::WorkflowExecutionIdentifier* p = execution_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ExecutionInputsRequest.execution_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_WorkflowExecutionIdentifier_default_instance_); -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ExecutionInputsRequest::release_execution_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ExecutionInputsRequest.execution_id) - - ::flyteidl::core::WorkflowExecutionIdentifier* temp = execution_id_; - execution_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ExecutionInputsRequest::mutable_execution_id() { - - if (execution_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::WorkflowExecutionIdentifier>(GetArenaNoVirtual()); - execution_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ExecutionInputsRequest.execution_id) - return execution_id_; -} -inline void ExecutionInputsRequest::set_allocated_execution_id(::flyteidl::core::WorkflowExecutionIdentifier* execution_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(execution_id_); - } - if (execution_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - execution_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, execution_id, submessage_arena); - } - - } else { - - } - execution_id_ = execution_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ExecutionInputsRequest.execution_id) -} - -// repeated .flyteidl.core.ArtifactID inputs = 2; -inline int ExecutionInputsRequest::inputs_size() const { - return inputs_.size(); -} -inline ::flyteidl::core::ArtifactID* ExecutionInputsRequest::mutable_inputs(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ExecutionInputsRequest.inputs) - return inputs_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >* -ExecutionInputsRequest::mutable_inputs() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.ExecutionInputsRequest.inputs) - return &inputs_; -} -inline const ::flyteidl::core::ArtifactID& ExecutionInputsRequest::inputs(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ExecutionInputsRequest.inputs) - return inputs_.Get(index); -} -inline ::flyteidl::core::ArtifactID* ExecutionInputsRequest::add_inputs() { - // @@protoc_insertion_point(field_add:flyteidl.artifact.ExecutionInputsRequest.inputs) - return inputs_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >& -ExecutionInputsRequest::inputs() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.ExecutionInputsRequest.inputs) - return inputs_; -} - -// ------------------------------------------------------------------- - -// ExecutionInputsResponse - -// ------------------------------------------------------------------- - -// ListUsageRequest - -// .flyteidl.core.ArtifactID artifact_id = 1; -inline bool ListUsageRequest::has_artifact_id() const { - return this != internal_default_instance() && artifact_id_ != nullptr; -} -inline const ::flyteidl::core::ArtifactID& ListUsageRequest::artifact_id() const { - const ::flyteidl::core::ArtifactID* p = artifact_id_; - // @@protoc_insertion_point(field_get:flyteidl.artifact.ListUsageRequest.artifact_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_ArtifactID_default_instance_); -} -inline ::flyteidl::core::ArtifactID* ListUsageRequest::release_artifact_id() { - // @@protoc_insertion_point(field_release:flyteidl.artifact.ListUsageRequest.artifact_id) - - ::flyteidl::core::ArtifactID* temp = artifact_id_; - artifact_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::ArtifactID* ListUsageRequest::mutable_artifact_id() { - - if (artifact_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::ArtifactID>(GetArenaNoVirtual()); - artifact_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ListUsageRequest.artifact_id) - return artifact_id_; -} -inline void ListUsageRequest::set_allocated_artifact_id(::flyteidl::core::ArtifactID* artifact_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(artifact_id_); - } - if (artifact_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - artifact_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, artifact_id, submessage_arena); - } - - } else { - - } - artifact_id_ = artifact_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.artifact.ListUsageRequest.artifact_id) -} - -// ------------------------------------------------------------------- - -// ListUsageResponse - -// repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; -inline int ListUsageResponse::executions_size() const { - return executions_.size(); -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ListUsageResponse::mutable_executions(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.artifact.ListUsageResponse.executions) - return executions_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::core::WorkflowExecutionIdentifier >* -ListUsageResponse::mutable_executions() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.artifact.ListUsageResponse.executions) - return &executions_; -} -inline const ::flyteidl::core::WorkflowExecutionIdentifier& ListUsageResponse::executions(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.artifact.ListUsageResponse.executions) - return executions_.Get(index); -} -inline ::flyteidl::core::WorkflowExecutionIdentifier* ListUsageResponse::add_executions() { - // @@protoc_insertion_point(field_add:flyteidl.artifact.ListUsageResponse.executions) - return executions_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::WorkflowExecutionIdentifier >& -ListUsageResponse::executions() const { - // @@protoc_insertion_point(field_list:flyteidl.artifact.ListUsageResponse.executions) - return executions_; -} - -#ifdef __GNUC__ - #pragma GCC diagnostic pop -#endif // __GNUC__ -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - -// ------------------------------------------------------------------- - - -// @@protoc_insertion_point(namespace_scope) - -} // namespace artifact -} // namespace flyteidl - -namespace google { -namespace protobuf { - -template <> struct is_proto_enum< ::flyteidl::artifact::FindByWorkflowExecRequest_Direction> : ::std::true_type {}; -template <> -inline const EnumDescriptor* GetEnumDescriptor< ::flyteidl::artifact::FindByWorkflowExecRequest_Direction>() { - return ::flyteidl::artifact::FindByWorkflowExecRequest_Direction_descriptor(); -} - -} // namespace protobuf -} // namespace google - -// @@protoc_insertion_point(global_scope) - -#include -#endif // PROTOBUF_INCLUDED_flyteidl_2fartifact_2fartifacts_2eproto diff --git a/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.cc b/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.cc index 265bdb6e63..29b1b207ed 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.cc +++ b/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.cc @@ -25,7 +25,6 @@ extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ArtifactTag_flyteidl_2fcore_2fartifact_5fid_2eproto; extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto; extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fartifact_5fid_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_LabelValue_flyteidl_2fcore_2fartifact_5fid_2eproto; -extern PROTOBUF_INTERNAL_EXPORT_flyteidl_2fcore_2fidentifier_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto; extern PROTOBUF_INTERNAL_EXPORT_google_2fprotobuf_2ftimestamp_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Timestamp_google_2fprotobuf_2ftimestamp_2eproto; namespace flyteidl { namespace core { @@ -79,10 +78,6 @@ class ArtifactQueryDefaultTypeInternal { ::google::protobuf::internal::ArenaStringPtr uri_; const ::flyteidl::core::ArtifactBindingData* binding_; } _ArtifactQuery_default_instance_; -class TriggerDefaultTypeInternal { - public: - ::google::protobuf::internal::ExplicitlyConstructed _instance; -} _Trigger_default_instance_; } // namespace core } // namespace flyteidl static void InitDefaultsArtifactKey_flyteidl_2fcore_2fartifact_5fid_2eproto() { @@ -238,22 +233,6 @@ ::google::protobuf::internal::SCCInfo<3> scc_info_ArtifactQuery_flyteidl_2fcore_ &scc_info_ArtifactTag_flyteidl_2fcore_2fartifact_5fid_2eproto.base, &scc_info_ArtifactBindingData_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; -static void InitDefaultsTrigger_flyteidl_2fcore_2fartifact_5fid_2eproto() { - GOOGLE_PROTOBUF_VERIFY_VERSION; - - { - void* ptr = &::flyteidl::core::_Trigger_default_instance_; - new (ptr) ::flyteidl::core::Trigger(); - ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); - } - ::flyteidl::core::Trigger::InitAsDefaultInstance(); -} - -::google::protobuf::internal::SCCInfo<2> scc_info_Trigger_flyteidl_2fcore_2fartifact_5fid_2eproto = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsTrigger_flyteidl_2fcore_2fartifact_5fid_2eproto}, { - &scc_info_Identifier_flyteidl_2fcore_2fidentifier_2eproto.base, - &scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base,}}; - void InitDefaults_flyteidl_2fcore_2fartifact_5fid_2eproto() { ::google::protobuf::internal::InitSCC(&scc_info_ArtifactKey_flyteidl_2fcore_2fartifact_5fid_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_ArtifactBindingData_flyteidl_2fcore_2fartifact_5fid_2eproto.base); @@ -265,10 +244,9 @@ void InitDefaults_flyteidl_2fcore_2fartifact_5fid_2eproto() { ::google::protobuf::internal::InitSCC(&scc_info_ArtifactID_flyteidl_2fcore_2fartifact_5fid_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_ArtifactTag_flyteidl_2fcore_2fartifact_5fid_2eproto.base); ::google::protobuf::internal::InitSCC(&scc_info_ArtifactQuery_flyteidl_2fcore_2fartifact_5fid_2eproto.base); - ::google::protobuf::internal::InitSCC(&scc_info_Trigger_flyteidl_2fcore_2fartifact_5fid_2eproto.base); } -::google::protobuf::Metadata file_level_metadata_flyteidl_2fcore_2fartifact_5fid_2eproto[11]; +::google::protobuf::Metadata file_level_metadata_flyteidl_2fcore_2fartifact_5fid_2eproto[10]; constexpr ::google::protobuf::EnumDescriptor const** file_level_enum_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto = nullptr; constexpr ::google::protobuf::ServiceDescriptor const** file_level_service_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto = nullptr; @@ -354,13 +332,6 @@ const ::google::protobuf::uint32 TableStruct_flyteidl_2fcore_2fartifact_5fid_2ep offsetof(::flyteidl::core::ArtifactQueryDefaultTypeInternal, uri_), offsetof(::flyteidl::core::ArtifactQueryDefaultTypeInternal, binding_), PROTOBUF_FIELD_OFFSET(::flyteidl::core::ArtifactQuery, identifier_), - ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::flyteidl::core::Trigger, _internal_metadata_), - ~0u, // no _extensions_ - ~0u, // no _oneof_case_ - ~0u, // no _weak_field_map_ - PROTOBUF_FIELD_OFFSET(::flyteidl::core::Trigger, trigger_id_), - PROTOBUF_FIELD_OFFSET(::flyteidl::core::Trigger, triggers_), }; static const ::google::protobuf::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { { 0, -1, sizeof(::flyteidl::core::ArtifactKey)}, @@ -373,7 +344,6 @@ static const ::google::protobuf::internal::MigrationSchema schemas[] PROTOBUF_SE { 55, -1, sizeof(::flyteidl::core::ArtifactID)}, { 64, -1, sizeof(::flyteidl::core::ArtifactTag)}, { 71, -1, sizeof(::flyteidl::core::ArtifactQuery)}, - { 81, -1, sizeof(::flyteidl::core::Trigger)}, }; static ::google::protobuf::Message const * const file_default_instances[] = { @@ -387,13 +357,12 @@ static ::google::protobuf::Message const * const file_default_instances[] = { reinterpret_cast(&::flyteidl::core::_ArtifactID_default_instance_), reinterpret_cast(&::flyteidl::core::_ArtifactTag_default_instance_), reinterpret_cast(&::flyteidl::core::_ArtifactQuery_default_instance_), - reinterpret_cast(&::flyteidl::core::_Trigger_default_instance_), }; ::google::protobuf::internal::AssignDescriptorsTable assign_descriptors_table_flyteidl_2fcore_2fartifact_5fid_2eproto = { {}, AddDescriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, "flyteidl/core/artifact_id.proto", schemas, file_default_instances, TableStruct_flyteidl_2fcore_2fartifact_5fid_2eproto::offsets, - file_level_metadata_flyteidl_2fcore_2fartifact_5fid_2eproto, 11, file_level_enum_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, file_level_service_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, + file_level_metadata_flyteidl_2fcore_2fartifact_5fid_2eproto, 10, file_level_enum_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, file_level_service_descriptors_flyteidl_2fcore_2fartifact_5fid_2eproto, }; const char descriptor_table_protodef_flyteidl_2fcore_2fartifact_5fid_2eproto[] = @@ -428,16 +397,14 @@ const char descriptor_table_protodef_flyteidl_2fcore_2fartifact_5fid_2eproto[] = "H\000\0222\n\014artifact_tag\030\002 \001(\0132\032.flyteidl.core" ".ArtifactTagH\000\022\r\n\003uri\030\003 \001(\tH\000\0225\n\007binding" "\030\004 \001(\0132\".flyteidl.core.ArtifactBindingDa" - "taH\000B\014\n\nidentifier\"e\n\007Trigger\022-\n\ntrigger" - "_id\030\001 \001(\0132\031.flyteidl.core.Identifier\022+\n\010" - "triggers\030\002 \003(\0132\031.flyteidl.core.ArtifactI" - "DBtrigger_id_ = const_cast< ::flyteidl::core::Identifier*>( - ::flyteidl::core::Identifier::internal_default_instance()); -} -class Trigger::HasBitSetters { - public: - static const ::flyteidl::core::Identifier& trigger_id(const Trigger* msg); -}; - -const ::flyteidl::core::Identifier& -Trigger::HasBitSetters::trigger_id(const Trigger* msg) { - return *msg->trigger_id_; -} -void Trigger::clear_trigger_id() { - if (GetArenaNoVirtual() == nullptr && trigger_id_ != nullptr) { - delete trigger_id_; - } - trigger_id_ = nullptr; -} -#if !defined(_MSC_VER) || _MSC_VER >= 1900 -const int Trigger::kTriggerIdFieldNumber; -const int Trigger::kTriggersFieldNumber; -#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 - -Trigger::Trigger() - : ::google::protobuf::Message(), _internal_metadata_(nullptr) { - SharedCtor(); - // @@protoc_insertion_point(constructor:flyteidl.core.Trigger) -} -Trigger::Trigger(const Trigger& from) - : ::google::protobuf::Message(), - _internal_metadata_(nullptr), - triggers_(from.triggers_) { - _internal_metadata_.MergeFrom(from._internal_metadata_); - if (from.has_trigger_id()) { - trigger_id_ = new ::flyteidl::core::Identifier(*from.trigger_id_); - } else { - trigger_id_ = nullptr; - } - // @@protoc_insertion_point(copy_constructor:flyteidl.core.Trigger) -} - -void Trigger::SharedCtor() { - ::google::protobuf::internal::InitSCC( - &scc_info_Trigger_flyteidl_2fcore_2fartifact_5fid_2eproto.base); - trigger_id_ = nullptr; -} - -Trigger::~Trigger() { - // @@protoc_insertion_point(destructor:flyteidl.core.Trigger) - SharedDtor(); -} - -void Trigger::SharedDtor() { - if (this != internal_default_instance()) delete trigger_id_; -} - -void Trigger::SetCachedSize(int size) const { - _cached_size_.Set(size); -} -const Trigger& Trigger::default_instance() { - ::google::protobuf::internal::InitSCC(&::scc_info_Trigger_flyteidl_2fcore_2fartifact_5fid_2eproto.base); - return *internal_default_instance(); -} - - -void Trigger::Clear() { -// @@protoc_insertion_point(message_clear_start:flyteidl.core.Trigger) - ::google::protobuf::uint32 cached_has_bits = 0; - // Prevent compiler warnings about cached_has_bits being unused - (void) cached_has_bits; - - triggers_.Clear(); - if (GetArenaNoVirtual() == nullptr && trigger_id_ != nullptr) { - delete trigger_id_; - } - trigger_id_ = nullptr; - _internal_metadata_.Clear(); -} - -#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER -const char* Trigger::_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.Identifier trigger_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::Identifier::_InternalParse; - object = msg->mutable_trigger_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; - } - // repeated .flyteidl.core.ArtifactID triggers = 2; - case 2: { - if (static_cast<::google::protobuf::uint8>(tag) != 18) goto handle_unusual; - do { - ptr = ::google::protobuf::io::ReadSize(ptr, &size); - GOOGLE_PROTOBUF_PARSER_ASSERT(ptr); - parser_till_end = ::flyteidl::core::ArtifactID::_InternalParse; - object = msg->add_triggers(); - if (size > end - ptr) goto len_delim_till_end; - ptr += size; - GOOGLE_PROTOBUF_PARSER_ASSERT(ctx->ParseExactRange( - {parser_till_end, object}, ptr - size, ptr)); - if (ptr >= end) break; - } while ((::google::protobuf::io::UnalignedLoad<::google::protobuf::uint64>(ptr) & 255) == 18 && (ptr += 1)); - 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 Trigger::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.core.Trigger) - 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.Identifier trigger_id = 1; - case 1: { - if (static_cast< ::google::protobuf::uint8>(tag) == (10 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, mutable_trigger_id())); - } else { - goto handle_unusual; - } - break; - } - - // repeated .flyteidl.core.ArtifactID triggers = 2; - case 2: { - if (static_cast< ::google::protobuf::uint8>(tag) == (18 & 0xFF)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( - input, add_triggers())); - } 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.core.Trigger) - return true; -failure: - // @@protoc_insertion_point(parse_failure:flyteidl.core.Trigger) - return false; -#undef DO_ -} -#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER - -void Trigger::SerializeWithCachedSizes( - ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:flyteidl.core.Trigger) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 1, HasBitSetters::trigger_id(this), output); - } - - // repeated .flyteidl.core.ArtifactID triggers = 2; - for (unsigned int i = 0, - n = static_cast(this->triggers_size()); i < n; i++) { - ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( - 2, - this->triggers(static_cast(i)), - output); - } - - if (_internal_metadata_.have_unknown_fields()) { - ::google::protobuf::internal::WireFormat::SerializeUnknownFields( - _internal_metadata_.unknown_fields(), output); - } - // @@protoc_insertion_point(serialize_end:flyteidl.core.Trigger) -} - -::google::protobuf::uint8* Trigger::InternalSerializeWithCachedSizesToArray( - ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:flyteidl.core.Trigger) - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - // .flyteidl.core.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 1, HasBitSetters::trigger_id(this), target); - } - - // repeated .flyteidl.core.ArtifactID triggers = 2; - for (unsigned int i = 0, - n = static_cast(this->triggers_size()); i < n; i++) { - target = ::google::protobuf::internal::WireFormatLite:: - InternalWriteMessageToArray( - 2, this->triggers(static_cast(i)), 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.core.Trigger) - return target; -} - -size_t Trigger::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:flyteidl.core.Trigger) - 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; - - // repeated .flyteidl.core.ArtifactID triggers = 2; - { - unsigned int count = static_cast(this->triggers_size()); - total_size += 1UL * count; - for (unsigned int i = 0; i < count; i++) { - total_size += - ::google::protobuf::internal::WireFormatLite::MessageSize( - this->triggers(static_cast(i))); - } - } - - // .flyteidl.core.Identifier trigger_id = 1; - if (this->has_trigger_id()) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::MessageSize( - *trigger_id_); - } - - int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); - SetCachedSize(cached_size); - return total_size; -} - -void Trigger::MergeFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_merge_from_start:flyteidl.core.Trigger) - GOOGLE_DCHECK_NE(&from, this); - const Trigger* source = - ::google::protobuf::DynamicCastToGenerated( - &from); - if (source == nullptr) { - // @@protoc_insertion_point(generalized_merge_from_cast_fail:flyteidl.core.Trigger) - ::google::protobuf::internal::ReflectionOps::Merge(from, this); - } else { - // @@protoc_insertion_point(generalized_merge_from_cast_success:flyteidl.core.Trigger) - MergeFrom(*source); - } -} - -void Trigger::MergeFrom(const Trigger& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:flyteidl.core.Trigger) - GOOGLE_DCHECK_NE(&from, this); - _internal_metadata_.MergeFrom(from._internal_metadata_); - ::google::protobuf::uint32 cached_has_bits = 0; - (void) cached_has_bits; - - triggers_.MergeFrom(from.triggers_); - if (from.has_trigger_id()) { - mutable_trigger_id()->::flyteidl::core::Identifier::MergeFrom(from.trigger_id()); - } -} - -void Trigger::CopyFrom(const ::google::protobuf::Message& from) { -// @@protoc_insertion_point(generalized_copy_from_start:flyteidl.core.Trigger) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -void Trigger::CopyFrom(const Trigger& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:flyteidl.core.Trigger) - if (&from == this) return; - Clear(); - MergeFrom(from); -} - -bool Trigger::IsInitialized() const { - return true; -} - -void Trigger::Swap(Trigger* other) { - if (other == this) return; - InternalSwap(other); -} -void Trigger::InternalSwap(Trigger* other) { - using std::swap; - _internal_metadata_.Swap(&other->_internal_metadata_); - CastToBase(&triggers_)->InternalSwap(CastToBase(&other->triggers_)); - swap(trigger_id_, other->trigger_id_); -} - -::google::protobuf::Metadata Trigger::GetMetadata() const { - ::google::protobuf::internal::AssignDescriptors(&::assign_descriptors_table_flyteidl_2fcore_2fartifact_5fid_2eproto); - return ::file_level_metadata_flyteidl_2fcore_2fartifact_5fid_2eproto[kIndexInFileMessages]; -} - - // @@protoc_insertion_point(namespace_scope) } // namespace core } // namespace flyteidl @@ -4768,9 +4382,6 @@ template<> PROTOBUF_NOINLINE ::flyteidl::core::ArtifactTag* Arena::CreateMaybeMe template<> PROTOBUF_NOINLINE ::flyteidl::core::ArtifactQuery* Arena::CreateMaybeMessage< ::flyteidl::core::ArtifactQuery >(Arena* arena) { return Arena::CreateInternal< ::flyteidl::core::ArtifactQuery >(arena); } -template<> PROTOBUF_NOINLINE ::flyteidl::core::Trigger* Arena::CreateMaybeMessage< ::flyteidl::core::Trigger >(Arena* arena) { - return Arena::CreateInternal< ::flyteidl::core::Trigger >(arena); -} } // namespace protobuf } // namespace google diff --git a/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.h b/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.h index fc48d878e1..5efd7a8cfb 100644 --- a/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.h +++ b/flyteidl/gen/pb-cpp/flyteidl/core/artifact_id.pb.h @@ -46,7 +46,7 @@ struct TableStruct_flyteidl_2fcore_2fartifact_5fid_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[11] + static const ::google::protobuf::internal::ParseTable schema[10] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::google::protobuf::internal::FieldMetadata field_metadata[]; static const ::google::protobuf::internal::SerializationTable serialization_table[]; @@ -85,9 +85,6 @@ extern Partitions_ValueEntry_DoNotUseDefaultTypeInternal _Partitions_ValueEntry_ class TimePartition; class TimePartitionDefaultTypeInternal; extern TimePartitionDefaultTypeInternal _TimePartition_default_instance_; -class Trigger; -class TriggerDefaultTypeInternal; -extern TriggerDefaultTypeInternal _Trigger_default_instance_; } // namespace core } // namespace flyteidl namespace google { @@ -102,7 +99,6 @@ template<> ::flyteidl::core::LabelValue* Arena::CreateMaybeMessage<::flyteidl::c template<> ::flyteidl::core::Partitions* Arena::CreateMaybeMessage<::flyteidl::core::Partitions>(Arena*); template<> ::flyteidl::core::Partitions_ValueEntry_DoNotUse* Arena::CreateMaybeMessage<::flyteidl::core::Partitions_ValueEntry_DoNotUse>(Arena*); template<> ::flyteidl::core::TimePartition* Arena::CreateMaybeMessage<::flyteidl::core::TimePartition>(Arena*); -template<> ::flyteidl::core::Trigger* Arena::CreateMaybeMessage<::flyteidl::core::Trigger>(Arena*); } // namespace protobuf } // namespace google namespace flyteidl { @@ -1436,134 +1432,6 @@ class ArtifactQuery final : friend struct ::TableStruct_flyteidl_2fcore_2fartifact_5fid_2eproto; }; -// ------------------------------------------------------------------- - -class Trigger final : - public ::google::protobuf::Message /* @@protoc_insertion_point(class_definition:flyteidl.core.Trigger) */ { - public: - Trigger(); - virtual ~Trigger(); - - Trigger(const Trigger& from); - - inline Trigger& operator=(const Trigger& from) { - CopyFrom(from); - return *this; - } - #if LANG_CXX11 - Trigger(Trigger&& from) noexcept - : Trigger() { - *this = ::std::move(from); - } - - inline Trigger& operator=(Trigger&& 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 Trigger& default_instance(); - - static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY - static inline const Trigger* internal_default_instance() { - return reinterpret_cast( - &_Trigger_default_instance_); - } - static constexpr int kIndexInFileMessages = - 10; - - void Swap(Trigger* other); - friend void swap(Trigger& a, Trigger& b) { - a.Swap(&b); - } - - // implements Message ---------------------------------------------- - - inline Trigger* New() const final { - return CreateMaybeMessage(nullptr); - } - - Trigger* 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 Trigger& from); - void MergeFrom(const Trigger& 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(Trigger* 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 ------------------------------------------------------- - - // repeated .flyteidl.core.ArtifactID triggers = 2; - int triggers_size() const; - void clear_triggers(); - static const int kTriggersFieldNumber = 2; - ::flyteidl::core::ArtifactID* mutable_triggers(int index); - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >* - mutable_triggers(); - const ::flyteidl::core::ArtifactID& triggers(int index) const; - ::flyteidl::core::ArtifactID* add_triggers(); - const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >& - triggers() const; - - // .flyteidl.core.Identifier trigger_id = 1; - bool has_trigger_id() const; - void clear_trigger_id(); - static const int kTriggerIdFieldNumber = 1; - const ::flyteidl::core::Identifier& trigger_id() const; - ::flyteidl::core::Identifier* release_trigger_id(); - ::flyteidl::core::Identifier* mutable_trigger_id(); - void set_allocated_trigger_id(::flyteidl::core::Identifier* trigger_id); - - // @@protoc_insertion_point(class_scope:flyteidl.core.Trigger) - private: - class HasBitSetters; - - ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; - ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID > triggers_; - ::flyteidl::core::Identifier* trigger_id_; - mutable ::google::protobuf::internal::CachedSize _cached_size_; - friend struct ::TableStruct_flyteidl_2fcore_2fartifact_5fid_2eproto; -}; // =================================================================== @@ -2837,85 +2705,6 @@ inline void ArtifactQuery::clear_has_identifier() { inline ArtifactQuery::IdentifierCase ArtifactQuery::identifier_case() const { return ArtifactQuery::IdentifierCase(_oneof_case_[0]); } -// ------------------------------------------------------------------- - -// Trigger - -// .flyteidl.core.Identifier trigger_id = 1; -inline bool Trigger::has_trigger_id() const { - return this != internal_default_instance() && trigger_id_ != nullptr; -} -inline const ::flyteidl::core::Identifier& Trigger::trigger_id() const { - const ::flyteidl::core::Identifier* p = trigger_id_; - // @@protoc_insertion_point(field_get:flyteidl.core.Trigger.trigger_id) - return p != nullptr ? *p : *reinterpret_cast( - &::flyteidl::core::_Identifier_default_instance_); -} -inline ::flyteidl::core::Identifier* Trigger::release_trigger_id() { - // @@protoc_insertion_point(field_release:flyteidl.core.Trigger.trigger_id) - - ::flyteidl::core::Identifier* temp = trigger_id_; - trigger_id_ = nullptr; - return temp; -} -inline ::flyteidl::core::Identifier* Trigger::mutable_trigger_id() { - - if (trigger_id_ == nullptr) { - auto* p = CreateMaybeMessage<::flyteidl::core::Identifier>(GetArenaNoVirtual()); - trigger_id_ = p; - } - // @@protoc_insertion_point(field_mutable:flyteidl.core.Trigger.trigger_id) - return trigger_id_; -} -inline void Trigger::set_allocated_trigger_id(::flyteidl::core::Identifier* trigger_id) { - ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::google::protobuf::MessageLite*>(trigger_id_); - } - if (trigger_id) { - ::google::protobuf::Arena* submessage_arena = nullptr; - if (message_arena != submessage_arena) { - trigger_id = ::google::protobuf::internal::GetOwnedMessage( - message_arena, trigger_id, submessage_arena); - } - - } else { - - } - trigger_id_ = trigger_id; - // @@protoc_insertion_point(field_set_allocated:flyteidl.core.Trigger.trigger_id) -} - -// repeated .flyteidl.core.ArtifactID triggers = 2; -inline int Trigger::triggers_size() const { - return triggers_.size(); -} -inline void Trigger::clear_triggers() { - triggers_.Clear(); -} -inline ::flyteidl::core::ArtifactID* Trigger::mutable_triggers(int index) { - // @@protoc_insertion_point(field_mutable:flyteidl.core.Trigger.triggers) - return triggers_.Mutable(index); -} -inline ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >* -Trigger::mutable_triggers() { - // @@protoc_insertion_point(field_mutable_list:flyteidl.core.Trigger.triggers) - return &triggers_; -} -inline const ::flyteidl::core::ArtifactID& Trigger::triggers(int index) const { - // @@protoc_insertion_point(field_get:flyteidl.core.Trigger.triggers) - return triggers_.Get(index); -} -inline ::flyteidl::core::ArtifactID* Trigger::add_triggers() { - // @@protoc_insertion_point(field_add:flyteidl.core.Trigger.triggers) - return triggers_.Add(); -} -inline const ::google::protobuf::RepeatedPtrField< ::flyteidl::core::ArtifactID >& -Trigger::triggers() const { - // @@protoc_insertion_point(field_list:flyteidl.core.Trigger.triggers) - return triggers_; -} - #ifdef __GNUC__ #pragma GCC diagnostic pop #endif // __GNUC__ @@ -2937,8 +2726,6 @@ Trigger::triggers() const { // ------------------------------------------------------------------- -// ------------------------------------------------------------------- - // @@protoc_insertion_point(namespace_scope) diff --git a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.go b/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.go deleted file mode 100644 index b0ad21ba74..0000000000 --- a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.go +++ /dev/null @@ -1,1908 +0,0 @@ -// Code generated by protoc-gen-go. DO NOT EDIT. -// source: flyteidl/artifact/artifacts.proto - -package artifact - -import ( - context "context" - fmt "fmt" - admin "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" - core "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" - _ "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" - proto "github.com/golang/protobuf/proto" - any "github.com/golang/protobuf/ptypes/any" - timestamp "github.com/golang/protobuf/ptypes/timestamp" - _ "google.golang.org/genproto/googleapis/api/annotations" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" - math "math" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package - -type FindByWorkflowExecRequest_Direction int32 - -const ( - FindByWorkflowExecRequest_INPUTS FindByWorkflowExecRequest_Direction = 0 - FindByWorkflowExecRequest_OUTPUTS FindByWorkflowExecRequest_Direction = 1 -) - -var FindByWorkflowExecRequest_Direction_name = map[int32]string{ - 0: "INPUTS", - 1: "OUTPUTS", -} - -var FindByWorkflowExecRequest_Direction_value = map[string]int32{ - "INPUTS": 0, - "OUTPUTS": 1, -} - -func (x FindByWorkflowExecRequest_Direction) String() string { - return proto.EnumName(FindByWorkflowExecRequest_Direction_name, int32(x)) -} - -func (FindByWorkflowExecRequest_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{10, 0} -} - -type Artifact struct { - ArtifactId *core.ArtifactID `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"` - Spec *ArtifactSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` - // references the tag field in ArtifactTag - Tags []string `protobuf:"bytes,3,rep,name=tags,proto3" json:"tags,omitempty"` - Source *ArtifactSource `protobuf:"bytes,4,opt,name=source,proto3" json:"source,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Artifact) Reset() { *m = Artifact{} } -func (m *Artifact) String() string { return proto.CompactTextString(m) } -func (*Artifact) ProtoMessage() {} -func (*Artifact) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{0} -} - -func (m *Artifact) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Artifact.Unmarshal(m, b) -} -func (m *Artifact) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Artifact.Marshal(b, m, deterministic) -} -func (m *Artifact) XXX_Merge(src proto.Message) { - xxx_messageInfo_Artifact.Merge(m, src) -} -func (m *Artifact) XXX_Size() int { - return xxx_messageInfo_Artifact.Size(m) -} -func (m *Artifact) XXX_DiscardUnknown() { - xxx_messageInfo_Artifact.DiscardUnknown(m) -} - -var xxx_messageInfo_Artifact proto.InternalMessageInfo - -func (m *Artifact) GetArtifactId() *core.ArtifactID { - if m != nil { - return m.ArtifactId - } - return nil -} - -func (m *Artifact) GetSpec() *ArtifactSpec { - if m != nil { - return m.Spec - } - return nil -} - -func (m *Artifact) GetTags() []string { - if m != nil { - return m.Tags - } - return nil -} - -func (m *Artifact) GetSource() *ArtifactSource { - if m != nil { - return m.Source - } - return nil -} - -type CreateArtifactRequest struct { - // Specify just project/domain on creation - ArtifactKey *core.ArtifactKey `protobuf:"bytes,1,opt,name=artifact_key,json=artifactKey,proto3" json:"artifact_key,omitempty"` - Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"` - Spec *ArtifactSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` - Partitions map[string]string `protobuf:"bytes,4,rep,name=partitions,proto3" json:"partitions,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - TimePartitionValue *timestamp.Timestamp `protobuf:"bytes,5,opt,name=time_partition_value,json=timePartitionValue,proto3" json:"time_partition_value,omitempty"` - Source *ArtifactSource `protobuf:"bytes,6,opt,name=source,proto3" json:"source,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateArtifactRequest) Reset() { *m = CreateArtifactRequest{} } -func (m *CreateArtifactRequest) String() string { return proto.CompactTextString(m) } -func (*CreateArtifactRequest) ProtoMessage() {} -func (*CreateArtifactRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{1} -} - -func (m *CreateArtifactRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateArtifactRequest.Unmarshal(m, b) -} -func (m *CreateArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateArtifactRequest.Marshal(b, m, deterministic) -} -func (m *CreateArtifactRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateArtifactRequest.Merge(m, src) -} -func (m *CreateArtifactRequest) XXX_Size() int { - return xxx_messageInfo_CreateArtifactRequest.Size(m) -} -func (m *CreateArtifactRequest) XXX_DiscardUnknown() { - xxx_messageInfo_CreateArtifactRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateArtifactRequest proto.InternalMessageInfo - -func (m *CreateArtifactRequest) GetArtifactKey() *core.ArtifactKey { - if m != nil { - return m.ArtifactKey - } - return nil -} - -func (m *CreateArtifactRequest) GetVersion() string { - if m != nil { - return m.Version - } - return "" -} - -func (m *CreateArtifactRequest) GetSpec() *ArtifactSpec { - if m != nil { - return m.Spec - } - return nil -} - -func (m *CreateArtifactRequest) GetPartitions() map[string]string { - if m != nil { - return m.Partitions - } - return nil -} - -func (m *CreateArtifactRequest) GetTimePartitionValue() *timestamp.Timestamp { - if m != nil { - return m.TimePartitionValue - } - return nil -} - -func (m *CreateArtifactRequest) GetSource() *ArtifactSource { - if m != nil { - return m.Source - } - return nil -} - -type ArtifactSource struct { - WorkflowExecution *core.WorkflowExecutionIdentifier `protobuf:"bytes,1,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` - NodeId string `protobuf:"bytes,2,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - TaskId *core.Identifier `protobuf:"bytes,3,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` - RetryAttempt uint32 `protobuf:"varint,4,opt,name=retry_attempt,json=retryAttempt,proto3" json:"retry_attempt,omitempty"` - // Uploads, either from the UI or from the CLI, or FlyteRemote, will have this. - Principal string `protobuf:"bytes,5,opt,name=principal,proto3" json:"principal,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ArtifactSource) Reset() { *m = ArtifactSource{} } -func (m *ArtifactSource) String() string { return proto.CompactTextString(m) } -func (*ArtifactSource) ProtoMessage() {} -func (*ArtifactSource) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{2} -} - -func (m *ArtifactSource) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ArtifactSource.Unmarshal(m, b) -} -func (m *ArtifactSource) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ArtifactSource.Marshal(b, m, deterministic) -} -func (m *ArtifactSource) XXX_Merge(src proto.Message) { - xxx_messageInfo_ArtifactSource.Merge(m, src) -} -func (m *ArtifactSource) XXX_Size() int { - return xxx_messageInfo_ArtifactSource.Size(m) -} -func (m *ArtifactSource) XXX_DiscardUnknown() { - xxx_messageInfo_ArtifactSource.DiscardUnknown(m) -} - -var xxx_messageInfo_ArtifactSource proto.InternalMessageInfo - -func (m *ArtifactSource) GetWorkflowExecution() *core.WorkflowExecutionIdentifier { - if m != nil { - return m.WorkflowExecution - } - return nil -} - -func (m *ArtifactSource) GetNodeId() string { - if m != nil { - return m.NodeId - } - return "" -} - -func (m *ArtifactSource) GetTaskId() *core.Identifier { - if m != nil { - return m.TaskId - } - return nil -} - -func (m *ArtifactSource) GetRetryAttempt() uint32 { - if m != nil { - return m.RetryAttempt - } - return 0 -} - -func (m *ArtifactSource) GetPrincipal() string { - if m != nil { - return m.Principal - } - return "" -} - -type ArtifactSpec struct { - Value *core.Literal `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` - // This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but - // forgets to change the name, that is okay. And the reason why this is a separate field is because adding the - // type to all Literals is a lot of work. - Type *core.LiteralType `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` - ShortDescription string `protobuf:"bytes,3,opt,name=short_description,json=shortDescription,proto3" json:"short_description,omitempty"` - // Additional user metadata - UserMetadata *any.Any `protobuf:"bytes,4,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` - MetadataType string `protobuf:"bytes,5,opt,name=metadata_type,json=metadataType,proto3" json:"metadata_type,omitempty"` - CreatedAt *timestamp.Timestamp `protobuf:"bytes,6,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` - FileFormat string `protobuf:"bytes,7,opt,name=file_format,json=fileFormat,proto3" json:"file_format,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ArtifactSpec) Reset() { *m = ArtifactSpec{} } -func (m *ArtifactSpec) String() string { return proto.CompactTextString(m) } -func (*ArtifactSpec) ProtoMessage() {} -func (*ArtifactSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{3} -} - -func (m *ArtifactSpec) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ArtifactSpec.Unmarshal(m, b) -} -func (m *ArtifactSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ArtifactSpec.Marshal(b, m, deterministic) -} -func (m *ArtifactSpec) XXX_Merge(src proto.Message) { - xxx_messageInfo_ArtifactSpec.Merge(m, src) -} -func (m *ArtifactSpec) XXX_Size() int { - return xxx_messageInfo_ArtifactSpec.Size(m) -} -func (m *ArtifactSpec) XXX_DiscardUnknown() { - xxx_messageInfo_ArtifactSpec.DiscardUnknown(m) -} - -var xxx_messageInfo_ArtifactSpec proto.InternalMessageInfo - -func (m *ArtifactSpec) GetValue() *core.Literal { - if m != nil { - return m.Value - } - return nil -} - -func (m *ArtifactSpec) GetType() *core.LiteralType { - if m != nil { - return m.Type - } - return nil -} - -func (m *ArtifactSpec) GetShortDescription() string { - if m != nil { - return m.ShortDescription - } - return "" -} - -func (m *ArtifactSpec) GetUserMetadata() *any.Any { - if m != nil { - return m.UserMetadata - } - return nil -} - -func (m *ArtifactSpec) GetMetadataType() string { - if m != nil { - return m.MetadataType - } - return "" -} - -func (m *ArtifactSpec) GetCreatedAt() *timestamp.Timestamp { - if m != nil { - return m.CreatedAt - } - return nil -} - -func (m *ArtifactSpec) GetFileFormat() string { - if m != nil { - return m.FileFormat - } - return "" -} - -type CreateArtifactResponse struct { - Artifact *Artifact `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateArtifactResponse) Reset() { *m = CreateArtifactResponse{} } -func (m *CreateArtifactResponse) String() string { return proto.CompactTextString(m) } -func (*CreateArtifactResponse) ProtoMessage() {} -func (*CreateArtifactResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{4} -} - -func (m *CreateArtifactResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateArtifactResponse.Unmarshal(m, b) -} -func (m *CreateArtifactResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateArtifactResponse.Marshal(b, m, deterministic) -} -func (m *CreateArtifactResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateArtifactResponse.Merge(m, src) -} -func (m *CreateArtifactResponse) XXX_Size() int { - return xxx_messageInfo_CreateArtifactResponse.Size(m) -} -func (m *CreateArtifactResponse) XXX_DiscardUnknown() { - xxx_messageInfo_CreateArtifactResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateArtifactResponse proto.InternalMessageInfo - -func (m *CreateArtifactResponse) GetArtifact() *Artifact { - if m != nil { - return m.Artifact - } - return nil -} - -type GetArtifactRequest struct { - Query *core.ArtifactQuery `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` - // If false, then long_description is not returned. - Details bool `protobuf:"varint,2,opt,name=details,proto3" json:"details,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetArtifactRequest) Reset() { *m = GetArtifactRequest{} } -func (m *GetArtifactRequest) String() string { return proto.CompactTextString(m) } -func (*GetArtifactRequest) ProtoMessage() {} -func (*GetArtifactRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{5} -} - -func (m *GetArtifactRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetArtifactRequest.Unmarshal(m, b) -} -func (m *GetArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetArtifactRequest.Marshal(b, m, deterministic) -} -func (m *GetArtifactRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetArtifactRequest.Merge(m, src) -} -func (m *GetArtifactRequest) XXX_Size() int { - return xxx_messageInfo_GetArtifactRequest.Size(m) -} -func (m *GetArtifactRequest) XXX_DiscardUnknown() { - xxx_messageInfo_GetArtifactRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_GetArtifactRequest proto.InternalMessageInfo - -func (m *GetArtifactRequest) GetQuery() *core.ArtifactQuery { - if m != nil { - return m.Query - } - return nil -} - -func (m *GetArtifactRequest) GetDetails() bool { - if m != nil { - return m.Details - } - return false -} - -type GetArtifactResponse struct { - Artifact *Artifact `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetArtifactResponse) Reset() { *m = GetArtifactResponse{} } -func (m *GetArtifactResponse) String() string { return proto.CompactTextString(m) } -func (*GetArtifactResponse) ProtoMessage() {} -func (*GetArtifactResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{6} -} - -func (m *GetArtifactResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetArtifactResponse.Unmarshal(m, b) -} -func (m *GetArtifactResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetArtifactResponse.Marshal(b, m, deterministic) -} -func (m *GetArtifactResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetArtifactResponse.Merge(m, src) -} -func (m *GetArtifactResponse) XXX_Size() int { - return xxx_messageInfo_GetArtifactResponse.Size(m) -} -func (m *GetArtifactResponse) XXX_DiscardUnknown() { - xxx_messageInfo_GetArtifactResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_GetArtifactResponse proto.InternalMessageInfo - -func (m *GetArtifactResponse) GetArtifact() *Artifact { - if m != nil { - return m.Artifact - } - return nil -} - -type SearchOptions struct { - // If true, this means a strict partition search. meaning if you don't specify the partition - // field, that will mean, non-partitioned, rather than any partition. - StrictPartitions bool `protobuf:"varint,1,opt,name=strict_partitions,json=strictPartitions,proto3" json:"strict_partitions,omitempty"` - // If true, only one artifact per key will be returned. It will be the latest one by creation time. - LatestByKey bool `protobuf:"varint,2,opt,name=latest_by_key,json=latestByKey,proto3" json:"latest_by_key,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SearchOptions) Reset() { *m = SearchOptions{} } -func (m *SearchOptions) String() string { return proto.CompactTextString(m) } -func (*SearchOptions) ProtoMessage() {} -func (*SearchOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{7} -} - -func (m *SearchOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SearchOptions.Unmarshal(m, b) -} -func (m *SearchOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SearchOptions.Marshal(b, m, deterministic) -} -func (m *SearchOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_SearchOptions.Merge(m, src) -} -func (m *SearchOptions) XXX_Size() int { - return xxx_messageInfo_SearchOptions.Size(m) -} -func (m *SearchOptions) XXX_DiscardUnknown() { - xxx_messageInfo_SearchOptions.DiscardUnknown(m) -} - -var xxx_messageInfo_SearchOptions proto.InternalMessageInfo - -func (m *SearchOptions) GetStrictPartitions() bool { - if m != nil { - return m.StrictPartitions - } - return false -} - -func (m *SearchOptions) GetLatestByKey() bool { - if m != nil { - return m.LatestByKey - } - return false -} - -type SearchArtifactsRequest struct { - ArtifactKey *core.ArtifactKey `protobuf:"bytes,1,opt,name=artifact_key,json=artifactKey,proto3" json:"artifact_key,omitempty"` - Partitions *core.Partitions `protobuf:"bytes,2,opt,name=partitions,proto3" json:"partitions,omitempty"` - TimePartitionValue *timestamp.Timestamp `protobuf:"bytes,3,opt,name=time_partition_value,json=timePartitionValue,proto3" json:"time_partition_value,omitempty"` - Principal string `protobuf:"bytes,4,opt,name=principal,proto3" json:"principal,omitempty"` - Version string `protobuf:"bytes,5,opt,name=version,proto3" json:"version,omitempty"` - Options *SearchOptions `protobuf:"bytes,6,opt,name=options,proto3" json:"options,omitempty"` - Token string `protobuf:"bytes,7,opt,name=token,proto3" json:"token,omitempty"` - Limit int32 `protobuf:"varint,8,opt,name=limit,proto3" json:"limit,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SearchArtifactsRequest) Reset() { *m = SearchArtifactsRequest{} } -func (m *SearchArtifactsRequest) String() string { return proto.CompactTextString(m) } -func (*SearchArtifactsRequest) ProtoMessage() {} -func (*SearchArtifactsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{8} -} - -func (m *SearchArtifactsRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SearchArtifactsRequest.Unmarshal(m, b) -} -func (m *SearchArtifactsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SearchArtifactsRequest.Marshal(b, m, deterministic) -} -func (m *SearchArtifactsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_SearchArtifactsRequest.Merge(m, src) -} -func (m *SearchArtifactsRequest) XXX_Size() int { - return xxx_messageInfo_SearchArtifactsRequest.Size(m) -} -func (m *SearchArtifactsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_SearchArtifactsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_SearchArtifactsRequest proto.InternalMessageInfo - -func (m *SearchArtifactsRequest) GetArtifactKey() *core.ArtifactKey { - if m != nil { - return m.ArtifactKey - } - return nil -} - -func (m *SearchArtifactsRequest) GetPartitions() *core.Partitions { - if m != nil { - return m.Partitions - } - return nil -} - -func (m *SearchArtifactsRequest) GetTimePartitionValue() *timestamp.Timestamp { - if m != nil { - return m.TimePartitionValue - } - return nil -} - -func (m *SearchArtifactsRequest) GetPrincipal() string { - if m != nil { - return m.Principal - } - return "" -} - -func (m *SearchArtifactsRequest) GetVersion() string { - if m != nil { - return m.Version - } - return "" -} - -func (m *SearchArtifactsRequest) GetOptions() *SearchOptions { - if m != nil { - return m.Options - } - return nil -} - -func (m *SearchArtifactsRequest) GetToken() string { - if m != nil { - return m.Token - } - return "" -} - -func (m *SearchArtifactsRequest) GetLimit() int32 { - if m != nil { - return m.Limit - } - return 0 -} - -type SearchArtifactsResponse struct { - // If artifact specs are not requested, the resultant artifacts may be empty. - Artifacts []*Artifact `protobuf:"bytes,1,rep,name=artifacts,proto3" json:"artifacts,omitempty"` - // continuation token if relevant. - Token string `protobuf:"bytes,2,opt,name=token,proto3" json:"token,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SearchArtifactsResponse) Reset() { *m = SearchArtifactsResponse{} } -func (m *SearchArtifactsResponse) String() string { return proto.CompactTextString(m) } -func (*SearchArtifactsResponse) ProtoMessage() {} -func (*SearchArtifactsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{9} -} - -func (m *SearchArtifactsResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SearchArtifactsResponse.Unmarshal(m, b) -} -func (m *SearchArtifactsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SearchArtifactsResponse.Marshal(b, m, deterministic) -} -func (m *SearchArtifactsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_SearchArtifactsResponse.Merge(m, src) -} -func (m *SearchArtifactsResponse) XXX_Size() int { - return xxx_messageInfo_SearchArtifactsResponse.Size(m) -} -func (m *SearchArtifactsResponse) XXX_DiscardUnknown() { - xxx_messageInfo_SearchArtifactsResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_SearchArtifactsResponse proto.InternalMessageInfo - -func (m *SearchArtifactsResponse) GetArtifacts() []*Artifact { - if m != nil { - return m.Artifacts - } - return nil -} - -func (m *SearchArtifactsResponse) GetToken() string { - if m != nil { - return m.Token - } - return "" -} - -type FindByWorkflowExecRequest struct { - ExecId *core.WorkflowExecutionIdentifier `protobuf:"bytes,1,opt,name=exec_id,json=execId,proto3" json:"exec_id,omitempty"` - Direction FindByWorkflowExecRequest_Direction `protobuf:"varint,2,opt,name=direction,proto3,enum=flyteidl.artifact.FindByWorkflowExecRequest_Direction" json:"direction,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *FindByWorkflowExecRequest) Reset() { *m = FindByWorkflowExecRequest{} } -func (m *FindByWorkflowExecRequest) String() string { return proto.CompactTextString(m) } -func (*FindByWorkflowExecRequest) ProtoMessage() {} -func (*FindByWorkflowExecRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{10} -} - -func (m *FindByWorkflowExecRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_FindByWorkflowExecRequest.Unmarshal(m, b) -} -func (m *FindByWorkflowExecRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_FindByWorkflowExecRequest.Marshal(b, m, deterministic) -} -func (m *FindByWorkflowExecRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_FindByWorkflowExecRequest.Merge(m, src) -} -func (m *FindByWorkflowExecRequest) XXX_Size() int { - return xxx_messageInfo_FindByWorkflowExecRequest.Size(m) -} -func (m *FindByWorkflowExecRequest) XXX_DiscardUnknown() { - xxx_messageInfo_FindByWorkflowExecRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_FindByWorkflowExecRequest proto.InternalMessageInfo - -func (m *FindByWorkflowExecRequest) GetExecId() *core.WorkflowExecutionIdentifier { - if m != nil { - return m.ExecId - } - return nil -} - -func (m *FindByWorkflowExecRequest) GetDirection() FindByWorkflowExecRequest_Direction { - if m != nil { - return m.Direction - } - return FindByWorkflowExecRequest_INPUTS -} - -// Aliases identify a particular version of an artifact. They are different than tags in that they -// have to be unique for a given artifact project/domain/name. That is, for a given project/domain/name/kind, -// at most one version can have any given value at any point. -type AddTagRequest struct { - ArtifactId *core.ArtifactID `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"` - Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` - // If true, and another version already has the specified kind/value, set this version instead - Overwrite bool `protobuf:"varint,3,opt,name=overwrite,proto3" json:"overwrite,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *AddTagRequest) Reset() { *m = AddTagRequest{} } -func (m *AddTagRequest) String() string { return proto.CompactTextString(m) } -func (*AddTagRequest) ProtoMessage() {} -func (*AddTagRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{11} -} - -func (m *AddTagRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AddTagRequest.Unmarshal(m, b) -} -func (m *AddTagRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AddTagRequest.Marshal(b, m, deterministic) -} -func (m *AddTagRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_AddTagRequest.Merge(m, src) -} -func (m *AddTagRequest) XXX_Size() int { - return xxx_messageInfo_AddTagRequest.Size(m) -} -func (m *AddTagRequest) XXX_DiscardUnknown() { - xxx_messageInfo_AddTagRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_AddTagRequest proto.InternalMessageInfo - -func (m *AddTagRequest) GetArtifactId() *core.ArtifactID { - if m != nil { - return m.ArtifactId - } - return nil -} - -func (m *AddTagRequest) GetValue() string { - if m != nil { - return m.Value - } - return "" -} - -func (m *AddTagRequest) GetOverwrite() bool { - if m != nil { - return m.Overwrite - } - return false -} - -type AddTagResponse struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *AddTagResponse) Reset() { *m = AddTagResponse{} } -func (m *AddTagResponse) String() string { return proto.CompactTextString(m) } -func (*AddTagResponse) ProtoMessage() {} -func (*AddTagResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{12} -} - -func (m *AddTagResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AddTagResponse.Unmarshal(m, b) -} -func (m *AddTagResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AddTagResponse.Marshal(b, m, deterministic) -} -func (m *AddTagResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_AddTagResponse.Merge(m, src) -} -func (m *AddTagResponse) XXX_Size() int { - return xxx_messageInfo_AddTagResponse.Size(m) -} -func (m *AddTagResponse) XXX_DiscardUnknown() { - xxx_messageInfo_AddTagResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_AddTagResponse proto.InternalMessageInfo - -type CreateTriggerRequest struct { - TriggerLaunchPlan *admin.LaunchPlan `protobuf:"bytes,1,opt,name=trigger_launch_plan,json=triggerLaunchPlan,proto3" json:"trigger_launch_plan,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateTriggerRequest) Reset() { *m = CreateTriggerRequest{} } -func (m *CreateTriggerRequest) String() string { return proto.CompactTextString(m) } -func (*CreateTriggerRequest) ProtoMessage() {} -func (*CreateTriggerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{13} -} - -func (m *CreateTriggerRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateTriggerRequest.Unmarshal(m, b) -} -func (m *CreateTriggerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateTriggerRequest.Marshal(b, m, deterministic) -} -func (m *CreateTriggerRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateTriggerRequest.Merge(m, src) -} -func (m *CreateTriggerRequest) XXX_Size() int { - return xxx_messageInfo_CreateTriggerRequest.Size(m) -} -func (m *CreateTriggerRequest) XXX_DiscardUnknown() { - xxx_messageInfo_CreateTriggerRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateTriggerRequest proto.InternalMessageInfo - -func (m *CreateTriggerRequest) GetTriggerLaunchPlan() *admin.LaunchPlan { - if m != nil { - return m.TriggerLaunchPlan - } - return nil -} - -type CreateTriggerResponse struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateTriggerResponse) Reset() { *m = CreateTriggerResponse{} } -func (m *CreateTriggerResponse) String() string { return proto.CompactTextString(m) } -func (*CreateTriggerResponse) ProtoMessage() {} -func (*CreateTriggerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{14} -} - -func (m *CreateTriggerResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateTriggerResponse.Unmarshal(m, b) -} -func (m *CreateTriggerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateTriggerResponse.Marshal(b, m, deterministic) -} -func (m *CreateTriggerResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateTriggerResponse.Merge(m, src) -} -func (m *CreateTriggerResponse) XXX_Size() int { - return xxx_messageInfo_CreateTriggerResponse.Size(m) -} -func (m *CreateTriggerResponse) XXX_DiscardUnknown() { - xxx_messageInfo_CreateTriggerResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateTriggerResponse proto.InternalMessageInfo - -type DeactivateTriggerRequest struct { - TriggerId *core.Identifier `protobuf:"bytes,1,opt,name=trigger_id,json=triggerId,proto3" json:"trigger_id,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *DeactivateTriggerRequest) Reset() { *m = DeactivateTriggerRequest{} } -func (m *DeactivateTriggerRequest) String() string { return proto.CompactTextString(m) } -func (*DeactivateTriggerRequest) ProtoMessage() {} -func (*DeactivateTriggerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{15} -} - -func (m *DeactivateTriggerRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DeactivateTriggerRequest.Unmarshal(m, b) -} -func (m *DeactivateTriggerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DeactivateTriggerRequest.Marshal(b, m, deterministic) -} -func (m *DeactivateTriggerRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_DeactivateTriggerRequest.Merge(m, src) -} -func (m *DeactivateTriggerRequest) XXX_Size() int { - return xxx_messageInfo_DeactivateTriggerRequest.Size(m) -} -func (m *DeactivateTriggerRequest) XXX_DiscardUnknown() { - xxx_messageInfo_DeactivateTriggerRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_DeactivateTriggerRequest proto.InternalMessageInfo - -func (m *DeactivateTriggerRequest) GetTriggerId() *core.Identifier { - if m != nil { - return m.TriggerId - } - return nil -} - -type DeactivateTriggerResponse struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *DeactivateTriggerResponse) Reset() { *m = DeactivateTriggerResponse{} } -func (m *DeactivateTriggerResponse) String() string { return proto.CompactTextString(m) } -func (*DeactivateTriggerResponse) ProtoMessage() {} -func (*DeactivateTriggerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{16} -} - -func (m *DeactivateTriggerResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DeactivateTriggerResponse.Unmarshal(m, b) -} -func (m *DeactivateTriggerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DeactivateTriggerResponse.Marshal(b, m, deterministic) -} -func (m *DeactivateTriggerResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_DeactivateTriggerResponse.Merge(m, src) -} -func (m *DeactivateTriggerResponse) XXX_Size() int { - return xxx_messageInfo_DeactivateTriggerResponse.Size(m) -} -func (m *DeactivateTriggerResponse) XXX_DiscardUnknown() { - xxx_messageInfo_DeactivateTriggerResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_DeactivateTriggerResponse proto.InternalMessageInfo - -type ArtifactProducer struct { - // These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in - // Admin's domain. - EntityId *core.Identifier `protobuf:"bytes,1,opt,name=entity_id,json=entityId,proto3" json:"entity_id,omitempty"` - Outputs *core.VariableMap `protobuf:"bytes,2,opt,name=outputs,proto3" json:"outputs,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ArtifactProducer) Reset() { *m = ArtifactProducer{} } -func (m *ArtifactProducer) String() string { return proto.CompactTextString(m) } -func (*ArtifactProducer) ProtoMessage() {} -func (*ArtifactProducer) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{17} -} - -func (m *ArtifactProducer) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ArtifactProducer.Unmarshal(m, b) -} -func (m *ArtifactProducer) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ArtifactProducer.Marshal(b, m, deterministic) -} -func (m *ArtifactProducer) XXX_Merge(src proto.Message) { - xxx_messageInfo_ArtifactProducer.Merge(m, src) -} -func (m *ArtifactProducer) XXX_Size() int { - return xxx_messageInfo_ArtifactProducer.Size(m) -} -func (m *ArtifactProducer) XXX_DiscardUnknown() { - xxx_messageInfo_ArtifactProducer.DiscardUnknown(m) -} - -var xxx_messageInfo_ArtifactProducer proto.InternalMessageInfo - -func (m *ArtifactProducer) GetEntityId() *core.Identifier { - if m != nil { - return m.EntityId - } - return nil -} - -func (m *ArtifactProducer) GetOutputs() *core.VariableMap { - if m != nil { - return m.Outputs - } - return nil -} - -type RegisterProducerRequest struct { - Producers []*ArtifactProducer `protobuf:"bytes,1,rep,name=producers,proto3" json:"producers,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RegisterProducerRequest) Reset() { *m = RegisterProducerRequest{} } -func (m *RegisterProducerRequest) String() string { return proto.CompactTextString(m) } -func (*RegisterProducerRequest) ProtoMessage() {} -func (*RegisterProducerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{18} -} - -func (m *RegisterProducerRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RegisterProducerRequest.Unmarshal(m, b) -} -func (m *RegisterProducerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RegisterProducerRequest.Marshal(b, m, deterministic) -} -func (m *RegisterProducerRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_RegisterProducerRequest.Merge(m, src) -} -func (m *RegisterProducerRequest) XXX_Size() int { - return xxx_messageInfo_RegisterProducerRequest.Size(m) -} -func (m *RegisterProducerRequest) XXX_DiscardUnknown() { - xxx_messageInfo_RegisterProducerRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_RegisterProducerRequest proto.InternalMessageInfo - -func (m *RegisterProducerRequest) GetProducers() []*ArtifactProducer { - if m != nil { - return m.Producers - } - return nil -} - -type ArtifactConsumer struct { - // These should all be launch plan IDs - EntityId *core.Identifier `protobuf:"bytes,1,opt,name=entity_id,json=entityId,proto3" json:"entity_id,omitempty"` - Inputs *core.ParameterMap `protobuf:"bytes,2,opt,name=inputs,proto3" json:"inputs,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ArtifactConsumer) Reset() { *m = ArtifactConsumer{} } -func (m *ArtifactConsumer) String() string { return proto.CompactTextString(m) } -func (*ArtifactConsumer) ProtoMessage() {} -func (*ArtifactConsumer) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{19} -} - -func (m *ArtifactConsumer) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ArtifactConsumer.Unmarshal(m, b) -} -func (m *ArtifactConsumer) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ArtifactConsumer.Marshal(b, m, deterministic) -} -func (m *ArtifactConsumer) XXX_Merge(src proto.Message) { - xxx_messageInfo_ArtifactConsumer.Merge(m, src) -} -func (m *ArtifactConsumer) XXX_Size() int { - return xxx_messageInfo_ArtifactConsumer.Size(m) -} -func (m *ArtifactConsumer) XXX_DiscardUnknown() { - xxx_messageInfo_ArtifactConsumer.DiscardUnknown(m) -} - -var xxx_messageInfo_ArtifactConsumer proto.InternalMessageInfo - -func (m *ArtifactConsumer) GetEntityId() *core.Identifier { - if m != nil { - return m.EntityId - } - return nil -} - -func (m *ArtifactConsumer) GetInputs() *core.ParameterMap { - if m != nil { - return m.Inputs - } - return nil -} - -type RegisterConsumerRequest struct { - Consumers []*ArtifactConsumer `protobuf:"bytes,1,rep,name=consumers,proto3" json:"consumers,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RegisterConsumerRequest) Reset() { *m = RegisterConsumerRequest{} } -func (m *RegisterConsumerRequest) String() string { return proto.CompactTextString(m) } -func (*RegisterConsumerRequest) ProtoMessage() {} -func (*RegisterConsumerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{20} -} - -func (m *RegisterConsumerRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RegisterConsumerRequest.Unmarshal(m, b) -} -func (m *RegisterConsumerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RegisterConsumerRequest.Marshal(b, m, deterministic) -} -func (m *RegisterConsumerRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_RegisterConsumerRequest.Merge(m, src) -} -func (m *RegisterConsumerRequest) XXX_Size() int { - return xxx_messageInfo_RegisterConsumerRequest.Size(m) -} -func (m *RegisterConsumerRequest) XXX_DiscardUnknown() { - xxx_messageInfo_RegisterConsumerRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_RegisterConsumerRequest proto.InternalMessageInfo - -func (m *RegisterConsumerRequest) GetConsumers() []*ArtifactConsumer { - if m != nil { - return m.Consumers - } - return nil -} - -type RegisterResponse struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RegisterResponse) Reset() { *m = RegisterResponse{} } -func (m *RegisterResponse) String() string { return proto.CompactTextString(m) } -func (*RegisterResponse) ProtoMessage() {} -func (*RegisterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{21} -} - -func (m *RegisterResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RegisterResponse.Unmarshal(m, b) -} -func (m *RegisterResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RegisterResponse.Marshal(b, m, deterministic) -} -func (m *RegisterResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_RegisterResponse.Merge(m, src) -} -func (m *RegisterResponse) XXX_Size() int { - return xxx_messageInfo_RegisterResponse.Size(m) -} -func (m *RegisterResponse) XXX_DiscardUnknown() { - xxx_messageInfo_RegisterResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_RegisterResponse proto.InternalMessageInfo - -type ExecutionInputsRequest struct { - ExecutionId *core.WorkflowExecutionIdentifier `protobuf:"bytes,1,opt,name=execution_id,json=executionId,proto3" json:"execution_id,omitempty"` - // can make this a map in the future, currently no need. - Inputs []*core.ArtifactID `protobuf:"bytes,2,rep,name=inputs,proto3" json:"inputs,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ExecutionInputsRequest) Reset() { *m = ExecutionInputsRequest{} } -func (m *ExecutionInputsRequest) String() string { return proto.CompactTextString(m) } -func (*ExecutionInputsRequest) ProtoMessage() {} -func (*ExecutionInputsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{22} -} - -func (m *ExecutionInputsRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ExecutionInputsRequest.Unmarshal(m, b) -} -func (m *ExecutionInputsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ExecutionInputsRequest.Marshal(b, m, deterministic) -} -func (m *ExecutionInputsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecutionInputsRequest.Merge(m, src) -} -func (m *ExecutionInputsRequest) XXX_Size() int { - return xxx_messageInfo_ExecutionInputsRequest.Size(m) -} -func (m *ExecutionInputsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_ExecutionInputsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_ExecutionInputsRequest proto.InternalMessageInfo - -func (m *ExecutionInputsRequest) GetExecutionId() *core.WorkflowExecutionIdentifier { - if m != nil { - return m.ExecutionId - } - return nil -} - -func (m *ExecutionInputsRequest) GetInputs() []*core.ArtifactID { - if m != nil { - return m.Inputs - } - return nil -} - -type ExecutionInputsResponse struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ExecutionInputsResponse) Reset() { *m = ExecutionInputsResponse{} } -func (m *ExecutionInputsResponse) String() string { return proto.CompactTextString(m) } -func (*ExecutionInputsResponse) ProtoMessage() {} -func (*ExecutionInputsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{23} -} - -func (m *ExecutionInputsResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ExecutionInputsResponse.Unmarshal(m, b) -} -func (m *ExecutionInputsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ExecutionInputsResponse.Marshal(b, m, deterministic) -} -func (m *ExecutionInputsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecutionInputsResponse.Merge(m, src) -} -func (m *ExecutionInputsResponse) XXX_Size() int { - return xxx_messageInfo_ExecutionInputsResponse.Size(m) -} -func (m *ExecutionInputsResponse) XXX_DiscardUnknown() { - xxx_messageInfo_ExecutionInputsResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_ExecutionInputsResponse proto.InternalMessageInfo - -type ListUsageRequest struct { - ArtifactId *core.ArtifactID `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ListUsageRequest) Reset() { *m = ListUsageRequest{} } -func (m *ListUsageRequest) String() string { return proto.CompactTextString(m) } -func (*ListUsageRequest) ProtoMessage() {} -func (*ListUsageRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{24} -} - -func (m *ListUsageRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListUsageRequest.Unmarshal(m, b) -} -func (m *ListUsageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListUsageRequest.Marshal(b, m, deterministic) -} -func (m *ListUsageRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListUsageRequest.Merge(m, src) -} -func (m *ListUsageRequest) XXX_Size() int { - return xxx_messageInfo_ListUsageRequest.Size(m) -} -func (m *ListUsageRequest) XXX_DiscardUnknown() { - xxx_messageInfo_ListUsageRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_ListUsageRequest proto.InternalMessageInfo - -func (m *ListUsageRequest) GetArtifactId() *core.ArtifactID { - if m != nil { - return m.ArtifactId - } - return nil -} - -type ListUsageResponse struct { - Executions []*core.WorkflowExecutionIdentifier `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ListUsageResponse) Reset() { *m = ListUsageResponse{} } -func (m *ListUsageResponse) String() string { return proto.CompactTextString(m) } -func (*ListUsageResponse) ProtoMessage() {} -func (*ListUsageResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_804518da5936dedb, []int{25} -} - -func (m *ListUsageResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListUsageResponse.Unmarshal(m, b) -} -func (m *ListUsageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListUsageResponse.Marshal(b, m, deterministic) -} -func (m *ListUsageResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListUsageResponse.Merge(m, src) -} -func (m *ListUsageResponse) XXX_Size() int { - return xxx_messageInfo_ListUsageResponse.Size(m) -} -func (m *ListUsageResponse) XXX_DiscardUnknown() { - xxx_messageInfo_ListUsageResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_ListUsageResponse proto.InternalMessageInfo - -func (m *ListUsageResponse) GetExecutions() []*core.WorkflowExecutionIdentifier { - if m != nil { - return m.Executions - } - return nil -} - -func init() { - proto.RegisterEnum("flyteidl.artifact.FindByWorkflowExecRequest_Direction", FindByWorkflowExecRequest_Direction_name, FindByWorkflowExecRequest_Direction_value) - proto.RegisterType((*Artifact)(nil), "flyteidl.artifact.Artifact") - proto.RegisterType((*CreateArtifactRequest)(nil), "flyteidl.artifact.CreateArtifactRequest") - proto.RegisterMapType((map[string]string)(nil), "flyteidl.artifact.CreateArtifactRequest.PartitionsEntry") - proto.RegisterType((*ArtifactSource)(nil), "flyteidl.artifact.ArtifactSource") - proto.RegisterType((*ArtifactSpec)(nil), "flyteidl.artifact.ArtifactSpec") - proto.RegisterType((*CreateArtifactResponse)(nil), "flyteidl.artifact.CreateArtifactResponse") - proto.RegisterType((*GetArtifactRequest)(nil), "flyteidl.artifact.GetArtifactRequest") - proto.RegisterType((*GetArtifactResponse)(nil), "flyteidl.artifact.GetArtifactResponse") - proto.RegisterType((*SearchOptions)(nil), "flyteidl.artifact.SearchOptions") - proto.RegisterType((*SearchArtifactsRequest)(nil), "flyteidl.artifact.SearchArtifactsRequest") - proto.RegisterType((*SearchArtifactsResponse)(nil), "flyteidl.artifact.SearchArtifactsResponse") - proto.RegisterType((*FindByWorkflowExecRequest)(nil), "flyteidl.artifact.FindByWorkflowExecRequest") - proto.RegisterType((*AddTagRequest)(nil), "flyteidl.artifact.AddTagRequest") - proto.RegisterType((*AddTagResponse)(nil), "flyteidl.artifact.AddTagResponse") - proto.RegisterType((*CreateTriggerRequest)(nil), "flyteidl.artifact.CreateTriggerRequest") - proto.RegisterType((*CreateTriggerResponse)(nil), "flyteidl.artifact.CreateTriggerResponse") - proto.RegisterType((*DeactivateTriggerRequest)(nil), "flyteidl.artifact.DeactivateTriggerRequest") - proto.RegisterType((*DeactivateTriggerResponse)(nil), "flyteidl.artifact.DeactivateTriggerResponse") - proto.RegisterType((*ArtifactProducer)(nil), "flyteidl.artifact.ArtifactProducer") - proto.RegisterType((*RegisterProducerRequest)(nil), "flyteidl.artifact.RegisterProducerRequest") - proto.RegisterType((*ArtifactConsumer)(nil), "flyteidl.artifact.ArtifactConsumer") - proto.RegisterType((*RegisterConsumerRequest)(nil), "flyteidl.artifact.RegisterConsumerRequest") - proto.RegisterType((*RegisterResponse)(nil), "flyteidl.artifact.RegisterResponse") - proto.RegisterType((*ExecutionInputsRequest)(nil), "flyteidl.artifact.ExecutionInputsRequest") - proto.RegisterType((*ExecutionInputsResponse)(nil), "flyteidl.artifact.ExecutionInputsResponse") - proto.RegisterType((*ListUsageRequest)(nil), "flyteidl.artifact.ListUsageRequest") - proto.RegisterType((*ListUsageResponse)(nil), "flyteidl.artifact.ListUsageResponse") -} - -func init() { proto.RegisterFile("flyteidl/artifact/artifacts.proto", fileDescriptor_804518da5936dedb) } - -var fileDescriptor_804518da5936dedb = []byte{ - // 1721 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xdf, 0x6f, 0xdb, 0xc6, - 0x1d, 0x0f, 0x2d, 0x5b, 0xb2, 0xbe, 0xb2, 0x53, 0xf9, 0xea, 0xd9, 0xb2, 0x9c, 0x2e, 0x0a, 0x9d, - 0x75, 0x4e, 0x9a, 0x89, 0xa8, 0x33, 0x74, 0xb1, 0x81, 0x0e, 0x73, 0xe2, 0x76, 0x50, 0xe3, 0x24, - 0x2e, 0xad, 0x74, 0x3f, 0x30, 0x40, 0x3d, 0x91, 0x27, 0x99, 0x35, 0x45, 0xb2, 0xc7, 0x93, 0x3d, - 0x22, 0x30, 0x5a, 0x0c, 0x7b, 0xdd, 0xcb, 0x86, 0xbd, 0x0c, 0x03, 0xf6, 0xe7, 0x0c, 0xd8, 0xeb, - 0x30, 0x60, 0x0f, 0x7b, 0xdc, 0xbf, 0xb0, 0xa7, 0xbe, 0x0c, 0x77, 0xbc, 0x23, 0x29, 0x89, 0x52, - 0xe5, 0xa4, 0x6f, 0xe4, 0xf7, 0x3e, 0xdf, 0x9f, 0xf7, 0xbd, 0xcf, 0x7d, 0x49, 0xb8, 0xd3, 0x73, - 0x23, 0x46, 0x1c, 0xdb, 0x35, 0x30, 0x65, 0x4e, 0x0f, 0x5b, 0x2c, 0x79, 0x08, 0x9b, 0x01, 0xf5, - 0x99, 0x8f, 0xd6, 0x14, 0xa4, 0xa9, 0x56, 0xea, 0x5b, 0x7d, 0xdf, 0xef, 0xbb, 0xc4, 0x10, 0x80, - 0xee, 0xb0, 0x67, 0x60, 0x2f, 0x8a, 0xd1, 0xf5, 0x5b, 0x72, 0x09, 0x07, 0x8e, 0x81, 0x3d, 0xcf, - 0x67, 0x98, 0x39, 0xbe, 0x27, 0x6d, 0xd5, 0x6f, 0x8f, 0x2b, 0x32, 0x67, 0x40, 0x42, 0x86, 0x07, - 0x81, 0x04, 0x34, 0xd2, 0x78, 0xec, 0x81, 0xe3, 0x19, 0x2e, 0x1e, 0x7a, 0xd6, 0x59, 0x27, 0x70, - 0xb1, 0xa7, 0x1c, 0x24, 0x08, 0xcb, 0xa7, 0xc4, 0x70, 0x1d, 0x46, 0x28, 0x76, 0x95, 0x83, 0xad, - 0xd1, 0x55, 0x16, 0x05, 0x44, 0x2d, 0x7d, 0x7f, 0x74, 0xc9, 0xb1, 0x89, 0xc7, 0x9c, 0x9e, 0x43, - 0xa8, 0x8a, 0x6d, 0x74, 0x5d, 0x25, 0xdb, 0x71, 0x6c, 0x09, 0x78, 0x67, 0xcc, 0x80, 0xc7, 0x08, - 0xed, 0x61, 0x8b, 0x4c, 0x84, 0x4e, 0x2e, 0x88, 0xc7, 0x0c, 0xcb, 0xf5, 0x87, 0xb6, 0x78, 0x94, - 0x11, 0xe8, 0xff, 0xd0, 0x60, 0xf9, 0x50, 0x9a, 0x45, 0x07, 0x50, 0xc9, 0xb8, 0xa8, 0x69, 0x0d, - 0x6d, 0xb7, 0xb2, 0xb7, 0xd5, 0x4c, 0x8a, 0xcd, 0x7d, 0x34, 0x15, 0xba, 0x75, 0x64, 0x82, 0x42, - 0xb7, 0x6c, 0xf4, 0x10, 0x16, 0xc3, 0x80, 0x58, 0xb5, 0x05, 0xa1, 0x74, 0xbb, 0x39, 0xb1, 0x43, - 0x89, 0xe2, 0x69, 0x40, 0x2c, 0x53, 0x80, 0x11, 0x82, 0x45, 0x86, 0xfb, 0x61, 0xad, 0xd0, 0x28, - 0xec, 0x96, 0x4d, 0xf1, 0x8c, 0xf6, 0xa1, 0x18, 0xfa, 0x43, 0x6a, 0x91, 0xda, 0xa2, 0x30, 0x75, - 0x67, 0x96, 0x29, 0x01, 0x34, 0xa5, 0x82, 0xfe, 0xf7, 0x02, 0x7c, 0xef, 0x09, 0x25, 0x98, 0x11, - 0x05, 0x30, 0xc9, 0x97, 0x43, 0x12, 0x32, 0xf4, 0x21, 0xac, 0x24, 0x99, 0x9d, 0x93, 0x48, 0xa6, - 0x56, 0x9f, 0x92, 0xda, 0x53, 0x12, 0x99, 0x49, 0x25, 0x9e, 0x92, 0x08, 0xd5, 0xa0, 0x74, 0x41, - 0x68, 0xe8, 0xf8, 0x5e, 0xad, 0xd0, 0xd0, 0x76, 0xcb, 0xa6, 0x7a, 0x7d, 0xbd, 0xb4, 0x7f, 0x09, - 0x10, 0xf0, 0x75, 0xd1, 0x86, 0xb5, 0xc5, 0x46, 0x61, 0xb7, 0xb2, 0xf7, 0x28, 0x47, 0x35, 0x37, - 0x97, 0xe6, 0x49, 0xa2, 0xfa, 0x91, 0xc7, 0x68, 0x64, 0x66, 0x6c, 0xa1, 0x63, 0x58, 0xe7, 0xed, - 0xdb, 0x49, 0x44, 0x9d, 0x0b, 0xec, 0x0e, 0x49, 0x6d, 0x49, 0xe6, 0x1b, 0xf7, 0x7a, 0x53, 0xf5, - 0x7a, 0xb3, 0xad, 0x7a, 0xdd, 0x44, 0x5c, 0x2f, 0x31, 0xfd, 0x19, 0xd7, 0xca, 0x6c, 0x45, 0xf1, - 0x9a, 0x5b, 0x51, 0xff, 0x10, 0xde, 0x1a, 0x8b, 0x13, 0x55, 0xa1, 0xa0, 0x4a, 0x5f, 0x36, 0xf9, - 0x23, 0x5a, 0x87, 0xa5, 0x38, 0xbc, 0x05, 0x21, 0x8b, 0x5f, 0x0e, 0x16, 0x1e, 0x69, 0xfa, 0x37, - 0x1a, 0xdc, 0x1c, 0xb5, 0x8c, 0x7e, 0x05, 0xe8, 0xd2, 0xa7, 0xe7, 0x3d, 0xd7, 0xbf, 0xec, 0x90, - 0xdf, 0x12, 0x6b, 0xc8, 0x4d, 0xcb, 0x8d, 0xbc, 0x3f, 0xb6, 0x91, 0xbf, 0x90, 0xc0, 0x8f, 0x14, - 0xae, 0x95, 0x9c, 0x2c, 0x73, 0xed, 0x72, 0x7c, 0x11, 0x6d, 0x42, 0xc9, 0xf3, 0x6d, 0xc2, 0x7b, - 0x3e, 0x8e, 0xa4, 0xc8, 0x5f, 0x5b, 0x36, 0xda, 0x83, 0x12, 0xc3, 0xe1, 0x39, 0x5f, 0x28, 0xe4, - 0x1e, 0x86, 0x8c, 0xdd, 0x22, 0x47, 0xb6, 0x6c, 0xb4, 0x03, 0xab, 0x94, 0x30, 0x1a, 0x75, 0x30, - 0x63, 0x64, 0x10, 0x30, 0xd1, 0xc6, 0xab, 0xe6, 0x8a, 0x10, 0x1e, 0xc6, 0x32, 0x74, 0x0b, 0xca, - 0x01, 0x75, 0x3c, 0xcb, 0x09, 0xb0, 0x2b, 0x36, 0xa7, 0x6c, 0xa6, 0x02, 0xfd, 0x5f, 0x0b, 0xb0, - 0x92, 0x6d, 0x1b, 0xf4, 0x40, 0x15, 0x2a, 0x4e, 0x77, 0x63, 0x2c, 0x8a, 0xe3, 0x98, 0x70, 0x64, - 0x01, 0x51, 0x13, 0x16, 0x39, 0xc9, 0xc8, 0x9e, 0xac, 0xe7, 0x83, 0xdb, 0x51, 0x40, 0x4c, 0x81, - 0x43, 0xef, 0xc1, 0x5a, 0x78, 0xe6, 0x53, 0xd6, 0xb1, 0x49, 0x68, 0x51, 0x27, 0x60, 0x69, 0x9f, - 0x57, 0xc5, 0xc2, 0x51, 0x2a, 0x47, 0xfb, 0xb0, 0x3a, 0x0c, 0x09, 0xed, 0x0c, 0x08, 0xc3, 0x36, - 0x66, 0x58, 0x9e, 0xd2, 0xf5, 0x89, 0xd6, 0x3a, 0xf4, 0x22, 0x73, 0x85, 0x43, 0x9f, 0x49, 0x24, - 0xaf, 0x8c, 0xd2, 0xea, 0x88, 0x00, 0xe3, 0xc4, 0x57, 0x94, 0x90, 0x87, 0x84, 0xf6, 0x01, 0x2c, - 0xd1, 0xf6, 0x76, 0x07, 0x33, 0xd9, 0x77, 0xb3, 0xfa, 0xb6, 0x2c, 0xd1, 0x87, 0x0c, 0xdd, 0x86, - 0x4a, 0xcf, 0x71, 0x49, 0xa7, 0xe7, 0xd3, 0x01, 0x66, 0xb5, 0x92, 0xb0, 0x0e, 0x5c, 0xf4, 0xb1, - 0x90, 0xe8, 0x9f, 0xc2, 0xc6, 0xf8, 0x91, 0x0a, 0x03, 0xdf, 0x0b, 0x09, 0xfa, 0x09, 0x2c, 0xab, - 0x8e, 0x96, 0x35, 0xde, 0x9e, 0xd1, 0xeb, 0x66, 0x02, 0xd6, 0xbb, 0x80, 0x7e, 0x4e, 0xd8, 0x38, - 0xdd, 0xec, 0xc1, 0xd2, 0x97, 0x43, 0x42, 0x15, 0xcf, 0xdc, 0x9a, 0xc2, 0x33, 0x9f, 0x72, 0x8c, - 0x19, 0x43, 0x39, 0xc7, 0xd8, 0x84, 0x61, 0xc7, 0x0d, 0xc5, 0xc6, 0x2d, 0x9b, 0xea, 0x55, 0x7f, - 0x0e, 0x6f, 0x8f, 0xf8, 0x78, 0xd3, 0x98, 0x3f, 0x87, 0xd5, 0x53, 0x82, 0xa9, 0x75, 0xf6, 0x22, - 0x88, 0x59, 0x83, 0x37, 0x00, 0xa3, 0x8e, 0xc5, 0x3a, 0x19, 0x5a, 0xd2, 0x44, 0x10, 0xd5, 0x78, - 0x21, 0x3d, 0xcb, 0x48, 0x87, 0x55, 0x17, 0x33, 0x12, 0xb2, 0x4e, 0x37, 0x12, 0x5c, 0x1a, 0x47, - 0x5b, 0x89, 0x85, 0x8f, 0xa3, 0xa7, 0x24, 0xd2, 0xbf, 0x2e, 0xc0, 0x46, 0xec, 0x42, 0xb9, 0x0f, - 0xbf, 0x23, 0x26, 0xde, 0x1f, 0xa1, 0xce, 0x85, 0xdc, 0x43, 0x99, 0x06, 0x3b, 0x17, 0x37, 0x16, - 0x5e, 0x8b, 0x1b, 0x47, 0x4e, 0xf0, 0xe2, 0xd8, 0x09, 0xce, 0x5e, 0x18, 0x4b, 0xa3, 0x17, 0xc6, - 0x01, 0x94, 0xfc, 0xb8, 0xec, 0xb2, 0xb9, 0x1b, 0x39, 0x9b, 0x36, 0xb2, 0x3d, 0xa6, 0x52, 0xe0, - 0x7c, 0xc9, 0xfc, 0x73, 0xe2, 0xc9, 0xd6, 0x8e, 0x5f, 0xb8, 0xd4, 0x75, 0x06, 0x0e, 0xab, 0x2d, - 0x37, 0xb4, 0xdd, 0x25, 0x33, 0x7e, 0xd1, 0xbf, 0x80, 0xcd, 0x89, 0x1d, 0x90, 0x8d, 0xb3, 0x0f, - 0xe5, 0x64, 0xa0, 0xaa, 0x69, 0xe2, 0xf6, 0x99, 0xd9, 0x39, 0x29, 0x3a, 0x8d, 0x60, 0x21, 0x13, - 0x81, 0xfe, 0x1f, 0x0d, 0xb6, 0x3e, 0x76, 0x3c, 0xfb, 0x71, 0x94, 0x25, 0x5e, 0xb5, 0xe3, 0x4f, - 0xa0, 0xc4, 0xf9, 0x3a, 0x9d, 0x28, 0xae, 0xc3, 0xd6, 0x45, 0xae, 0xda, 0xb2, 0x51, 0x1b, 0xca, - 0xb6, 0x43, 0x89, 0x25, 0xb8, 0x89, 0x3b, 0xbf, 0xb9, 0xf7, 0x41, 0x4e, 0xcc, 0x53, 0xa3, 0x68, - 0x1e, 0x29, 0x6d, 0x33, 0x35, 0xa4, 0xdf, 0x85, 0x72, 0x22, 0x47, 0x00, 0xc5, 0xd6, 0xf3, 0x93, - 0x97, 0xed, 0xd3, 0xea, 0x0d, 0x54, 0x81, 0xd2, 0x8b, 0x97, 0x6d, 0xf1, 0xa2, 0xe9, 0x5f, 0xc1, - 0xea, 0xa1, 0x6d, 0xb7, 0x71, 0x5f, 0x65, 0xf4, 0x26, 0x73, 0x52, 0xee, 0x9d, 0xc7, 0xbb, 0xc9, - 0xbf, 0x20, 0xf4, 0x92, 0x3a, 0x2c, 0x6e, 0xc8, 0x65, 0x33, 0x15, 0xe8, 0x55, 0xb8, 0xa9, 0x02, - 0x88, 0xb7, 0x50, 0xef, 0xc2, 0x7a, 0xcc, 0x64, 0x6d, 0xea, 0xf4, 0xfb, 0x84, 0xaa, 0xc8, 0x3e, - 0x81, 0xb7, 0x59, 0x2c, 0xe9, 0x64, 0xc6, 0xd4, 0xc9, 0x43, 0x26, 0x26, 0xd9, 0xe6, 0xb1, 0x80, - 0x9c, 0xb8, 0xd8, 0x33, 0xd7, 0xa4, 0x5a, 0x2a, 0xd2, 0x37, 0xd5, 0x30, 0x95, 0xf8, 0x90, 0xce, - 0xdb, 0x50, 0x3b, 0x22, 0xd8, 0x62, 0xce, 0xc5, 0x64, 0x00, 0x8f, 0x00, 0x54, 0x00, 0x53, 0x2b, - 0x93, 0xd9, 0xde, 0xb2, 0x04, 0xb7, 0x6c, 0x7d, 0x1b, 0xb6, 0x72, 0xac, 0x4a, 0x97, 0x5f, 0x6b, - 0x50, 0x55, 0x05, 0x3d, 0xa1, 0xbe, 0x3d, 0xb4, 0x08, 0x45, 0x1f, 0x40, 0x99, 0x1b, 0x62, 0xd1, - 0x5c, 0xae, 0x96, 0x63, 0x6c, 0xcb, 0x46, 0x3f, 0x86, 0x92, 0x3f, 0x64, 0xc1, 0x90, 0x85, 0x53, - 0xae, 0xc8, 0xcf, 0x30, 0x75, 0x70, 0xd7, 0x25, 0xcf, 0x70, 0x60, 0x2a, 0xa8, 0xfe, 0x1b, 0xd8, - 0x34, 0x49, 0xdf, 0x09, 0x19, 0xa1, 0x2a, 0x02, 0x95, 0xf4, 0x21, 0xe7, 0x82, 0x58, 0xa4, 0x0e, - 0xd4, 0xce, 0x8c, 0x03, 0x95, 0xa8, 0xa7, 0x5a, 0xfa, 0x57, 0x69, 0x7e, 0x4f, 0x7c, 0x2f, 0x1c, - 0x0e, 0xde, 0x20, 0xbf, 0x87, 0x50, 0x74, 0xbc, 0x4c, 0x7a, 0xdb, 0x93, 0xfc, 0x88, 0x07, 0x84, - 0x11, 0xca, 0xf3, 0x93, 0xd0, 0x6c, 0x7a, 0x2a, 0x80, 0x4c, 0x7a, 0x96, 0x14, 0xcd, 0x93, 0x5e, - 0xa2, 0x9e, 0x6a, 0xe9, 0x08, 0xaa, 0xca, 0x7a, 0xb2, 0xa7, 0x7f, 0xd1, 0x60, 0x23, 0x3d, 0xf2, - 0x22, 0x0a, 0xe5, 0xf1, 0x19, 0xac, 0x24, 0x23, 0xde, 0xeb, 0xf1, 0x46, 0x85, 0xa4, 0x42, 0xf4, - 0x7e, 0xa6, 0x20, 0x85, 0xd9, 0x47, 0x55, 0x95, 0x63, 0x0b, 0x36, 0x27, 0x62, 0x93, 0x71, 0x3f, - 0x87, 0xea, 0xb1, 0x13, 0xb2, 0x97, 0x21, 0xee, 0x93, 0xef, 0x80, 0x11, 0xf4, 0x0e, 0xac, 0x65, - 0xec, 0x49, 0x8e, 0xfe, 0x04, 0x20, 0xc9, 0x40, 0x15, 0xfd, 0x3a, 0xf9, 0x67, 0xb4, 0xf7, 0xbe, - 0xa9, 0xa4, 0xcd, 0x15, 0xef, 0x02, 0x8d, 0x50, 0x1f, 0x6e, 0x8e, 0xce, 0x42, 0x68, 0x77, 0xde, - 0x2f, 0x90, 0xfa, 0xbd, 0x39, 0x90, 0xb2, 0x58, 0x37, 0xd0, 0xef, 0x35, 0xa8, 0x64, 0xc6, 0x17, - 0xf4, 0x83, 0x1c, 0xe5, 0xc9, 0x11, 0xaa, 0xfe, 0xee, 0xb7, 0xc1, 0xa4, 0x83, 0x77, 0x7f, 0xf7, - 0xcf, 0xff, 0xfe, 0x69, 0xa1, 0xa1, 0x6f, 0xa7, 0x3f, 0x09, 0xc4, 0x87, 0xfe, 0xc5, 0xfb, 0xa9, - 0xe0, 0x40, 0xbb, 0x8f, 0xfe, 0xa0, 0xc1, 0x5b, 0x63, 0x17, 0x22, 0xba, 0x37, 0xf5, 0xea, 0x1d, - 0x1f, 0x5b, 0xea, 0xf7, 0xe7, 0x81, 0xca, 0x90, 0x76, 0x44, 0x48, 0xef, 0xe8, 0xb5, 0xc9, 0x90, - 0x42, 0xa1, 0xc2, 0xe3, 0xb1, 0x61, 0x75, 0x84, 0x5d, 0xd1, 0x0f, 0xa7, 0x16, 0x75, 0x94, 0x62, - 0xeb, 0xbb, 0xdf, 0x0e, 0x4c, 0x8a, 0xff, 0x37, 0x0d, 0xd6, 0x26, 0x58, 0x15, 0xbd, 0x97, 0x63, - 0x61, 0x1a, 0xa3, 0xd7, 0x1f, 0xcc, 0x07, 0x96, 0x2e, 0x0d, 0x91, 0xfb, 0xbd, 0xbd, 0xbb, 0x93, - 0xb9, 0x4b, 0xaa, 0x37, 0xec, 0x44, 0x99, 0xd7, 0xe1, 0x05, 0x14, 0xe3, 0xbb, 0x0d, 0xe5, 0x0d, - 0x42, 0x23, 0xf7, 0x6e, 0xfd, 0xce, 0x0c, 0x44, 0x92, 0x32, 0x49, 0xa9, 0x26, 0xb9, 0x29, 0xf2, - 0x76, 0x6f, 0x0a, 0x99, 0xd7, 0x77, 0x66, 0x60, 0xf3, 0xdd, 0x24, 0x84, 0x3d, 0xcb, 0xcd, 0x18, - 0xa9, 0xce, 0xeb, 0x66, 0x00, 0xe8, 0x94, 0xb0, 0x31, 0x2a, 0xca, 0x6d, 0xdc, 0x7c, 0x2a, 0xcd, - 0x6d, 0xdc, 0x69, 0xcc, 0x76, 0x03, 0xfd, 0x5b, 0x03, 0x34, 0x39, 0x43, 0xa1, 0x07, 0xd7, 0x19, - 0xb5, 0xae, 0x75, 0x56, 0xce, 0x44, 0xbf, 0x74, 0xd1, 0xe7, 0xd3, 0xce, 0x8a, 0x91, 0x10, 0x99, - 0xf1, 0x4a, 0x8e, 0x91, 0x7c, 0x44, 0xff, 0x82, 0x58, 0xec, 0x2a, 0x95, 0xd8, 0xfe, 0x00, 0x3b, - 0x5e, 0x46, 0xe0, 0xe1, 0x01, 0xb9, 0x32, 0x5e, 0x25, 0xa3, 0xde, 0x15, 0xfa, 0x9f, 0x06, 0xe5, - 0x84, 0x67, 0x51, 0x5e, 0xf9, 0xc7, 0x59, 0xbd, 0x7e, 0x77, 0x36, 0x48, 0xa6, 0xf0, 0x57, 0x4d, - 0xe4, 0xf0, 0x67, 0x0d, 0xfd, 0x51, 0x9b, 0xcc, 0x62, 0xc8, 0xb1, 0xc6, 0xab, 0xec, 0xaf, 0xbb, - 0xec, 0xf7, 0x4f, 0x26, 0x97, 0xa9, 0x90, 0x24, 0xb9, 0xa9, 0x08, 0x99, 0x6d, 0x76, 0x5d, 0x7e, - 0x6f, 0x5c, 0x3d, 0xfe, 0xd9, 0xaf, 0x7f, 0xda, 0x77, 0xd8, 0xd9, 0xb0, 0xdb, 0xb4, 0xfc, 0x81, - 0x21, 0x12, 0xf2, 0x69, 0x3f, 0x7e, 0x30, 0x92, 0xff, 0x83, 0x7d, 0xe2, 0x19, 0x41, 0xf7, 0x47, - 0x7d, 0xdf, 0x98, 0xf8, 0xfb, 0xda, 0x2d, 0x8a, 0x4f, 0xa2, 0x87, 0xff, 0x0f, 0x00, 0x00, 0xff, - 0xff, 0x1c, 0xb2, 0xae, 0x8e, 0x99, 0x15, 0x00, 0x00, -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConn - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion4 - -// ArtifactRegistryClient is the client API for ArtifactRegistry service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type ArtifactRegistryClient interface { - CreateArtifact(ctx context.Context, in *CreateArtifactRequest, opts ...grpc.CallOption) (*CreateArtifactResponse, error) - GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (*GetArtifactResponse, error) - SearchArtifacts(ctx context.Context, in *SearchArtifactsRequest, opts ...grpc.CallOption) (*SearchArtifactsResponse, error) - CreateTrigger(ctx context.Context, in *CreateTriggerRequest, opts ...grpc.CallOption) (*CreateTriggerResponse, error) - DeactivateTrigger(ctx context.Context, in *DeactivateTriggerRequest, opts ...grpc.CallOption) (*DeactivateTriggerResponse, error) - AddTag(ctx context.Context, in *AddTagRequest, opts ...grpc.CallOption) (*AddTagResponse, error) - RegisterProducer(ctx context.Context, in *RegisterProducerRequest, opts ...grpc.CallOption) (*RegisterResponse, error) - RegisterConsumer(ctx context.Context, in *RegisterConsumerRequest, opts ...grpc.CallOption) (*RegisterResponse, error) - SetExecutionInputs(ctx context.Context, in *ExecutionInputsRequest, opts ...grpc.CallOption) (*ExecutionInputsResponse, error) - FindByWorkflowExec(ctx context.Context, in *FindByWorkflowExecRequest, opts ...grpc.CallOption) (*SearchArtifactsResponse, error) - ListUsage(ctx context.Context, in *ListUsageRequest, opts ...grpc.CallOption) (*ListUsageResponse, error) -} - -type artifactRegistryClient struct { - cc *grpc.ClientConn -} - -func NewArtifactRegistryClient(cc *grpc.ClientConn) ArtifactRegistryClient { - return &artifactRegistryClient{cc} -} - -func (c *artifactRegistryClient) CreateArtifact(ctx context.Context, in *CreateArtifactRequest, opts ...grpc.CallOption) (*CreateArtifactResponse, error) { - out := new(CreateArtifactResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/CreateArtifact", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (*GetArtifactResponse, error) { - out := new(GetArtifactResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/GetArtifact", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) SearchArtifacts(ctx context.Context, in *SearchArtifactsRequest, opts ...grpc.CallOption) (*SearchArtifactsResponse, error) { - out := new(SearchArtifactsResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/SearchArtifacts", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) CreateTrigger(ctx context.Context, in *CreateTriggerRequest, opts ...grpc.CallOption) (*CreateTriggerResponse, error) { - out := new(CreateTriggerResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/CreateTrigger", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) DeactivateTrigger(ctx context.Context, in *DeactivateTriggerRequest, opts ...grpc.CallOption) (*DeactivateTriggerResponse, error) { - out := new(DeactivateTriggerResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/DeactivateTrigger", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) AddTag(ctx context.Context, in *AddTagRequest, opts ...grpc.CallOption) (*AddTagResponse, error) { - out := new(AddTagResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/AddTag", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) RegisterProducer(ctx context.Context, in *RegisterProducerRequest, opts ...grpc.CallOption) (*RegisterResponse, error) { - out := new(RegisterResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/RegisterProducer", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) RegisterConsumer(ctx context.Context, in *RegisterConsumerRequest, opts ...grpc.CallOption) (*RegisterResponse, error) { - out := new(RegisterResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/RegisterConsumer", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) SetExecutionInputs(ctx context.Context, in *ExecutionInputsRequest, opts ...grpc.CallOption) (*ExecutionInputsResponse, error) { - out := new(ExecutionInputsResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/SetExecutionInputs", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) FindByWorkflowExec(ctx context.Context, in *FindByWorkflowExecRequest, opts ...grpc.CallOption) (*SearchArtifactsResponse, error) { - out := new(SearchArtifactsResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/FindByWorkflowExec", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *artifactRegistryClient) ListUsage(ctx context.Context, in *ListUsageRequest, opts ...grpc.CallOption) (*ListUsageResponse, error) { - out := new(ListUsageResponse) - err := c.cc.Invoke(ctx, "/flyteidl.artifact.ArtifactRegistry/ListUsage", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// ArtifactRegistryServer is the server API for ArtifactRegistry service. -type ArtifactRegistryServer interface { - CreateArtifact(context.Context, *CreateArtifactRequest) (*CreateArtifactResponse, error) - GetArtifact(context.Context, *GetArtifactRequest) (*GetArtifactResponse, error) - SearchArtifacts(context.Context, *SearchArtifactsRequest) (*SearchArtifactsResponse, error) - CreateTrigger(context.Context, *CreateTriggerRequest) (*CreateTriggerResponse, error) - DeactivateTrigger(context.Context, *DeactivateTriggerRequest) (*DeactivateTriggerResponse, error) - AddTag(context.Context, *AddTagRequest) (*AddTagResponse, error) - RegisterProducer(context.Context, *RegisterProducerRequest) (*RegisterResponse, error) - RegisterConsumer(context.Context, *RegisterConsumerRequest) (*RegisterResponse, error) - SetExecutionInputs(context.Context, *ExecutionInputsRequest) (*ExecutionInputsResponse, error) - FindByWorkflowExec(context.Context, *FindByWorkflowExecRequest) (*SearchArtifactsResponse, error) - ListUsage(context.Context, *ListUsageRequest) (*ListUsageResponse, error) -} - -// UnimplementedArtifactRegistryServer can be embedded to have forward compatible implementations. -type UnimplementedArtifactRegistryServer struct { -} - -func (*UnimplementedArtifactRegistryServer) CreateArtifact(ctx context.Context, req *CreateArtifactRequest) (*CreateArtifactResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreateArtifact not implemented") -} -func (*UnimplementedArtifactRegistryServer) GetArtifact(ctx context.Context, req *GetArtifactRequest) (*GetArtifactResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetArtifact not implemented") -} -func (*UnimplementedArtifactRegistryServer) SearchArtifacts(ctx context.Context, req *SearchArtifactsRequest) (*SearchArtifactsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method SearchArtifacts not implemented") -} -func (*UnimplementedArtifactRegistryServer) CreateTrigger(ctx context.Context, req *CreateTriggerRequest) (*CreateTriggerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreateTrigger not implemented") -} -func (*UnimplementedArtifactRegistryServer) DeactivateTrigger(ctx context.Context, req *DeactivateTriggerRequest) (*DeactivateTriggerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DeactivateTrigger not implemented") -} -func (*UnimplementedArtifactRegistryServer) AddTag(ctx context.Context, req *AddTagRequest) (*AddTagResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AddTag not implemented") -} -func (*UnimplementedArtifactRegistryServer) RegisterProducer(ctx context.Context, req *RegisterProducerRequest) (*RegisterResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method RegisterProducer not implemented") -} -func (*UnimplementedArtifactRegistryServer) RegisterConsumer(ctx context.Context, req *RegisterConsumerRequest) (*RegisterResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method RegisterConsumer not implemented") -} -func (*UnimplementedArtifactRegistryServer) SetExecutionInputs(ctx context.Context, req *ExecutionInputsRequest) (*ExecutionInputsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method SetExecutionInputs not implemented") -} -func (*UnimplementedArtifactRegistryServer) FindByWorkflowExec(ctx context.Context, req *FindByWorkflowExecRequest) (*SearchArtifactsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method FindByWorkflowExec not implemented") -} -func (*UnimplementedArtifactRegistryServer) ListUsage(ctx context.Context, req *ListUsageRequest) (*ListUsageResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ListUsage not implemented") -} - -func RegisterArtifactRegistryServer(s *grpc.Server, srv ArtifactRegistryServer) { - s.RegisterService(&_ArtifactRegistry_serviceDesc, srv) -} - -func _ArtifactRegistry_CreateArtifact_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreateArtifactRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).CreateArtifact(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/CreateArtifact", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).CreateArtifact(ctx, req.(*CreateArtifactRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_GetArtifact_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(GetArtifactRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).GetArtifact(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/GetArtifact", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).GetArtifact(ctx, req.(*GetArtifactRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_SearchArtifacts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(SearchArtifactsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).SearchArtifacts(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/SearchArtifacts", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).SearchArtifacts(ctx, req.(*SearchArtifactsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_CreateTrigger_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreateTriggerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).CreateTrigger(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/CreateTrigger", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).CreateTrigger(ctx, req.(*CreateTriggerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_DeactivateTrigger_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DeactivateTriggerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).DeactivateTrigger(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/DeactivateTrigger", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).DeactivateTrigger(ctx, req.(*DeactivateTriggerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_AddTag_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AddTagRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).AddTag(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/AddTag", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).AddTag(ctx, req.(*AddTagRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_RegisterProducer_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(RegisterProducerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).RegisterProducer(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/RegisterProducer", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).RegisterProducer(ctx, req.(*RegisterProducerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_RegisterConsumer_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(RegisterConsumerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).RegisterConsumer(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/RegisterConsumer", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).RegisterConsumer(ctx, req.(*RegisterConsumerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_SetExecutionInputs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ExecutionInputsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).SetExecutionInputs(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/SetExecutionInputs", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).SetExecutionInputs(ctx, req.(*ExecutionInputsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_FindByWorkflowExec_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(FindByWorkflowExecRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).FindByWorkflowExec(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/FindByWorkflowExec", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).FindByWorkflowExec(ctx, req.(*FindByWorkflowExecRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _ArtifactRegistry_ListUsage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ListUsageRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(ArtifactRegistryServer).ListUsage(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/flyteidl.artifact.ArtifactRegistry/ListUsage", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ArtifactRegistryServer).ListUsage(ctx, req.(*ListUsageRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _ArtifactRegistry_serviceDesc = grpc.ServiceDesc{ - ServiceName: "flyteidl.artifact.ArtifactRegistry", - HandlerType: (*ArtifactRegistryServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "CreateArtifact", - Handler: _ArtifactRegistry_CreateArtifact_Handler, - }, - { - MethodName: "GetArtifact", - Handler: _ArtifactRegistry_GetArtifact_Handler, - }, - { - MethodName: "SearchArtifacts", - Handler: _ArtifactRegistry_SearchArtifacts_Handler, - }, - { - MethodName: "CreateTrigger", - Handler: _ArtifactRegistry_CreateTrigger_Handler, - }, - { - MethodName: "DeactivateTrigger", - Handler: _ArtifactRegistry_DeactivateTrigger_Handler, - }, - { - MethodName: "AddTag", - Handler: _ArtifactRegistry_AddTag_Handler, - }, - { - MethodName: "RegisterProducer", - Handler: _ArtifactRegistry_RegisterProducer_Handler, - }, - { - MethodName: "RegisterConsumer", - Handler: _ArtifactRegistry_RegisterConsumer_Handler, - }, - { - MethodName: "SetExecutionInputs", - Handler: _ArtifactRegistry_SetExecutionInputs_Handler, - }, - { - MethodName: "FindByWorkflowExec", - Handler: _ArtifactRegistry_FindByWorkflowExec_Handler, - }, - { - MethodName: "ListUsage", - Handler: _ArtifactRegistry_ListUsage_Handler, - }, - }, - Streams: []grpc.StreamDesc{}, - Metadata: "flyteidl/artifact/artifacts.proto", -} diff --git a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.gw.go b/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.gw.go deleted file mode 100644 index 6e92073451..0000000000 --- a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.pb.gw.go +++ /dev/null @@ -1,390 +0,0 @@ -// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT. -// source: flyteidl/artifact/artifacts.proto - -/* -Package artifact is a reverse proxy. - -It translates gRPC into RESTful JSON APIs. -*/ -package artifact - -import ( - "context" - "io" - "net/http" - - "github.com/golang/protobuf/proto" - "github.com/grpc-ecosystem/grpc-gateway/runtime" - "github.com/grpc-ecosystem/grpc-gateway/utilities" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/grpclog" - "google.golang.org/grpc/status" -) - -var _ codes.Code -var _ io.Reader -var _ status.Status -var _ = runtime.String -var _ = utilities.NewDoubleArray - -func request_ArtifactRegistry_GetArtifact_0(ctx context.Context, marshaler runtime.Marshaler, client ArtifactRegistryClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { - var protoReq GetArtifactRequest - var metadata runtime.ServerMetadata - - newReader, berr := utilities.IOReaderFactory(req.Body) - if berr != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) - } - if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - - msg, err := client.GetArtifact(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) - return msg, metadata, err - -} - -func request_ArtifactRegistry_SearchArtifacts_0(ctx context.Context, marshaler runtime.Marshaler, client ArtifactRegistryClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { - var protoReq SearchArtifactsRequest - var metadata runtime.ServerMetadata - - newReader, berr := utilities.IOReaderFactory(req.Body) - if berr != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) - } - if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - - msg, err := client.SearchArtifacts(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) - return msg, metadata, err - -} - -func request_ArtifactRegistry_DeactivateTrigger_0(ctx context.Context, marshaler runtime.Marshaler, client ArtifactRegistryClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { - var protoReq DeactivateTriggerRequest - var metadata runtime.ServerMetadata - - newReader, berr := utilities.IOReaderFactory(req.Body) - if berr != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) - } - if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - - msg, err := client.DeactivateTrigger(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) - return msg, metadata, err - -} - -var ( - filter_ArtifactRegistry_FindByWorkflowExec_0 = &utilities.DoubleArray{Encoding: map[string]int{"exec_id": 0, "project": 1, "domain": 2, "name": 3, "direction": 4}, Base: []int{1, 1, 1, 2, 3, 4, 0, 0, 0, 0}, Check: []int{0, 1, 2, 2, 2, 1, 3, 4, 5, 6}} -) - -func request_ArtifactRegistry_FindByWorkflowExec_0(ctx context.Context, marshaler runtime.Marshaler, client ArtifactRegistryClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { - var protoReq FindByWorkflowExecRequest - var metadata runtime.ServerMetadata - - var ( - val string - e int32 - ok bool - err error - _ = err - ) - - val, ok = pathParams["exec_id.project"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "exec_id.project") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "exec_id.project", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "exec_id.project", err) - } - - val, ok = pathParams["exec_id.domain"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "exec_id.domain") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "exec_id.domain", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "exec_id.domain", err) - } - - val, ok = pathParams["exec_id.name"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "exec_id.name") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "exec_id.name", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "exec_id.name", err) - } - - val, ok = pathParams["direction"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "direction") - } - - e, err = runtime.Enum(val, FindByWorkflowExecRequest_Direction_value) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "direction", err) - } - - protoReq.Direction = FindByWorkflowExecRequest_Direction(e) - - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ArtifactRegistry_FindByWorkflowExec_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - - msg, err := client.FindByWorkflowExec(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) - return msg, metadata, err - -} - -var ( - filter_ArtifactRegistry_ListUsage_0 = &utilities.DoubleArray{Encoding: map[string]int{"artifact_id": 0, "artifact_key": 1, "project": 2, "domain": 3, "name": 4, "version": 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_ArtifactRegistry_ListUsage_0(ctx context.Context, marshaler runtime.Marshaler, client ArtifactRegistryClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { - var protoReq ListUsageRequest - var metadata runtime.ServerMetadata - - var ( - val string - ok bool - err error - _ = err - ) - - val, ok = pathParams["artifact_id.artifact_key.project"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "artifact_id.artifact_key.project") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "artifact_id.artifact_key.project", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "artifact_id.artifact_key.project", err) - } - - val, ok = pathParams["artifact_id.artifact_key.domain"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "artifact_id.artifact_key.domain") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "artifact_id.artifact_key.domain", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "artifact_id.artifact_key.domain", err) - } - - val, ok = pathParams["artifact_id.artifact_key.name"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "artifact_id.artifact_key.name") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "artifact_id.artifact_key.name", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "artifact_id.artifact_key.name", err) - } - - val, ok = pathParams["artifact_id.version"] - if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "artifact_id.version") - } - - err = runtime.PopulateFieldFromPath(&protoReq, "artifact_id.version", val) - - if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "artifact_id.version", err) - } - - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ArtifactRegistry_ListUsage_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) - } - - msg, err := client.ListUsage(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) - return msg, metadata, err - -} - -// RegisterArtifactRegistryHandlerFromEndpoint is same as RegisterArtifactRegistryHandler but -// automatically dials to "endpoint" and closes the connection when "ctx" gets done. -func RegisterArtifactRegistryHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { - conn, err := grpc.Dial(endpoint, opts...) - if err != nil { - return err - } - defer func() { - if err != nil { - if cerr := conn.Close(); cerr != nil { - grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr) - } - return - } - go func() { - <-ctx.Done() - if cerr := conn.Close(); cerr != nil { - grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr) - } - }() - }() - - return RegisterArtifactRegistryHandler(ctx, mux, conn) -} - -// RegisterArtifactRegistryHandler registers the http handlers for service ArtifactRegistry to "mux". -// The handlers forward requests to the grpc endpoint over "conn". -func RegisterArtifactRegistryHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error { - return RegisterArtifactRegistryHandlerClient(ctx, mux, NewArtifactRegistryClient(conn)) -} - -// RegisterArtifactRegistryHandlerClient registers the http handlers for service ArtifactRegistry -// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "ArtifactRegistryClient". -// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "ArtifactRegistryClient" -// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in -// "ArtifactRegistryClient" to call the correct interceptors. -func RegisterArtifactRegistryHandlerClient(ctx context.Context, mux *runtime.ServeMux, client ArtifactRegistryClient) error { - - mux.Handle("POST", pattern_ArtifactRegistry_GetArtifact_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_ArtifactRegistry_GetArtifact_0(rctx, inboundMarshaler, client, req, pathParams) - ctx = runtime.NewServerMetadataContext(ctx, md) - if err != nil { - runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) - return - } - - forward_ArtifactRegistry_GetArtifact_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) - - }) - - mux.Handle("POST", pattern_ArtifactRegistry_SearchArtifacts_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_ArtifactRegistry_SearchArtifacts_0(rctx, inboundMarshaler, client, req, pathParams) - ctx = runtime.NewServerMetadataContext(ctx, md) - if err != nil { - runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) - return - } - - forward_ArtifactRegistry_SearchArtifacts_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) - - }) - - mux.Handle("PATCH", pattern_ArtifactRegistry_DeactivateTrigger_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_ArtifactRegistry_DeactivateTrigger_0(rctx, inboundMarshaler, client, req, pathParams) - ctx = runtime.NewServerMetadataContext(ctx, md) - if err != nil { - runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) - return - } - - forward_ArtifactRegistry_DeactivateTrigger_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) - - }) - - mux.Handle("GET", pattern_ArtifactRegistry_FindByWorkflowExec_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_ArtifactRegistry_FindByWorkflowExec_0(rctx, inboundMarshaler, client, req, pathParams) - ctx = runtime.NewServerMetadataContext(ctx, md) - if err != nil { - runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) - return - } - - forward_ArtifactRegistry_FindByWorkflowExec_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) - - }) - - mux.Handle("GET", pattern_ArtifactRegistry_ListUsage_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_ArtifactRegistry_ListUsage_0(rctx, inboundMarshaler, client, req, pathParams) - ctx = runtime.NewServerMetadataContext(ctx, md) - if err != nil { - runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) - return - } - - forward_ArtifactRegistry_ListUsage_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) - - }) - - return nil -} - -var ( - pattern_ArtifactRegistry_GetArtifact_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 0}, []string{"artifacts", "api", "v1"}, "")) - - pattern_ArtifactRegistry_SearchArtifacts_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"artifacts", "api", "v1", "search"}, "")) - - pattern_ArtifactRegistry_DeactivateTrigger_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4}, []string{"artifacts", "api", "v1", "trigger", "deactivate"}, "")) - - pattern_ArtifactRegistry_FindByWorkflowExec_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 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{"artifacts", "api", "v1", "search", "execution", "exec_id.project", "exec_id.domain", "exec_id.name", "direction"}, "")) - - pattern_ArtifactRegistry_ListUsage_0 = 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{"artifacts", "api", "v1", "usage", "artifact_id.artifact_key.project", "artifact_id.artifact_key.domain", "artifact_id.artifact_key.name", "artifact_id.version"}, "")) -) - -var ( - forward_ArtifactRegistry_GetArtifact_0 = runtime.ForwardResponseMessage - - forward_ArtifactRegistry_SearchArtifacts_0 = runtime.ForwardResponseMessage - - forward_ArtifactRegistry_DeactivateTrigger_0 = runtime.ForwardResponseMessage - - forward_ArtifactRegistry_FindByWorkflowExec_0 = runtime.ForwardResponseMessage - - forward_ArtifactRegistry_ListUsage_0 = runtime.ForwardResponseMessage -) diff --git a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.swagger.json b/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.swagger.json deleted file mode 100644 index 47785a90b0..0000000000 --- a/flyteidl/gen/pb-go/flyteidl/artifact/artifacts.swagger.json +++ /dev/null @@ -1,1783 +0,0 @@ -{ - "swagger": "2.0", - "info": { - "title": "flyteidl/artifact/artifacts.proto", - "version": "version not set" - }, - "schemes": [ - "http", - "https" - ], - "consumes": [ - "application/json" - ], - "produces": [ - "application/json" - ], - "paths": { - "/artifacts/api/v1/artifacts": { - "post": { - "operationId": "GetArtifact", - "responses": { - "200": { - "description": "A successful response.", - "schema": { - "$ref": "#/definitions/artifactGetArtifactResponse" - } - } - }, - "parameters": [ - { - "name": "body", - "in": "body", - "required": true, - "schema": { - "$ref": "#/definitions/artifactGetArtifactRequest" - } - } - ], - "tags": [ - "ArtifactRegistry" - ] - } - }, - "/artifacts/api/v1/search": { - "post": { - "operationId": "SearchArtifacts", - "responses": { - "200": { - "description": "A successful response.", - "schema": { - "$ref": "#/definitions/artifactSearchArtifactsResponse" - } - } - }, - "parameters": [ - { - "name": "body", - "in": "body", - "required": true, - "schema": { - "$ref": "#/definitions/artifactSearchArtifactsRequest" - } - } - ], - "tags": [ - "ArtifactRegistry" - ] - } - }, - "/artifacts/api/v1/search/execution/{exec_id.project}/{exec_id.domain}/{exec_id.name}/{direction}": { - "get": { - "operationId": "FindByWorkflowExec", - "responses": { - "200": { - "description": "A successful response.", - "schema": { - "$ref": "#/definitions/artifactSearchArtifactsResponse" - } - } - }, - "parameters": [ - { - "name": "exec_id.project", - "description": "Name of the project the resource belongs to.", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "exec_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": "exec_id.name", - "description": "User or system provided value for the resource.", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "direction", - "in": "path", - "required": true, - "type": "string", - "enum": [ - "INPUTS", - "OUTPUTS" - ] - }, - { - "name": "exec_id.org", - "description": "Optional, org key applied to the resource.", - "in": "query", - "required": false, - "type": "string" - } - ], - "tags": [ - "ArtifactRegistry" - ] - } - }, - "/artifacts/api/v1/trigger/deactivate": { - "patch": { - "operationId": "DeactivateTrigger", - "responses": { - "200": { - "description": "A successful response.", - "schema": { - "$ref": "#/definitions/artifactDeactivateTriggerResponse" - } - } - }, - "parameters": [ - { - "name": "body", - "in": "body", - "required": true, - "schema": { - "$ref": "#/definitions/artifactDeactivateTriggerRequest" - } - } - ], - "tags": [ - "ArtifactRegistry" - ] - } - }, - "/artifacts/api/v1/usage/{artifact_id.artifact_key.project}/{artifact_id.artifact_key.domain}/{artifact_id.artifact_key.name}/{artifact_id.version}": { - "get": { - "operationId": "ListUsage", - "responses": { - "200": { - "description": "A successful response.", - "schema": { - "$ref": "#/definitions/artifactListUsageResponse" - } - } - }, - "parameters": [ - { - "name": "artifact_id.artifact_key.project", - "description": "Project and domain and suffix needs to be unique across a given artifact store.", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "artifact_id.artifact_key.domain", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "artifact_id.artifact_key.name", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "artifact_id.version", - "in": "path", - "required": true, - "type": "string" - }, - { - "name": "artifact_id.time_partition.value.static_value", - "description": "The string static value is for use in the Partitions object.", - "in": "query", - "required": false, - "type": "string" - }, - { - "name": "artifact_id.time_partition.value.time_value", - "description": "The time value is for use in the TimePartition case.", - "in": "query", - "required": false, - "type": "string", - "format": "date-time" - }, - { - "name": "artifact_id.time_partition.value.triggered_binding.index", - "in": "query", - "required": false, - "type": "integer", - "format": "int64" - }, - { - "name": "artifact_id.time_partition.value.triggered_binding.partition_key", - "in": "query", - "required": false, - "type": "string" - }, - { - "name": "artifact_id.time_partition.value.triggered_binding.bind_to_time_partition", - "in": "query", - "required": false, - "type": "boolean", - "format": "boolean" - }, - { - "name": "artifact_id.time_partition.value.triggered_binding.transform", - "description": "This is only relevant in the time partition case.", - "in": "query", - "required": false, - "type": "string" - }, - { - "name": "artifact_id.time_partition.value.input_binding.var", - "in": "query", - "required": false, - "type": "string" - } - ], - "tags": [ - "ArtifactRegistry" - ] - } - } - }, - "definitions": { - "BlobTypeBlobDimensionality": { - "type": "string", - "enum": [ - "SINGLE", - "MULTIPART" - ], - "default": "SINGLE" - }, - "QualityOfServiceTier": { - "type": "string", - "enum": [ - "UNDEFINED", - "HIGH", - "MEDIUM", - "LOW" - ], - "default": "UNDEFINED", - "description": " - UNDEFINED: Default: no quality of service specified." - }, - "SchemaColumnSchemaColumnType": { - "type": "string", - "enum": [ - "INTEGER", - "FLOAT", - "STRING", - "BOOLEAN", - "DATETIME", - "DURATION" - ], - "default": "INTEGER" - }, - "SchemaTypeSchemaColumn": { - "type": "object", - "properties": { - "name": { - "type": "string", - "title": "A unique name -within the schema type- for the column" - }, - "type": { - "$ref": "#/definitions/SchemaColumnSchemaColumnType", - "description": "The column type. This allows a limited set of types currently." - } - } - }, - "SecretMountType": { - "type": "string", - "enum": [ - "ANY", - "ENV_VAR", - "FILE" - ], - "default": "ANY", - "description": " - ANY: Default case, indicates the client can tolerate either mounting options.\n - ENV_VAR: ENV_VAR indicates the secret needs to be mounted as an environment variable.\n - FILE: FILE indicates the secret needs to be mounted as a file." - }, - "StructuredDatasetTypeDatasetColumn": { - "type": "object", - "properties": { - "name": { - "type": "string", - "description": "A unique name within the schema type for the column." - }, - "literal_type": { - "$ref": "#/definitions/coreLiteralType", - "description": "The column type." - } - } - }, - "adminAnnotations": { - "type": "object", - "properties": { - "values": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Map of custom annotations to be applied to the execution resource." - } - }, - "description": "Annotation values to be applied to an execution resource.\nIn the future a mode (e.g. OVERRIDE, APPEND, etc) can be defined\nto specify how to merge annotations defined at registration and execution time." - }, - "adminAuth": { - "type": "object", - "properties": { - "assumable_iam_role": { - "type": "string", - "description": "Defines an optional iam role which will be used for tasks run in executions created with this launch plan." - }, - "kubernetes_service_account": { - "type": "string", - "description": "Defines an optional kubernetes service account which will be used for tasks run in executions created with this launch plan." - } - }, - "description": "Defines permissions associated with executions created by this launch plan spec.\nUse either of these roles when they have permissions required by your workflow execution.\nDeprecated." - }, - "adminAuthRole": { - "type": "object", - "properties": { - "assumable_iam_role": { - "type": "string", - "description": "Defines an optional iam role which will be used for tasks run in executions created with this launch plan." - }, - "kubernetes_service_account": { - "type": "string", - "description": "Defines an optional kubernetes service account which will be used for tasks run in executions created with this launch plan." - } - }, - "description": "Defines permissions associated with executions created by this launch plan spec.\nUse either of these roles when they have permissions required by your workflow execution.\nDeprecated." - }, - "adminCronSchedule": { - "type": "object", - "properties": { - "schedule": { - "type": "string", - "title": "Standard/default cron implementation as described by https://en.wikipedia.org/wiki/Cron#CRON_expression;\nAlso supports nonstandard predefined scheduling definitions\nas described by https://docs.aws.amazon.com/AmazonCloudWatch/latest/events/ScheduledEvents.html#CronExpressions\nexcept @reboot" - }, - "offset": { - "type": "string", - "title": "ISO 8601 duration as described by https://en.wikipedia.org/wiki/ISO_8601#Durations" - } - }, - "description": "Options for schedules to run according to a cron expression." - }, - "adminEmailNotification": { - "type": "object", - "properties": { - "recipients_email": { - "type": "array", - "items": { - "type": "string" - }, - "title": "The list of email addresses recipients for this notification.\n+required" - } - }, - "description": "Defines an email notification specification." - }, - "adminEnvs": { - "type": "object", - "properties": { - "values": { - "type": "array", - "items": { - "$ref": "#/definitions/coreKeyValuePair" - }, - "description": "Map of custom environment variables to be applied to the execution resource." - } - }, - "description": "Environment variable values to be applied to an execution resource.\nIn the future a mode (e.g. OVERRIDE, APPEND, etc) can be defined\nto specify how to merge environment variables defined at registration and execution time." - }, - "adminFixedRate": { - "type": "object", - "properties": { - "value": { - "type": "integer", - "format": "int64" - }, - "unit": { - "$ref": "#/definitions/adminFixedRateUnit" - } - }, - "description": "Option for schedules run at a certain frequency e.g. every 2 minutes." - }, - "adminFixedRateUnit": { - "type": "string", - "enum": [ - "MINUTE", - "HOUR", - "DAY" - ], - "default": "MINUTE", - "description": "Represents a frequency at which to run a schedule." - }, - "adminLabels": { - "type": "object", - "properties": { - "values": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Map of custom labels to be applied to the execution resource." - } - }, - "description": "Label values to be applied to an execution resource.\nIn the future a mode (e.g. OVERRIDE, APPEND, etc) can be defined\nto specify how to merge labels defined at registration and execution time." - }, - "adminLaunchPlan": { - "type": "object", - "properties": { - "id": { - "$ref": "#/definitions/coreIdentifier", - "description": "Uniquely identifies a launch plan entity." - }, - "spec": { - "$ref": "#/definitions/adminLaunchPlanSpec", - "description": "User-provided launch plan details, including reference workflow, inputs and other metadata." - }, - "closure": { - "$ref": "#/definitions/adminLaunchPlanClosure", - "description": "Values computed by the flyte platform after launch plan registration." - } - }, - "description": "A LaunchPlan provides the capability to templatize workflow executions.\nLaunch plans simplify associating one or more schedules, inputs and notifications with your workflows.\nLaunch plans can be shared and used to trigger executions with predefined inputs even when a workflow\ndefinition doesn't necessarily have a default value for said input." - }, - "adminLaunchPlanClosure": { - "type": "object", - "properties": { - "state": { - "$ref": "#/definitions/adminLaunchPlanState", - "description": "Indicate the Launch plan state." - }, - "expected_inputs": { - "$ref": "#/definitions/coreParameterMap", - "title": "Indicates the set of inputs expected when creating an execution with the Launch plan" - }, - "expected_outputs": { - "$ref": "#/definitions/coreVariableMap", - "title": "Indicates the set of outputs expected to be produced by creating an execution with the Launch plan" - }, - "created_at": { - "type": "string", - "format": "date-time", - "description": "Time at which the launch plan was created." - }, - "updated_at": { - "type": "string", - "format": "date-time", - "description": "Time at which the launch plan was last updated." - } - }, - "description": "Values computed by the flyte platform after launch plan registration.\nThese include expected_inputs required to be present in a CreateExecutionRequest\nto launch the reference workflow as well timestamp values associated with the launch plan." - }, - "adminLaunchPlanMetadata": { - "type": "object", - "properties": { - "schedule": { - "$ref": "#/definitions/adminSchedule", - "title": "Schedule to execute the Launch Plan" - }, - "notifications": { - "type": "array", - "items": { - "$ref": "#/definitions/adminNotification" - }, - "title": "List of notifications based on Execution status transitions" - }, - "launch_conditions": { - "$ref": "#/definitions/protobufAny", - "title": "Additional metadata for how to launch the launch plan" - } - }, - "description": "Additional launch plan attributes included in the LaunchPlanSpec not strictly required to launch\nthe reference workflow." - }, - "adminLaunchPlanSpec": { - "type": "object", - "properties": { - "workflow_id": { - "$ref": "#/definitions/coreIdentifier", - "title": "Reference to the Workflow template that the launch plan references" - }, - "entity_metadata": { - "$ref": "#/definitions/adminLaunchPlanMetadata", - "title": "Metadata for the Launch Plan" - }, - "default_inputs": { - "$ref": "#/definitions/coreParameterMap", - "description": "Input values to be passed for the execution.\nThese can be overridden when an execution is created with this launch plan." - }, - "fixed_inputs": { - "$ref": "#/definitions/coreLiteralMap", - "description": "Fixed, non-overridable inputs for the Launch Plan.\nThese can not be overridden when an execution is created with this launch plan." - }, - "role": { - "type": "string", - "title": "String to indicate the role to use to execute the workflow underneath" - }, - "labels": { - "$ref": "#/definitions/adminLabels", - "description": "Custom labels to be applied to the execution resource." - }, - "annotations": { - "$ref": "#/definitions/adminAnnotations", - "description": "Custom annotations to be applied to the execution resource." - }, - "auth": { - "$ref": "#/definitions/adminAuth", - "description": "Indicates the permission associated with workflow executions triggered with this launch plan." - }, - "auth_role": { - "$ref": "#/definitions/adminAuthRole" - }, - "security_context": { - "$ref": "#/definitions/coreSecurityContext", - "title": "Indicates security context for permissions triggered with this launch plan" - }, - "quality_of_service": { - "$ref": "#/definitions/coreQualityOfService", - "description": "Indicates the runtime priority of the execution." - }, - "raw_output_data_config": { - "$ref": "#/definitions/adminRawOutputDataConfig", - "description": "Encapsulates user settings pertaining to offloaded data (i.e. Blobs, Schema, query data, etc.)." - }, - "max_parallelism": { - "type": "integer", - "format": "int32", - "description": "Controls the maximum number of tasknodes that can be run in parallel for the entire workflow.\nThis is useful to achieve fairness. Note: MapTasks are regarded as one unit,\nand parallelism/concurrency of MapTasks is independent from this." - }, - "interruptible": { - "type": "boolean", - "format": "boolean", - "description": "Allows for the interruptible flag of a workflow to be overwritten for a single execution.\nOmitting this field uses the workflow's value as a default.\nAs we need to distinguish between the field not being provided and its default value false, we have to use a wrapper\naround the bool field." - }, - "overwrite_cache": { - "type": "boolean", - "format": "boolean", - "description": "Allows for all cached values of a workflow and its tasks to be overwritten for a single execution.\nIf enabled, all calculations are performed even if cached results would be available, overwriting the stored\ndata once execution finishes successfully." - }, - "envs": { - "$ref": "#/definitions/adminEnvs", - "description": "Environment variables to be set for the execution." - } - }, - "description": "User-provided launch plan definition and configuration values." - }, - "adminLaunchPlanState": { - "type": "string", - "enum": [ - "INACTIVE", - "ACTIVE" - ], - "default": "INACTIVE", - "description": "By default any launch plan regardless of state can be used to launch a workflow execution.\nHowever, at most one version of a launch plan\n(e.g. a NamedEntityIdentifier set of shared project, domain and name values) can be\nactive at a time in regards to *schedules*. That is, at most one schedule in a NamedEntityIdentifier\ngroup will be observed and trigger executions at a defined cadence." - }, - "adminNotification": { - "type": "object", - "properties": { - "phases": { - "type": "array", - "items": { - "$ref": "#/definitions/coreWorkflowExecutionPhase" - }, - "title": "A list of phases to which users can associate the notifications to.\n+required" - }, - "email": { - "$ref": "#/definitions/adminEmailNotification" - }, - "pager_duty": { - "$ref": "#/definitions/adminPagerDutyNotification" - }, - "slack": { - "$ref": "#/definitions/adminSlackNotification" - } - }, - "description": "Represents a structure for notifications based on execution status.\nThe notification content is configured within flyte admin but can be templatized.\nFuture iterations could expose configuring notifications with custom content." - }, - "adminPagerDutyNotification": { - "type": "object", - "properties": { - "recipients_email": { - "type": "array", - "items": { - "type": "string" - }, - "title": "Currently, PagerDuty notifications leverage email to trigger a notification.\n+required" - } - }, - "description": "Defines a pager duty notification specification." - }, - "adminRawOutputDataConfig": { - "type": "object", - "properties": { - "output_location_prefix": { - "type": "string", - "title": "Prefix for where offloaded data from user workflows will be written\ne.g. s3://bucket/key or s3://bucket/" - } - }, - "description": "Encapsulates user settings pertaining to offloaded data (i.e. Blobs, Schema, query data, etc.).\nSee https://github.com/flyteorg/flyte/issues/211 for more background information." - }, - "adminSchedule": { - "type": "object", - "properties": { - "cron_expression": { - "type": "string", - "title": "Uses AWS syntax: Minutes Hours Day-of-month Month Day-of-week Year\ne.g. for a schedule that runs every 15 minutes: 0/15 * * * ? *" - }, - "rate": { - "$ref": "#/definitions/adminFixedRate" - }, - "cron_schedule": { - "$ref": "#/definitions/adminCronSchedule" - }, - "kickoff_time_input_arg": { - "type": "string", - "description": "Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off." - } - }, - "description": "Defines complete set of information required to trigger an execution on a schedule." - }, - "adminSlackNotification": { - "type": "object", - "properties": { - "recipients_email": { - "type": "array", - "items": { - "type": "string" - }, - "title": "Currently, Slack notifications leverage email to trigger a notification.\n+required" - } - }, - "description": "Defines a slack notification specification." - }, - "artifactAddTagResponse": { - "type": "object" - }, - "artifactArtifact": { - "type": "object", - "properties": { - "artifact_id": { - "$ref": "#/definitions/coreArtifactID" - }, - "spec": { - "$ref": "#/definitions/artifactArtifactSpec" - }, - "tags": { - "type": "array", - "items": { - "type": "string" - }, - "title": "references the tag field in ArtifactTag" - }, - "source": { - "$ref": "#/definitions/artifactArtifactSource" - } - } - }, - "artifactArtifactConsumer": { - "type": "object", - "properties": { - "entity_id": { - "$ref": "#/definitions/coreIdentifier", - "title": "These should all be launch plan IDs" - }, - "inputs": { - "$ref": "#/definitions/coreParameterMap" - } - } - }, - "artifactArtifactProducer": { - "type": "object", - "properties": { - "entity_id": { - "$ref": "#/definitions/coreIdentifier", - "description": "These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in\nAdmin's domain." - }, - "outputs": { - "$ref": "#/definitions/coreVariableMap" - } - } - }, - "artifactArtifactSource": { - "type": "object", - "properties": { - "workflow_execution": { - "$ref": "#/definitions/coreWorkflowExecutionIdentifier" - }, - "node_id": { - "type": "string" - }, - "task_id": { - "$ref": "#/definitions/coreIdentifier" - }, - "retry_attempt": { - "type": "integer", - "format": "int64" - }, - "principal": { - "type": "string", - "description": "Uploads, either from the UI or from the CLI, or FlyteRemote, will have this." - } - } - }, - "artifactArtifactSpec": { - "type": "object", - "properties": { - "value": { - "$ref": "#/definitions/coreLiteral" - }, - "type": { - "$ref": "#/definitions/coreLiteralType", - "description": "This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but\nforgets to change the name, that is okay. And the reason why this is a separate field is because adding the\ntype to all Literals is a lot of work." - }, - "short_description": { - "type": "string" - }, - "user_metadata": { - "$ref": "#/definitions/protobufAny", - "title": "Additional user metadata" - }, - "metadata_type": { - "type": "string" - }, - "created_at": { - "type": "string", - "format": "date-time" - }, - "file_format": { - "type": "string" - } - } - }, - "artifactCreateArtifactResponse": { - "type": "object", - "properties": { - "artifact": { - "$ref": "#/definitions/artifactArtifact" - } - } - }, - "artifactCreateTriggerResponse": { - "type": "object" - }, - "artifactDeactivateTriggerRequest": { - "type": "object", - "properties": { - "trigger_id": { - "$ref": "#/definitions/coreIdentifier" - } - } - }, - "artifactDeactivateTriggerResponse": { - "type": "object" - }, - "artifactExecutionInputsResponse": { - "type": "object" - }, - "artifactFindByWorkflowExecRequestDirection": { - "type": "string", - "enum": [ - "INPUTS", - "OUTPUTS" - ], - "default": "INPUTS" - }, - "artifactGetArtifactRequest": { - "type": "object", - "properties": { - "query": { - "$ref": "#/definitions/coreArtifactQuery" - }, - "details": { - "type": "boolean", - "format": "boolean", - "description": "If false, then long_description is not returned." - } - } - }, - "artifactGetArtifactResponse": { - "type": "object", - "properties": { - "artifact": { - "$ref": "#/definitions/artifactArtifact" - } - } - }, - "artifactListUsageResponse": { - "type": "object", - "properties": { - "executions": { - "type": "array", - "items": { - "$ref": "#/definitions/coreWorkflowExecutionIdentifier" - } - } - } - }, - "artifactRegisterResponse": { - "type": "object" - }, - "artifactSearchArtifactsRequest": { - "type": "object", - "properties": { - "artifact_key": { - "$ref": "#/definitions/coreArtifactKey" - }, - "partitions": { - "$ref": "#/definitions/corePartitions" - }, - "time_partition_value": { - "type": "string", - "format": "date-time" - }, - "principal": { - "type": "string" - }, - "version": { - "type": "string" - }, - "options": { - "$ref": "#/definitions/artifactSearchOptions" - }, - "token": { - "type": "string" - }, - "limit": { - "type": "integer", - "format": "int32" - } - } - }, - "artifactSearchArtifactsResponse": { - "type": "object", - "properties": { - "artifacts": { - "type": "array", - "items": { - "$ref": "#/definitions/artifactArtifact" - }, - "description": "If artifact specs are not requested, the resultant artifacts may be empty." - }, - "token": { - "type": "string", - "description": "continuation token if relevant." - } - } - }, - "artifactSearchOptions": { - "type": "object", - "properties": { - "strict_partitions": { - "type": "boolean", - "format": "boolean", - "description": "If true, this means a strict partition search. meaning if you don't specify the partition\nfield, that will mean, non-partitioned, rather than any partition." - }, - "latest_by_key": { - "type": "boolean", - "format": "boolean", - "description": "If true, only one artifact per key will be returned. It will be the latest one by creation time." - } - } - }, - "coreArtifactBindingData": { - "type": "object", - "properties": { - "index": { - "type": "integer", - "format": "int64" - }, - "partition_key": { - "type": "string" - }, - "bind_to_time_partition": { - "type": "boolean", - "format": "boolean" - }, - "transform": { - "type": "string", - "title": "This is only relevant in the time partition case" - } - }, - "title": "Only valid for triggers" - }, - "coreArtifactID": { - "type": "object", - "properties": { - "artifact_key": { - "$ref": "#/definitions/coreArtifactKey" - }, - "version": { - "type": "string" - }, - "partitions": { - "$ref": "#/definitions/corePartitions", - "description": "Think of a partition as a tag on an Artifact, except it's a key-value pair.\nDifferent partitions naturally have different versions (execution ids)." - }, - "time_partition": { - "$ref": "#/definitions/coreTimePartition", - "description": "There is no such thing as an empty time partition - if it's not set, then there is no time partition." - } - } - }, - "coreArtifactKey": { - "type": "object", - "properties": { - "project": { - "type": "string", - "description": "Project and domain and suffix needs to be unique across a given artifact store." - }, - "domain": { - "type": "string" - }, - "name": { - "type": "string" - } - } - }, - "coreArtifactQuery": { - "type": "object", - "properties": { - "artifact_id": { - "$ref": "#/definitions/coreArtifactID" - }, - "artifact_tag": { - "$ref": "#/definitions/coreArtifactTag" - }, - "uri": { - "type": "string" - }, - "binding": { - "$ref": "#/definitions/coreArtifactBindingData", - "description": "This is used in the trigger case, where a user specifies a value for an input that is one of the triggering\nartifacts, or a partition value derived from a triggering artifact." - } - }, - "title": "Uniqueness constraints for Artifacts\n - project, domain, name, version, partitions\nOption 2 (tags are standalone, point to an individual artifact id):\n - project, domain, name, alias (points to one partition if partitioned)\n - project, domain, name, partition key, partition value" - }, - "coreArtifactTag": { - "type": "object", - "properties": { - "artifact_key": { - "$ref": "#/definitions/coreArtifactKey" - }, - "value": { - "$ref": "#/definitions/coreLabelValue" - } - } - }, - "coreBinary": { - "type": "object", - "properties": { - "value": { - "type": "string", - "format": "byte" - }, - "tag": { - "type": "string" - } - }, - "description": "A simple byte array with a tag to help different parts of the system communicate about what is in the byte array.\nIt's strongly advisable that consumers of this type define a unique tag and validate the tag before parsing the data." - }, - "coreBlob": { - "type": "object", - "properties": { - "metadata": { - "$ref": "#/definitions/coreBlobMetadata" - }, - "uri": { - "type": "string" - } - }, - "description": "Refers to an offloaded set of files. It encapsulates the type of the store and a unique uri for where the data is.\nThere are no restrictions on how the uri is formatted since it will depend on how to interact with the store." - }, - "coreBlobMetadata": { - "type": "object", - "properties": { - "type": { - "$ref": "#/definitions/coreBlobType" - } - } - }, - "coreBlobType": { - "type": "object", - "properties": { - "format": { - "type": "string", - "title": "Format can be a free form string understood by SDK/UI etc like\ncsv, parquet etc" - }, - "dimensionality": { - "$ref": "#/definitions/BlobTypeBlobDimensionality" - } - }, - "title": "Defines type behavior for blob objects" - }, - "coreEnumType": { - "type": "object", - "properties": { - "values": { - "type": "array", - "items": { - "type": "string" - }, - "description": "Predefined set of enum values." - } - }, - "description": "Enables declaring enum types, with predefined string values\nFor len(values) \u003e 0, the first value in the ordered list is regarded as the default value. If you wish\nTo provide no defaults, make the first value as undefined." - }, - "coreError": { - "type": "object", - "properties": { - "failed_node_id": { - "type": "string", - "description": "The node id that threw the error." - }, - "message": { - "type": "string", - "description": "Error message thrown." - } - }, - "description": "Represents an error thrown from a node." - }, - "coreIdentifier": { - "type": "object", - "properties": { - "resource_type": { - "$ref": "#/definitions/coreResourceType", - "description": "Identifies the specific type of resource that this identifier corresponds to." - }, - "project": { - "type": "string", - "description": "Name of the project the resource belongs to." - }, - "domain": { - "type": "string", - "description": "Name of the domain the resource belongs to.\nA domain can be considered as a subset within a specific project." - }, - "name": { - "type": "string", - "description": "User provided value for the resource." - }, - "version": { - "type": "string", - "description": "Specific version of the resource." - }, - "org": { - "type": "string", - "description": "Optional, org key applied to the resource." - } - }, - "description": "Encapsulation of fields that uniquely identifies a Flyte resource." - }, - "coreIdentity": { - "type": "object", - "properties": { - "iam_role": { - "type": "string", - "description": "iam_role references the fully qualified name of Identity \u0026 Access Management role to impersonate." - }, - "k8s_service_account": { - "type": "string", - "description": "k8s_service_account references a kubernetes service account to impersonate." - }, - "oauth2_client": { - "$ref": "#/definitions/coreOAuth2Client", - "description": "oauth2_client references an oauth2 client. Backend plugins can use this information to impersonate the client when\nmaking external calls." - }, - "execution_identity": { - "type": "string", - "title": "execution_identity references the subject who makes the execution" - } - }, - "description": "Identity encapsulates the various security identities a task can run as. It's up to the underlying plugin to pick the\nright identity for the execution environment." - }, - "coreInputBindingData": { - "type": "object", - "properties": { - "var": { - "type": "string" - } - } - }, - "coreKeyValuePair": { - "type": "object", - "properties": { - "key": { - "type": "string", - "description": "required." - }, - "value": { - "type": "string", - "description": "+optional." - } - }, - "description": "A generic key value pair." - }, - "coreLabelValue": { - "type": "object", - "properties": { - "static_value": { - "type": "string", - "title": "The string static value is for use in the Partitions object" - }, - "time_value": { - "type": "string", - "format": "date-time", - "title": "The time value is for use in the TimePartition case" - }, - "triggered_binding": { - "$ref": "#/definitions/coreArtifactBindingData" - }, - "input_binding": { - "$ref": "#/definitions/coreInputBindingData" - } - } - }, - "coreLiteral": { - "type": "object", - "properties": { - "scalar": { - "$ref": "#/definitions/coreScalar", - "description": "A simple value." - }, - "collection": { - "$ref": "#/definitions/coreLiteralCollection", - "description": "A collection of literals to allow nesting." - }, - "map": { - "$ref": "#/definitions/coreLiteralMap", - "description": "A map of strings to literals." - }, - "hash": { - "type": "string", - "title": "A hash representing this literal.\nThis is used for caching purposes. For more details refer to RFC 1893\n(https://github.com/flyteorg/flyte/blob/master/rfc/system/1893-caching-of-offloaded-objects.md)" - }, - "metadata": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Additional metadata for literals." - } - }, - "description": "A simple value. This supports any level of nesting (e.g. array of array of array of Blobs) as well as simple primitives." - }, - "coreLiteralCollection": { - "type": "object", - "properties": { - "literals": { - "type": "array", - "items": { - "$ref": "#/definitions/coreLiteral" - } - } - }, - "description": "A collection of literals. This is a workaround since oneofs in proto messages cannot contain a repeated field." - }, - "coreLiteralMap": { - "type": "object", - "properties": { - "literals": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/coreLiteral" - } - } - }, - "description": "A map of literals. This is a workaround since oneofs in proto messages cannot contain a repeated field." - }, - "coreLiteralType": { - "type": "object", - "properties": { - "simple": { - "$ref": "#/definitions/coreSimpleType", - "description": "A simple type that can be compared one-to-one with another." - }, - "schema": { - "$ref": "#/definitions/coreSchemaType", - "description": "A complex type that requires matching of inner fields." - }, - "collection_type": { - "$ref": "#/definitions/coreLiteralType", - "description": "Defines the type of the value of a collection. Only homogeneous collections are allowed." - }, - "map_value_type": { - "$ref": "#/definitions/coreLiteralType", - "description": "Defines the type of the value of a map type. The type of the key is always a string." - }, - "blob": { - "$ref": "#/definitions/coreBlobType", - "description": "A blob might have specialized implementation details depending on associated metadata." - }, - "enum_type": { - "$ref": "#/definitions/coreEnumType", - "description": "Defines an enum with pre-defined string values." - }, - "structured_dataset_type": { - "$ref": "#/definitions/coreStructuredDatasetType", - "title": "Generalized schema support" - }, - "union_type": { - "$ref": "#/definitions/coreUnionType", - "description": "Defines an union type with pre-defined LiteralTypes." - }, - "metadata": { - "$ref": "#/definitions/protobufStruct", - "description": "This field contains type metadata that is descriptive of the type, but is NOT considered in type-checking. This might be used by\nconsumers to identify special behavior or display extended information for the type." - }, - "annotation": { - "$ref": "#/definitions/coreTypeAnnotation", - "description": "This field contains arbitrary data that might have special semantic\nmeaning for the client but does not effect internal flyte behavior." - }, - "structure": { - "$ref": "#/definitions/coreTypeStructure", - "description": "Hints to improve type matching." - } - }, - "description": "Defines a strong type to allow type checking between interfaces." - }, - "coreOAuth2Client": { - "type": "object", - "properties": { - "client_id": { - "type": "string", - "title": "client_id is the public id for the client to use. The system will not perform any pre-auth validation that the\nsecret requested matches the client_id indicated here.\n+required" - }, - "client_secret": { - "$ref": "#/definitions/coreSecret", - "title": "client_secret is a reference to the secret used to authenticate the OAuth2 client.\n+required" - } - }, - "description": "OAuth2Client encapsulates OAuth2 Client Credentials to be used when making calls on behalf of that task." - }, - "coreOAuth2TokenRequest": { - "type": "object", - "properties": { - "name": { - "type": "string", - "title": "name indicates a unique id for the token request within this task token requests. It'll be used as a suffix for\nenvironment variables and as a filename for mounting tokens as files.\n+required" - }, - "type": { - "$ref": "#/definitions/coreOAuth2TokenRequestType", - "title": "type indicates the type of the request to make. Defaults to CLIENT_CREDENTIALS.\n+required" - }, - "client": { - "$ref": "#/definitions/coreOAuth2Client", - "title": "client references the client_id/secret to use to request the OAuth2 token.\n+required" - }, - "idp_discovery_endpoint": { - "type": "string", - "title": "idp_discovery_endpoint references the discovery endpoint used to retrieve token endpoint and other related\ninformation.\n+optional" - }, - "token_endpoint": { - "type": "string", - "title": "token_endpoint references the token issuance endpoint. If idp_discovery_endpoint is not provided, this parameter is\nmandatory.\n+optional" - } - }, - "description": "OAuth2TokenRequest encapsulates information needed to request an OAuth2 token.\nFLYTE_TOKENS_ENV_PREFIX will be passed to indicate the prefix of the environment variables that will be present if\ntokens are passed through environment variables.\nFLYTE_TOKENS_PATH_PREFIX will be passed to indicate the prefix of the path where secrets will be mounted if tokens\nare passed through file mounts." - }, - "coreOAuth2TokenRequestType": { - "type": "string", - "enum": [ - "CLIENT_CREDENTIALS" - ], - "default": "CLIENT_CREDENTIALS", - "description": "Type of the token requested.\n\n - CLIENT_CREDENTIALS: CLIENT_CREDENTIALS indicates a 2-legged OAuth token requested using client credentials." - }, - "coreParameter": { - "type": "object", - "properties": { - "var": { - "$ref": "#/definitions/coreVariable", - "description": "+required Variable. Defines the type of the variable backing this parameter." - }, - "default": { - "$ref": "#/definitions/coreLiteral", - "description": "Defines a default value that has to match the variable type defined." - }, - "required": { - "type": "boolean", - "format": "boolean", - "description": "+optional, is this value required to be filled." - }, - "artifact_query": { - "$ref": "#/definitions/coreArtifactQuery", - "description": "This is an execution time search basically that should result in exactly one Artifact with a Type that\nmatches the type of the variable." - }, - "artifact_id": { - "$ref": "#/definitions/coreArtifactID" - } - }, - "description": "A parameter is used as input to a launch plan and has\nthe special ability to have a default value or mark itself as required." - }, - "coreParameterMap": { - "type": "object", - "properties": { - "parameters": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/coreParameter" - }, - "description": "Defines a map of parameter names to parameters." - } - }, - "description": "A map of Parameters." - }, - "corePartitions": { - "type": "object", - "properties": { - "value": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/coreLabelValue" - } - } - } - }, - "corePrimitive": { - "type": "object", - "properties": { - "integer": { - "type": "string", - "format": "int64" - }, - "float_value": { - "type": "number", - "format": "double" - }, - "string_value": { - "type": "string" - }, - "boolean": { - "type": "boolean", - "format": "boolean" - }, - "datetime": { - "type": "string", - "format": "date-time" - }, - "duration": { - "type": "string" - } - }, - "title": "Primitive Types" - }, - "coreQualityOfService": { - "type": "object", - "properties": { - "tier": { - "$ref": "#/definitions/QualityOfServiceTier" - }, - "spec": { - "$ref": "#/definitions/coreQualityOfServiceSpec" - } - }, - "description": "Indicates the priority of an execution." - }, - "coreQualityOfServiceSpec": { - "type": "object", - "properties": { - "queueing_budget": { - "type": "string", - "description": "Indicates how much queueing delay an execution can tolerate." - } - }, - "description": "Represents customized execution run-time attributes." - }, - "coreResourceType": { - "type": "string", - "enum": [ - "UNSPECIFIED", - "TASK", - "WORKFLOW", - "LAUNCH_PLAN", - "DATASET" - ], - "default": "UNSPECIFIED", - "description": "Indicates a resource type within Flyte.\n\n - DATASET: A dataset represents an entity modeled in Flyte DataCatalog. A Dataset is also a versioned entity and can be a compilation of multiple individual objects.\nEventually all Catalog objects should be modeled similar to Flyte Objects. The Dataset entities makes it possible for the UI and CLI to act on the objects \nin a similar manner to other Flyte objects" - }, - "coreScalar": { - "type": "object", - "properties": { - "primitive": { - "$ref": "#/definitions/corePrimitive" - }, - "blob": { - "$ref": "#/definitions/coreBlob" - }, - "binary": { - "$ref": "#/definitions/coreBinary" - }, - "schema": { - "$ref": "#/definitions/coreSchema" - }, - "none_type": { - "$ref": "#/definitions/coreVoid" - }, - "error": { - "$ref": "#/definitions/coreError" - }, - "generic": { - "$ref": "#/definitions/protobufStruct" - }, - "structured_dataset": { - "$ref": "#/definitions/coreStructuredDataset" - }, - "union": { - "$ref": "#/definitions/coreUnion" - } - } - }, - "coreSchema": { - "type": "object", - "properties": { - "uri": { - "type": "string" - }, - "type": { - "$ref": "#/definitions/coreSchemaType" - } - }, - "description": "A strongly typed schema that defines the interface of data retrieved from the underlying storage medium." - }, - "coreSchemaType": { - "type": "object", - "properties": { - "columns": { - "type": "array", - "items": { - "$ref": "#/definitions/SchemaTypeSchemaColumn" - }, - "description": "A list of ordered columns this schema comprises of." - } - }, - "description": "Defines schema columns and types to strongly type-validate schemas interoperability." - }, - "coreSecret": { - "type": "object", - "properties": { - "group": { - "type": "string", - "title": "The name of the secret group where to find the key referenced below. For K8s secrets, this should be the name of\nthe v1/secret object. For Confidant, this should be the Credential name. For Vault, this should be the secret name.\nFor AWS Secret Manager, this should be the name of the secret.\n+required" - }, - "group_version": { - "type": "string", - "title": "The group version to fetch. This is not supported in all secret management systems. It'll be ignored for the ones\nthat do not support it.\n+optional" - }, - "key": { - "type": "string", - "title": "The name of the secret to mount. This has to match an existing secret in the system. It's up to the implementation\nof the secret management system to require case sensitivity. For K8s secrets, Confidant and Vault, this should\nmatch one of the keys inside the secret. For AWS Secret Manager, it's ignored.\n+optional" - }, - "mount_requirement": { - "$ref": "#/definitions/SecretMountType", - "title": "mount_requirement is optional. Indicates where the secret has to be mounted. If provided, the execution will fail\nif the underlying key management system cannot satisfy that requirement. If not provided, the default location\nwill depend on the key management system.\n+optional" - } - }, - "description": "Secret encapsulates information about the secret a task needs to proceed. An environment variable\nFLYTE_SECRETS_ENV_PREFIX will be passed to indicate the prefix of the environment variables that will be present if\nsecrets are passed through environment variables.\nFLYTE_SECRETS_DEFAULT_DIR will be passed to indicate the prefix of the path where secrets will be mounted if secrets\nare passed through file mounts." - }, - "coreSecurityContext": { - "type": "object", - "properties": { - "run_as": { - "$ref": "#/definitions/coreIdentity", - "description": "run_as encapsulates the identity a pod should run as. If the task fills in multiple fields here, it'll be up to the\nbackend plugin to choose the appropriate identity for the execution engine the task will run on." - }, - "secrets": { - "type": "array", - "items": { - "$ref": "#/definitions/coreSecret" - }, - "description": "secrets indicate the list of secrets the task needs in order to proceed. Secrets will be mounted/passed to the\npod as it starts. If the plugin responsible for kicking of the task will not run it on a flyte cluster (e.g. AWS\nBatch), it's the responsibility of the plugin to fetch the secret (which means propeller identity will need access\nto the secret) and to pass it to the remote execution engine." - }, - "tokens": { - "type": "array", - "items": { - "$ref": "#/definitions/coreOAuth2TokenRequest" - }, - "description": "tokens indicate the list of token requests the task needs in order to proceed. Tokens will be mounted/passed to the\npod as it starts. If the plugin responsible for kicking of the task will not run it on a flyte cluster (e.g. AWS\nBatch), it's the responsibility of the plugin to fetch the secret (which means propeller identity will need access\nto the secret) and to pass it to the remote execution engine." - } - }, - "description": "SecurityContext holds security attributes that apply to tasks." - }, - "coreSimpleType": { - "type": "string", - "enum": [ - "NONE", - "INTEGER", - "FLOAT", - "STRING", - "BOOLEAN", - "DATETIME", - "DURATION", - "BINARY", - "ERROR", - "STRUCT" - ], - "default": "NONE", - "description": "Define a set of simple types." - }, - "coreStructuredDataset": { - "type": "object", - "properties": { - "uri": { - "type": "string", - "title": "String location uniquely identifying where the data is.\nShould start with the storage location (e.g. s3://, gs://, bq://, etc.)" - }, - "metadata": { - "$ref": "#/definitions/coreStructuredDatasetMetadata" - } - } - }, - "coreStructuredDatasetMetadata": { - "type": "object", - "properties": { - "structured_dataset_type": { - "$ref": "#/definitions/coreStructuredDatasetType", - "description": "Bundle the type information along with the literal.\nThis is here because StructuredDatasets can often be more defined at run time than at compile time.\nThat is, at compile time you might only declare a task to return a pandas dataframe or a StructuredDataset,\nwithout any column information, but at run time, you might have that column information.\nflytekit python will copy this type information into the literal, from the type information, if not provided by\nthe various plugins (encoders).\nSince this field is run time generated, it's not used for any type checking." - } - } - }, - "coreStructuredDatasetType": { - "type": "object", - "properties": { - "columns": { - "type": "array", - "items": { - "$ref": "#/definitions/StructuredDatasetTypeDatasetColumn" - }, - "description": "A list of ordered columns this schema comprises of." - }, - "format": { - "type": "string", - "description": "This is the storage format, the format of the bits at rest\nparquet, feather, csv, etc.\nFor two types to be compatible, the format will need to be an exact match." - }, - "external_schema_type": { - "type": "string", - "description": "This is a string representing the type that the bytes in external_schema_bytes are formatted in.\nThis is an optional field that will not be used for type checking." - }, - "external_schema_bytes": { - "type": "string", - "format": "byte", - "description": "The serialized bytes of a third-party schema library like Arrow.\nThis is an optional field that will not be used for type checking." - } - } - }, - "coreTimePartition": { - "type": "object", - "properties": { - "value": { - "$ref": "#/definitions/coreLabelValue" - } - } - }, - "coreTypeAnnotation": { - "type": "object", - "properties": { - "annotations": { - "$ref": "#/definitions/protobufStruct", - "description": "A arbitrary JSON payload to describe a type." - } - }, - "description": "TypeAnnotation encapsulates registration time information about a type. This can be used for various control-plane operations. TypeAnnotation will not be available at runtime when a task runs." - }, - "coreTypeStructure": { - "type": "object", - "properties": { - "tag": { - "type": "string", - "title": "Must exactly match for types to be castable" - }, - "dataclass_type": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/coreLiteralType" - }, - "title": "dataclass_type only exists for dataclasses.\nThis is used to resolve the type of the fields of dataclass\nThe key is the field name, and the value is the literal type of the field\ne.g. For dataclass Foo, with fields a, and a is a string\nFoo.a will be resolved as a literal type of string from dataclass_type" - } - }, - "description": "Hints to improve type matching\ne.g. allows distinguishing output from custom type transformers\neven if the underlying IDL serialization matches." - }, - "coreUnion": { - "type": "object", - "properties": { - "value": { - "$ref": "#/definitions/coreLiteral" - }, - "type": { - "$ref": "#/definitions/coreLiteralType" - } - }, - "description": "The runtime representation of a tagged union value. See `UnionType` for more details." - }, - "coreUnionType": { - "type": "object", - "properties": { - "variants": { - "type": "array", - "items": { - "$ref": "#/definitions/coreLiteralType" - }, - "description": "Predefined set of variants in union." - } - }, - "description": "Defines a tagged union type, also known as a variant (and formally as the sum type).\n\nA sum type S is defined by a sequence of types (A, B, C, ...), each tagged by a string tag\nA value of type S is constructed from a value of any of the variant types. The specific choice of type is recorded by\nstoring the varaint's tag with the literal value and can be examined in runtime.\n\nType S is typically written as\nS := Apple A | Banana B | Cantaloupe C | ...\n\nNotably, a nullable (optional) type is a sum type between some type X and the singleton type representing a null-value:\nOptional X := X | Null\n\nSee also: https://en.wikipedia.org/wiki/Tagged_union" - }, - "coreVariable": { - "type": "object", - "properties": { - "type": { - "$ref": "#/definitions/coreLiteralType", - "description": "Variable literal type." - }, - "description": { - "type": "string", - "title": "+optional string describing input variable" - }, - "artifact_partial_id": { - "$ref": "#/definitions/coreArtifactID", - "description": "+optional This object allows the user to specify how Artifacts are created.\nname, tag, partitions can be specified. The other fields (version and project/domain) are ignored." - }, - "artifact_tag": { - "$ref": "#/definitions/coreArtifactTag" - } - }, - "description": "Defines a strongly typed variable." - }, - "coreVariableMap": { - "type": "object", - "properties": { - "variables": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/coreVariable" - }, - "description": "Defines a map of variable names to variables." - } - }, - "title": "A map of Variables" - }, - "coreVoid": { - "type": "object", - "description": "Used to denote a nil/null/None assignment to a scalar value. The underlying LiteralType for Void is intentionally\nundefined since it can be assigned to a scalar of any LiteralType." - }, - "coreWorkflowExecutionIdentifier": { - "type": "object", - "properties": { - "project": { - "type": "string", - "description": "Name of the project the resource belongs to." - }, - "domain": { - "type": "string", - "description": "Name of the domain the resource belongs to.\nA domain can be considered as a subset within a specific project." - }, - "name": { - "type": "string", - "description": "User or system provided value for the resource." - }, - "org": { - "type": "string", - "description": "Optional, org key applied to the resource." - } - }, - "title": "Encapsulation of fields that uniquely identifies a Flyte workflow execution" - }, - "coreWorkflowExecutionPhase": { - "type": "string", - "enum": [ - "UNDEFINED", - "QUEUED", - "RUNNING", - "SUCCEEDING", - "SUCCEEDED", - "FAILING", - "FAILED", - "ABORTED", - "TIMED_OUT", - "ABORTING" - ], - "default": "UNDEFINED" - }, - "protobufAny": { - "type": "object", - "properties": { - "type_url": { - "type": "string", - "description": "A URL/resource name that uniquely identifies the type of the serialized\nprotocol buffer message. This string must contain at least\none \"/\" character. The last segment of the URL's path must represent\nthe fully qualified name of the type (as in\n`path/google.protobuf.Duration`). The name should be in a canonical form\n(e.g., leading \".\" is not accepted).\n\nIn practice, teams usually precompile into the binary all types that they\nexpect it to use in the context of Any. However, for URLs which use the\nscheme `http`, `https`, or no scheme, one can optionally set up a type\nserver that maps type URLs to message definitions as follows:\n\n* If no scheme is provided, `https` is assumed.\n* An HTTP GET on the URL must yield a [google.protobuf.Type][]\n value in binary format, or produce an error.\n* Applications are allowed to cache lookup results based on the\n URL, or have them precompiled into a binary to avoid any\n lookup. Therefore, binary compatibility needs to be preserved\n on changes to types. (Use versioned type names to manage\n breaking changes.)\n\nNote: this functionality is not currently available in the official\nprotobuf release, and it is not used for type URLs beginning with\ntype.googleapis.com.\n\nSchemes other than `http`, `https` (or the empty scheme) might be\nused with implementation specific semantics." - }, - "value": { - "type": "string", - "format": "byte", - "description": "Must be a valid serialized protocol buffer of the above specified type." - } - }, - "description": "`Any` contains an arbitrary serialized protocol buffer message along with a\nURL that describes the type of the serialized message.\n\nProtobuf library provides support to pack/unpack Any values in the form\nof utility functions or additional generated methods of the Any type.\n\nExample 1: Pack and unpack a message in C++.\n\n Foo foo = ...;\n Any any;\n any.PackFrom(foo);\n ...\n if (any.UnpackTo(\u0026foo)) {\n ...\n }\n\nExample 2: Pack and unpack a message in Java.\n\n Foo foo = ...;\n Any any = Any.pack(foo);\n ...\n if (any.is(Foo.class)) {\n foo = any.unpack(Foo.class);\n }\n\n Example 3: Pack and unpack a message in Python.\n\n foo = Foo(...)\n any = Any()\n any.Pack(foo)\n ...\n if any.Is(Foo.DESCRIPTOR):\n any.Unpack(foo)\n ...\n\n Example 4: Pack and unpack a message in Go\n\n foo := \u0026pb.Foo{...}\n any, err := ptypes.MarshalAny(foo)\n ...\n foo := \u0026pb.Foo{}\n if err := ptypes.UnmarshalAny(any, foo); err != nil {\n ...\n }\n\nThe pack methods provided by protobuf library will by default use\n'type.googleapis.com/full.type.name' as the type URL and the unpack\nmethods only use the fully qualified type name after the last '/'\nin the type URL, for example \"foo.bar.com/x/y.z\" will yield type\nname \"y.z\".\n\n\nJSON\n====\nThe JSON representation of an `Any` value uses the regular\nrepresentation of the deserialized, embedded message, with an\nadditional field `@type` which contains the type URL. Example:\n\n package google.profile;\n message Person {\n string first_name = 1;\n string last_name = 2;\n }\n\n {\n \"@type\": \"type.googleapis.com/google.profile.Person\",\n \"firstName\": \u003cstring\u003e,\n \"lastName\": \u003cstring\u003e\n }\n\nIf the embedded message type is well-known and has a custom JSON\nrepresentation, that representation will be embedded adding a field\n`value` which holds the custom JSON in addition to the `@type`\nfield. Example (for message [google.protobuf.Duration][]):\n\n {\n \"@type\": \"type.googleapis.com/google.protobuf.Duration\",\n \"value\": \"1.212s\"\n }" - }, - "protobufListValue": { - "type": "object", - "properties": { - "values": { - "type": "array", - "items": { - "$ref": "#/definitions/protobufValue" - }, - "description": "Repeated field of dynamically typed values." - } - }, - "description": "`ListValue` is a wrapper around a repeated field of values.\n\nThe JSON representation for `ListValue` is JSON array." - }, - "protobufNullValue": { - "type": "string", - "enum": [ - "NULL_VALUE" - ], - "default": "NULL_VALUE", - "description": "`NullValue` is a singleton enumeration to represent the null value for the\n`Value` type union.\n\n The JSON representation for `NullValue` is JSON `null`.\n\n - NULL_VALUE: Null value." - }, - "protobufStruct": { - "type": "object", - "properties": { - "fields": { - "type": "object", - "additionalProperties": { - "$ref": "#/definitions/protobufValue" - }, - "description": "Unordered map of dynamically typed values." - } - }, - "description": "`Struct` represents a structured data value, consisting of fields\nwhich map to dynamically typed values. In some languages, `Struct`\nmight be supported by a native representation. For example, in\nscripting languages like JS a struct is represented as an\nobject. The details of that representation are described together\nwith the proto support for the language.\n\nThe JSON representation for `Struct` is JSON object." - }, - "protobufValue": { - "type": "object", - "properties": { - "null_value": { - "$ref": "#/definitions/protobufNullValue", - "description": "Represents a null value." - }, - "number_value": { - "type": "number", - "format": "double", - "description": "Represents a double value." - }, - "string_value": { - "type": "string", - "description": "Represents a string value." - }, - "bool_value": { - "type": "boolean", - "format": "boolean", - "description": "Represents a boolean value." - }, - "struct_value": { - "$ref": "#/definitions/protobufStruct", - "description": "Represents a structured value." - }, - "list_value": { - "$ref": "#/definitions/protobufListValue", - "description": "Represents a repeated `Value`." - } - }, - "description": "`Value` represents a dynamically typed value which can be either\nnull, a number, a string, a boolean, a recursive struct value, or a\nlist of values. A producer of value is expected to set one of that\nvariants, absence of any variant indicates an error.\n\nThe JSON representation for `Value` is JSON value." - } - } -} diff --git a/flyteidl/gen/pb-go/flyteidl/core/artifact_id.pb.go b/flyteidl/gen/pb-go/flyteidl/core/artifact_id.pb.go index 927deec730..ec28336db7 100644 --- a/flyteidl/gen/pb-go/flyteidl/core/artifact_id.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/core/artifact_id.pb.go @@ -632,56 +632,6 @@ func (*ArtifactQuery) XXX_OneofWrappers() []interface{} { } } -type Trigger struct { - // This will be set to a launch plan type, but note that this is different than the actual launch plan type. - TriggerId *Identifier `protobuf:"bytes,1,opt,name=trigger_id,json=triggerId,proto3" json:"trigger_id,omitempty"` - // These are partial artifact IDs that will be triggered on - // Consider making these ArtifactQuery instead. - Triggers []*ArtifactID `protobuf:"bytes,2,rep,name=triggers,proto3" json:"triggers,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Trigger) Reset() { *m = Trigger{} } -func (m *Trigger) String() string { return proto.CompactTextString(m) } -func (*Trigger) ProtoMessage() {} -func (*Trigger) Descriptor() ([]byte, []int) { - return fileDescriptor_1079b0707e23f978, []int{9} -} - -func (m *Trigger) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Trigger.Unmarshal(m, b) -} -func (m *Trigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Trigger.Marshal(b, m, deterministic) -} -func (m *Trigger) XXX_Merge(src proto.Message) { - xxx_messageInfo_Trigger.Merge(m, src) -} -func (m *Trigger) XXX_Size() int { - return xxx_messageInfo_Trigger.Size(m) -} -func (m *Trigger) XXX_DiscardUnknown() { - xxx_messageInfo_Trigger.DiscardUnknown(m) -} - -var xxx_messageInfo_Trigger proto.InternalMessageInfo - -func (m *Trigger) GetTriggerId() *Identifier { - if m != nil { - return m.TriggerId - } - return nil -} - -func (m *Trigger) GetTriggers() []*ArtifactID { - if m != nil { - return m.Triggers - } - return nil -} - func init() { proto.RegisterType((*ArtifactKey)(nil), "flyteidl.core.ArtifactKey") proto.RegisterType((*ArtifactBindingData)(nil), "flyteidl.core.ArtifactBindingData") @@ -693,57 +643,53 @@ func init() { proto.RegisterType((*ArtifactID)(nil), "flyteidl.core.ArtifactID") proto.RegisterType((*ArtifactTag)(nil), "flyteidl.core.ArtifactTag") proto.RegisterType((*ArtifactQuery)(nil), "flyteidl.core.ArtifactQuery") - proto.RegisterType((*Trigger)(nil), "flyteidl.core.Trigger") } func init() { proto.RegisterFile("flyteidl/core/artifact_id.proto", fileDescriptor_1079b0707e23f978) } var fileDescriptor_1079b0707e23f978 = []byte{ - // 721 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0xcd, 0x6e, 0xd3, 0x40, - 0x10, 0x8e, 0x93, 0xb6, 0x69, 0x27, 0x49, 0x15, 0xb6, 0xa8, 0x4a, 0xa3, 0x8a, 0x56, 0xa6, 0x48, - 0xbd, 0x60, 0x4b, 0x45, 0x95, 0x4a, 0x29, 0x7f, 0xa1, 0xa0, 0x44, 0xe5, 0x40, 0xb7, 0x11, 0x07, - 0x2e, 0xd1, 0x26, 0xde, 0x98, 0xa5, 0xb1, 0x37, 0x5a, 0x6f, 0x2a, 0x8c, 0xc4, 0x4b, 0x70, 0xe7, - 0x41, 0x78, 0x20, 0xee, 0x3c, 0x02, 0xda, 0x5d, 0xff, 0xc4, 0x51, 0x23, 0x15, 0x71, 0xf3, 0xcc, - 0xce, 0x7c, 0x3b, 0xf3, 0x7d, 0x3b, 0x63, 0xd8, 0x1b, 0x4f, 0x62, 0x49, 0x99, 0x37, 0x71, 0x47, - 0x5c, 0x50, 0x97, 0x08, 0xc9, 0xc6, 0x64, 0x24, 0x07, 0xcc, 0x73, 0xa6, 0x82, 0x4b, 0x8e, 0x1a, - 0x69, 0x80, 0xa3, 0x02, 0xda, 0x7b, 0x3e, 0xe7, 0xfe, 0x84, 0xba, 0xfa, 0x70, 0x38, 0x1b, 0xbb, - 0x92, 0x05, 0x34, 0x92, 0x24, 0x98, 0x9a, 0xf8, 0xf6, 0x83, 0x22, 0x20, 0xf3, 0x68, 0x28, 0xd9, - 0x98, 0x51, 0x61, 0xce, 0xed, 0x2b, 0xa8, 0xbd, 0x4e, 0x2e, 0xb9, 0xa0, 0x31, 0x6a, 0x41, 0x75, - 0x2a, 0xf8, 0x17, 0x3a, 0x92, 0x2d, 0x6b, 0xdf, 0x3a, 0xdc, 0xc0, 0xa9, 0x89, 0xb6, 0x61, 0xcd, - 0xe3, 0x01, 0x61, 0x61, 0xab, 0xac, 0x0f, 0x12, 0x0b, 0x21, 0x58, 0x09, 0x49, 0x40, 0x5b, 0x15, - 0xed, 0xd5, 0xdf, 0xf6, 0x2f, 0x0b, 0xb6, 0x52, 0xd4, 0x0e, 0x0b, 0x3d, 0x16, 0xfa, 0xe7, 0x44, - 0x12, 0x74, 0x1f, 0x56, 0x59, 0xe8, 0xd1, 0xaf, 0x1a, 0xbb, 0x81, 0x8d, 0x81, 0x1e, 0x41, 0x63, - 0xaa, 0x1a, 0x95, 0x8c, 0x87, 0x83, 0x6b, 0x1a, 0x9b, 0x0b, 0xba, 0x25, 0x5c, 0xcf, 0xdc, 0xaa, - 0xb4, 0x63, 0xd8, 0x1e, 0xb2, 0xd0, 0x1b, 0x48, 0x3e, 0x50, 0x4d, 0x0e, 0xb2, 0x43, 0x7d, 0xf5, - 0x7a, 0xb7, 0x84, 0xb7, 0xd4, 0x79, 0x9f, 0xf7, 0x59, 0x40, 0x3f, 0xa4, 0x87, 0x68, 0x17, 0x36, - 0xa4, 0x20, 0x61, 0x34, 0xe6, 0x22, 0x68, 0xad, 0xe8, 0x22, 0x73, 0x47, 0xa7, 0x09, 0x9b, 0xf9, - 0xdd, 0x1e, 0x91, 0xc4, 0x3e, 0x80, 0x66, 0x2f, 0x9c, 0xce, 0x0a, 0x75, 0x37, 0xa1, 0x72, 0x43, - 0x44, 0xc2, 0x88, 0xfa, 0xb4, 0x7f, 0x94, 0x01, 0xde, 0x93, 0x21, 0x9d, 0x7c, 0x24, 0x93, 0x19, - 0x45, 0x0f, 0xa1, 0x1e, 0x49, 0x22, 0xd9, 0x68, 0x70, 0xa3, 0x6c, 0x13, 0xd9, 0x2d, 0xe1, 0x9a, - 0xf1, 0x9a, 0xa0, 0x67, 0x00, 0xba, 0x70, 0x13, 0xa2, 0x9a, 0xac, 0x1d, 0xb5, 0x1d, 0x23, 0xa0, - 0x93, 0x0a, 0xe8, 0xf4, 0x53, 0x01, 0xbb, 0x25, 0xbc, 0xa1, 0xe2, 0x4d, 0xf2, 0x25, 0xdc, 0x93, - 0x82, 0xf9, 0x3e, 0x15, 0xd4, 0x1b, 0x0c, 0x4d, 0x6d, 0xba, 0xf1, 0xda, 0x91, 0xed, 0x14, 0xde, - 0x84, 0x73, 0x0b, 0xf3, 0xdd, 0x12, 0x6e, 0x66, 0xe9, 0x89, 0x1f, 0xbd, 0x83, 0x06, 0x53, 0x9d, - 0x66, 0x70, 0x2b, 0x1a, 0x6e, 0x6f, 0x01, 0x6e, 0x91, 0x0d, 0x25, 0x0c, 0x9b, 0xf3, 0x75, 0xaa, - 0xb0, 0xaa, 0x5b, 0xb2, 0x7f, 0x5a, 0x00, 0x19, 0xf1, 0x11, 0x3a, 0x4d, 0xfc, 0x2d, 0x6b, 0xbf, - 0x72, 0x58, 0x3b, 0x3a, 0x58, 0xc0, 0xcd, 0x23, 0x1d, 0xdd, 0xe2, 0xdb, 0x50, 0x8a, 0x18, 0x9b, - 0x94, 0xf6, 0x15, 0x40, 0xee, 0x54, 0xfc, 0xab, 0x77, 0x91, 0xf0, 0x7f, 0x4d, 0x63, 0xe4, 0xa6, - 0xd8, 0x86, 0xc6, 0x9d, 0x05, 0xec, 0x5c, 0x9a, 0x04, 0xf0, 0xb4, 0x7c, 0x62, 0xd9, 0xaf, 0xa0, - 0x51, 0x7c, 0x1b, 0x6e, 0x5e, 0xe1, 0x9d, 0x50, 0xec, 0xdf, 0x16, 0x40, 0x4a, 0x6f, 0xef, 0x1c, - 0x3d, 0x87, 0x7a, 0x36, 0xa1, 0x69, 0x81, 0x4a, 0xd3, 0xdb, 0xf5, 0xb8, 0xa0, 0x31, 0xae, 0x91, - 0xe2, 0xb0, 0xdd, 0x50, 0x11, 0xa9, 0x27, 0x6c, 0x66, 0x2a, 0x35, 0xd1, 0x53, 0x80, 0xec, 0x59, - 0x46, 0x89, 0xcc, 0x3b, 0x4b, 0xf9, 0xc3, 0x73, 0xc1, 0xe8, 0x0d, 0x6c, 0x2e, 0x8c, 0x87, 0x91, - 0x75, 0x77, 0x21, 0xbd, 0xc0, 0x04, 0x6e, 0xc8, 0x79, 0xd3, 0xfe, 0x9e, 0x6f, 0x85, 0x3e, 0xf1, - 0xff, 0xb7, 0xcf, 0x7f, 0x15, 0xcb, 0xfe, 0x63, 0x41, 0x23, 0x45, 0xbb, 0x9c, 0x51, 0x11, 0xa3, - 0x33, 0xa8, 0xcd, 0xed, 0xc2, 0x25, 0x7a, 0xe5, 0xca, 0x74, 0x4b, 0x18, 0xd2, 0xf8, 0x9e, 0x87, - 0x5e, 0xce, 0xd5, 0x2f, 0x89, 0x9f, 0xcd, 0xde, 0xed, 0xe9, 0x7d, 0xe2, 0xab, 0xd1, 0x25, 0x73, - 0x04, 0x20, 0xa8, 0xcc, 0x04, 0x33, 0x3b, 0xae, 0x5b, 0xc2, 0xca, 0x40, 0x2f, 0xa0, 0x5a, 0x1c, - 0x9c, 0xbb, 0xcd, 0x61, 0x9a, 0xd4, 0xa9, 0x03, 0xe4, 0xdb, 0xd8, 0xfe, 0x06, 0xd5, 0xbe, 0x19, - 0x50, 0x74, 0x02, 0x90, 0xcc, 0xea, 0xf2, 0x56, 0x7b, 0x59, 0xa6, 0xda, 0x66, 0x3a, 0xb8, 0xe7, - 0xa1, 0x63, 0x58, 0x4f, 0x8c, 0xa8, 0x55, 0xd6, 0x43, 0xb7, 0x9c, 0x22, 0x9c, 0x85, 0x76, 0xce, - 0x3e, 0x9d, 0xfa, 0x4c, 0x7e, 0x9e, 0x0d, 0x9d, 0x11, 0x0f, 0x5c, 0x9d, 0xc0, 0x85, 0x6f, 0x3e, - 0xdc, 0xec, 0xff, 0xe1, 0xd3, 0xd0, 0x9d, 0x0e, 0x1f, 0xfb, 0xdc, 0x2d, 0xfc, 0x52, 0x86, 0x6b, - 0x7a, 0x75, 0x3d, 0xf9, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x35, 0x2f, 0x5a, 0x3b, 0xbb, 0x06, 0x00, - 0x00, + // 683 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xcd, 0x6e, 0xd3, 0x40, + 0x10, 0xb6, 0x9b, 0xfe, 0xd0, 0x71, 0x5c, 0x85, 0x2d, 0xaa, 0xd2, 0xa8, 0xa2, 0x95, 0x29, 0x52, + 0x2f, 0xd8, 0x52, 0x11, 0x12, 0x94, 0xf2, 0x17, 0x0a, 0x72, 0x55, 0x0e, 0xd4, 0x8d, 0x38, 0x70, + 0xb1, 0xd6, 0xf1, 0xc6, 0x2c, 0x8d, 0xbd, 0xd1, 0x66, 0x53, 0xe1, 0x03, 0x2f, 0xc1, 0x9d, 0x07, + 0xe1, 0x81, 0xb8, 0xf3, 0x08, 0x68, 0x77, 0xfd, 0x13, 0x47, 0xad, 0x54, 0xc4, 0x6d, 0x67, 0x76, + 0xe6, 0xdb, 0x99, 0xef, 0xdb, 0x19, 0xd8, 0x1d, 0x8d, 0x73, 0x41, 0x68, 0x3c, 0xf6, 0x86, 0x8c, + 0x13, 0x0f, 0x73, 0x41, 0x47, 0x78, 0x28, 0x42, 0x1a, 0xbb, 0x13, 0xce, 0x04, 0x43, 0x76, 0x19, + 0xe0, 0xca, 0x80, 0xde, 0x6e, 0xc2, 0x58, 0x32, 0x26, 0x9e, 0xba, 0x8c, 0x66, 0x23, 0x4f, 0xd0, + 0x94, 0x4c, 0x05, 0x4e, 0x27, 0x3a, 0xbe, 0x77, 0xbf, 0x09, 0x48, 0x63, 0x92, 0x09, 0x3a, 0xa2, + 0x84, 0xeb, 0x7b, 0xe7, 0x02, 0xac, 0x37, 0xc5, 0x23, 0x67, 0x24, 0x47, 0x5d, 0x58, 0x9b, 0x70, + 0xf6, 0x95, 0x0c, 0x45, 0xd7, 0xdc, 0x33, 0x0f, 0xd6, 0x83, 0xd2, 0x44, 0x5b, 0xb0, 0x1a, 0xb3, + 0x14, 0xd3, 0xac, 0xbb, 0xa4, 0x2e, 0x0a, 0x0b, 0x21, 0x58, 0xce, 0x70, 0x4a, 0xba, 0x2d, 0xe5, + 0x55, 0x67, 0xe7, 0x97, 0x09, 0x9b, 0x25, 0x6a, 0x9f, 0x66, 0x31, 0xcd, 0x92, 0x13, 0x2c, 0x30, + 0xba, 0x07, 0x2b, 0x34, 0x8b, 0xc9, 0x37, 0x85, 0x6d, 0x07, 0xda, 0x40, 0x0f, 0xc1, 0x9e, 0xc8, + 0x46, 0x05, 0x65, 0x59, 0x78, 0x49, 0x72, 0xfd, 0x80, 0x6f, 0x04, 0xed, 0xca, 0x2d, 0x4b, 0x7b, + 0x02, 0x5b, 0x11, 0xcd, 0xe2, 0x50, 0xb0, 0x50, 0x36, 0x19, 0x56, 0x97, 0xea, 0xe9, 0x3b, 0xbe, + 0x11, 0x6c, 0xca, 0xfb, 0x01, 0x1b, 0xd0, 0x94, 0x7c, 0x2c, 0x2f, 0xd1, 0x0e, 0xac, 0x0b, 0x8e, + 0xb3, 0xe9, 0x88, 0xf1, 0xb4, 0xbb, 0xac, 0x8a, 0xac, 0x1d, 0xfd, 0x0e, 0x6c, 0xd4, 0x6f, 0xc7, + 0x58, 0x60, 0x67, 0x1f, 0x3a, 0xa7, 0xd9, 0x64, 0xd6, 0xa8, 0xbb, 0x03, 0xad, 0x2b, 0xcc, 0x0b, + 0x46, 0xe4, 0xd1, 0xf9, 0xb1, 0x04, 0xf0, 0x01, 0x47, 0x64, 0xfc, 0x09, 0x8f, 0x67, 0x04, 0x3d, + 0x80, 0xf6, 0x54, 0x60, 0x41, 0x87, 0xe1, 0x95, 0xb4, 0x75, 0xa4, 0x6f, 0x04, 0x96, 0xf6, 0xea, + 0xa0, 0xe7, 0x00, 0xaa, 0x70, 0x1d, 0x22, 0x9b, 0xb4, 0x0e, 0x7b, 0xae, 0x16, 0xd0, 0x2d, 0x05, + 0x74, 0x07, 0xa5, 0x80, 0xbe, 0x11, 0xac, 0xcb, 0x78, 0x9d, 0x7c, 0x0e, 0x77, 0x05, 0xa7, 0x49, + 0x42, 0x38, 0x89, 0xc3, 0x48, 0xd7, 0xa6, 0x1a, 0xb7, 0x0e, 0x1d, 0xb7, 0xf1, 0x27, 0xdc, 0x6b, + 0x98, 0xf7, 0x8d, 0xa0, 0x53, 0xa5, 0x17, 0x7e, 0xf4, 0x1e, 0x6c, 0x2a, 0x3b, 0xad, 0xe0, 0x96, + 0x15, 0xdc, 0xee, 0x02, 0xdc, 0x22, 0x1b, 0x52, 0x18, 0x3a, 0xe7, 0xeb, 0xaf, 0xc1, 0x8a, 0x6a, + 0xc9, 0xf9, 0x69, 0x02, 0x54, 0xc4, 0x4f, 0xd1, 0x51, 0xe1, 0xef, 0x9a, 0x7b, 0xad, 0x03, 0xeb, + 0x70, 0x7f, 0x01, 0xb7, 0x8e, 0x74, 0x55, 0x8b, 0xef, 0x32, 0xc1, 0xf3, 0x40, 0xa7, 0xf4, 0x2e, + 0x00, 0x6a, 0xa7, 0xe4, 0x5f, 0xfe, 0x8b, 0x82, 0xff, 0x4b, 0x92, 0x23, 0xaf, 0xc4, 0xd6, 0x34, + 0x6e, 0x2f, 0x60, 0xd7, 0xd2, 0x14, 0x80, 0x47, 0x4b, 0x4f, 0x4d, 0xe7, 0x35, 0xd8, 0xcd, 0xbf, + 0xe1, 0xd5, 0x15, 0xde, 0x0a, 0xc5, 0xf9, 0x6d, 0x02, 0x94, 0xf4, 0x9e, 0x9e, 0xa0, 0x17, 0xd0, + 0xae, 0x26, 0xb4, 0x2c, 0x50, 0x6a, 0x7a, 0xbd, 0x1e, 0x67, 0x24, 0x0f, 0x2c, 0xdc, 0x1c, 0xb6, + 0x2b, 0xc2, 0xa7, 0xf2, 0x0b, 0xeb, 0x99, 0x2a, 0x4d, 0xf4, 0x0c, 0xa0, 0xfa, 0x96, 0xd3, 0x42, + 0xe6, 0xed, 0x1b, 0xf9, 0x0b, 0xe6, 0x82, 0xd1, 0x5b, 0xd8, 0x58, 0x18, 0x0f, 0x2d, 0xeb, 0xce, + 0x42, 0x7a, 0x83, 0x89, 0xc0, 0x16, 0xf3, 0xa6, 0xf3, 0xbd, 0xde, 0x0a, 0x03, 0x9c, 0xfc, 0x6f, + 0x9f, 0xff, 0x2a, 0x96, 0xf3, 0xc7, 0x04, 0xbb, 0x44, 0x3b, 0x9f, 0x11, 0x9e, 0xa3, 0x63, 0xb0, + 0xe6, 0x76, 0xe1, 0x0d, 0x7a, 0xd5, 0xca, 0xf8, 0x46, 0x00, 0x65, 0xfc, 0x69, 0x8c, 0x5e, 0xcd, + 0xd5, 0x2f, 0x70, 0x52, 0xcd, 0xde, 0xf5, 0xe9, 0x03, 0x9c, 0xc8, 0xd1, 0xc5, 0x73, 0x04, 0x20, + 0x68, 0xcd, 0x38, 0xd5, 0x3b, 0xce, 0x37, 0x02, 0x69, 0xa0, 0x97, 0xb0, 0xd6, 0x1c, 0x9c, 0xdb, + 0xcd, 0x61, 0x99, 0xd4, 0x6f, 0x03, 0xd4, 0xdb, 0xb8, 0x7f, 0xfc, 0xf9, 0x28, 0xa1, 0xe2, 0xcb, + 0x2c, 0x72, 0x87, 0x2c, 0xf5, 0x14, 0x10, 0xe3, 0x89, 0x3e, 0x78, 0xd5, 0x0e, 0x4f, 0x48, 0xe6, + 0x4d, 0xa2, 0x47, 0x09, 0xf3, 0x1a, 0x6b, 0x3d, 0x5a, 0x55, 0xeb, 0xe3, 0xf1, 0xdf, 0x00, 0x00, + 0x00, 0xff, 0xff, 0xdf, 0x09, 0x3c, 0xc7, 0x3f, 0x06, 0x00, 0x00, } diff --git a/flyteidl/gen/pb-java/flyteidl/artifact/Artifacts.java b/flyteidl/gen/pb-java/flyteidl/artifact/Artifacts.java deleted file mode 100644 index 96cb8d9edd..0000000000 --- a/flyteidl/gen/pb-java/flyteidl/artifact/Artifacts.java +++ /dev/null @@ -1,22252 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: flyteidl/artifact/artifacts.proto - -package flyteidl.artifact; - -public final class Artifacts { - private Artifacts() {} - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { - } - - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); - } - public interface ArtifactOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.Artifact) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - boolean hasArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactID getArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder(); - - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - boolean hasSpec(); - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - flyteidl.artifact.Artifacts.ArtifactSpec getSpec(); - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder(); - - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - java.util.List - getTagsList(); - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - int getTagsCount(); - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - java.lang.String getTags(int index); - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - com.google.protobuf.ByteString - getTagsBytes(int index); - - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - boolean hasSource(); - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - flyteidl.artifact.Artifacts.ArtifactSource getSource(); - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.Artifact} - */ - public static final class Artifact extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.Artifact) - ArtifactOrBuilder { - private static final long serialVersionUID = 0L; - // Use Artifact.newBuilder() to construct. - private Artifact(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private Artifact() { - tags_ = com.google.protobuf.LazyStringArrayList.EMPTY; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private Artifact( - 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.ArtifactId.ArtifactID.Builder subBuilder = null; - if (artifactId_ != null) { - subBuilder = artifactId_.toBuilder(); - } - artifactId_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactID.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifactId_); - artifactId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.artifact.Artifacts.ArtifactSpec.Builder subBuilder = null; - if (spec_ != null) { - subBuilder = spec_.toBuilder(); - } - spec_ = input.readMessage(flyteidl.artifact.Artifacts.ArtifactSpec.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(spec_); - spec_ = subBuilder.buildPartial(); - } - - break; - } - case 26: { - java.lang.String s = input.readStringRequireUtf8(); - if (!((mutable_bitField0_ & 0x00000004) != 0)) { - tags_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000004; - } - tags_.add(s); - break; - } - case 34: { - flyteidl.artifact.Artifacts.ArtifactSource.Builder subBuilder = null; - if (source_ != null) { - subBuilder = source_.toBuilder(); - } - source_ = input.readMessage(flyteidl.artifact.Artifacts.ArtifactSource.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(source_); - source_ = 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 { - if (((mutable_bitField0_ & 0x00000004) != 0)) { - tags_ = tags_.getUnmodifiableView(); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_Artifact_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_Artifact_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.Artifact.class, flyteidl.artifact.Artifacts.Artifact.Builder.class); - } - - private int bitField0_; - public static final int ARTIFACT_ID_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - return getArtifactId(); - } - - public static final int SPEC_FIELD_NUMBER = 2; - private flyteidl.artifact.Artifacts.ArtifactSpec spec_; - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public boolean hasSpec() { - return spec_ != null; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec getSpec() { - return spec_ == null ? flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder() { - return getSpec(); - } - - public static final int TAGS_FIELD_NUMBER = 3; - private com.google.protobuf.LazyStringList tags_; - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - public com.google.protobuf.ProtocolStringList - getTagsList() { - return tags_; - } - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - public int getTagsCount() { - return tags_.size(); - } - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - public java.lang.String getTags(int index) { - return tags_.get(index); - } - /** - *
-     * references the tag field in ArtifactTag
-     * 
- * - * repeated string tags = 3; - */ - public com.google.protobuf.ByteString - getTagsBytes(int index) { - return tags_.getByteString(index); - } - - public static final int SOURCE_FIELD_NUMBER = 4; - private flyteidl.artifact.Artifacts.ArtifactSource source_; - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public boolean hasSource() { - return source_ != null; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public flyteidl.artifact.Artifacts.ArtifactSource getSource() { - return source_ == null ? flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder() { - return getSource(); - } - - 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 (artifactId_ != null) { - output.writeMessage(1, getArtifactId()); - } - if (spec_ != null) { - output.writeMessage(2, getSpec()); - } - for (int i = 0; i < tags_.size(); i++) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 3, tags_.getRaw(i)); - } - if (source_ != null) { - output.writeMessage(4, getSource()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifactId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifactId()); - } - if (spec_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getSpec()); - } - { - int dataSize = 0; - for (int i = 0; i < tags_.size(); i++) { - dataSize += computeStringSizeNoTag(tags_.getRaw(i)); - } - size += dataSize; - size += 1 * getTagsList().size(); - } - if (source_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, getSource()); - } - 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.artifact.Artifacts.Artifact)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.Artifact other = (flyteidl.artifact.Artifacts.Artifact) obj; - - if (hasArtifactId() != other.hasArtifactId()) return false; - if (hasArtifactId()) { - if (!getArtifactId() - .equals(other.getArtifactId())) return false; - } - if (hasSpec() != other.hasSpec()) return false; - if (hasSpec()) { - if (!getSpec() - .equals(other.getSpec())) return false; - } - if (!getTagsList() - .equals(other.getTagsList())) return false; - if (hasSource() != other.hasSource()) return false; - if (hasSource()) { - if (!getSource() - .equals(other.getSource())) 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 (hasArtifactId()) { - hash = (37 * hash) + ARTIFACT_ID_FIELD_NUMBER; - hash = (53 * hash) + getArtifactId().hashCode(); - } - if (hasSpec()) { - hash = (37 * hash) + SPEC_FIELD_NUMBER; - hash = (53 * hash) + getSpec().hashCode(); - } - if (getTagsCount() > 0) { - hash = (37 * hash) + TAGS_FIELD_NUMBER; - hash = (53 * hash) + getTagsList().hashCode(); - } - if (hasSource()) { - hash = (37 * hash) + SOURCE_FIELD_NUMBER; - hash = (53 * hash) + getSource().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.Artifact parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.Artifact parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.Artifact 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.artifact.Artifacts.Artifact parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.Artifact 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.artifact.Artifacts.Artifact parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.Artifact 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.artifact.Artifacts.Artifact 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.artifact.Artifact} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.Artifact) - flyteidl.artifact.Artifacts.ArtifactOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_Artifact_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_Artifact_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.Artifact.class, flyteidl.artifact.Artifacts.Artifact.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.Artifact.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 (artifactIdBuilder_ == null) { - artifactId_ = null; - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - if (specBuilder_ == null) { - spec_ = null; - } else { - spec_ = null; - specBuilder_ = null; - } - tags_ = com.google.protobuf.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000004); - if (sourceBuilder_ == null) { - source_ = null; - } else { - source_ = null; - sourceBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_Artifact_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.Artifact getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.Artifact.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.Artifact build() { - flyteidl.artifact.Artifacts.Artifact result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.Artifact buildPartial() { - flyteidl.artifact.Artifacts.Artifact result = new flyteidl.artifact.Artifacts.Artifact(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (artifactIdBuilder_ == null) { - result.artifactId_ = artifactId_; - } else { - result.artifactId_ = artifactIdBuilder_.build(); - } - if (specBuilder_ == null) { - result.spec_ = spec_; - } else { - result.spec_ = specBuilder_.build(); - } - if (((bitField0_ & 0x00000004) != 0)) { - tags_ = tags_.getUnmodifiableView(); - bitField0_ = (bitField0_ & ~0x00000004); - } - result.tags_ = tags_; - if (sourceBuilder_ == null) { - result.source_ = source_; - } else { - result.source_ = sourceBuilder_.build(); - } - result.bitField0_ = to_bitField0_; - 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.artifact.Artifacts.Artifact) { - return mergeFrom((flyteidl.artifact.Artifacts.Artifact)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.Artifact other) { - if (other == flyteidl.artifact.Artifacts.Artifact.getDefaultInstance()) return this; - if (other.hasArtifactId()) { - mergeArtifactId(other.getArtifactId()); - } - if (other.hasSpec()) { - mergeSpec(other.getSpec()); - } - if (!other.tags_.isEmpty()) { - if (tags_.isEmpty()) { - tags_ = other.tags_; - bitField0_ = (bitField0_ & ~0x00000004); - } else { - ensureTagsIsMutable(); - tags_.addAll(other.tags_); - } - onChanged(); - } - if (other.hasSource()) { - mergeSource(other.getSource()); - } - 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.artifact.Artifacts.Artifact parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.Artifact) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> artifactIdBuilder_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactIdBuilder_ != null || artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - if (artifactIdBuilder_ == null) { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } else { - return artifactIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifactId_ = value; - onChanged(); - } else { - artifactIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId( - flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (artifactIdBuilder_ == null) { - artifactId_ = builderForValue.build(); - onChanged(); - } else { - artifactIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder mergeArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (artifactId_ != null) { - artifactId_ = - flyteidl.core.ArtifactId.ArtifactID.newBuilder(artifactId_).mergeFrom(value).buildPartial(); - } else { - artifactId_ = value; - } - onChanged(); - } else { - artifactIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder clearArtifactId() { - if (artifactIdBuilder_ == null) { - artifactId_ = null; - onChanged(); - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder getArtifactIdBuilder() { - - onChanged(); - return getArtifactIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - if (artifactIdBuilder_ != null) { - return artifactIdBuilder_.getMessageOrBuilder(); - } else { - return artifactId_ == null ? - flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> - getArtifactIdFieldBuilder() { - if (artifactIdBuilder_ == null) { - artifactIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder>( - getArtifactId(), - getParentForChildren(), - isClean()); - artifactId_ = null; - } - return artifactIdBuilder_; - } - - private flyteidl.artifact.Artifacts.ArtifactSpec spec_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder> specBuilder_; - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public boolean hasSpec() { - return specBuilder_ != null || spec_ != null; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec getSpec() { - if (specBuilder_ == null) { - return spec_ == null ? flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } else { - return specBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder setSpec(flyteidl.artifact.Artifacts.ArtifactSpec value) { - if (specBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - spec_ = value; - onChanged(); - } else { - specBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder setSpec( - flyteidl.artifact.Artifacts.ArtifactSpec.Builder builderForValue) { - if (specBuilder_ == null) { - spec_ = builderForValue.build(); - onChanged(); - } else { - specBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder mergeSpec(flyteidl.artifact.Artifacts.ArtifactSpec value) { - if (specBuilder_ == null) { - if (spec_ != null) { - spec_ = - flyteidl.artifact.Artifacts.ArtifactSpec.newBuilder(spec_).mergeFrom(value).buildPartial(); - } else { - spec_ = value; - } - onChanged(); - } else { - specBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder clearSpec() { - if (specBuilder_ == null) { - spec_ = null; - onChanged(); - } else { - spec_ = null; - specBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec.Builder getSpecBuilder() { - - onChanged(); - return getSpecFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder() { - if (specBuilder_ != null) { - return specBuilder_.getMessageOrBuilder(); - } else { - return spec_ == null ? - flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder> - getSpecFieldBuilder() { - if (specBuilder_ == null) { - specBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder>( - getSpec(), - getParentForChildren(), - isClean()); - spec_ = null; - } - return specBuilder_; - } - - private com.google.protobuf.LazyStringList tags_ = com.google.protobuf.LazyStringArrayList.EMPTY; - private void ensureTagsIsMutable() { - if (!((bitField0_ & 0x00000004) != 0)) { - tags_ = new com.google.protobuf.LazyStringArrayList(tags_); - bitField0_ |= 0x00000004; - } - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public com.google.protobuf.ProtocolStringList - getTagsList() { - return tags_.getUnmodifiableView(); - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public int getTagsCount() { - return tags_.size(); - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public java.lang.String getTags(int index) { - return tags_.get(index); - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public com.google.protobuf.ByteString - getTagsBytes(int index) { - return tags_.getByteString(index); - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public Builder setTags( - int index, java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureTagsIsMutable(); - tags_.set(index, value); - onChanged(); - return this; - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public Builder addTags( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureTagsIsMutable(); - tags_.add(value); - onChanged(); - return this; - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public Builder addAllTags( - java.lang.Iterable values) { - ensureTagsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, tags_); - onChanged(); - return this; - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public Builder clearTags() { - tags_ = com.google.protobuf.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000004); - onChanged(); - return this; - } - /** - *
-       * references the tag field in ArtifactTag
-       * 
- * - * repeated string tags = 3; - */ - public Builder addTagsBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - ensureTagsIsMutable(); - tags_.add(value); - onChanged(); - return this; - } - - private flyteidl.artifact.Artifacts.ArtifactSource source_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder> sourceBuilder_; - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public boolean hasSource() { - return sourceBuilder_ != null || source_ != null; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public flyteidl.artifact.Artifacts.ArtifactSource getSource() { - if (sourceBuilder_ == null) { - return source_ == null ? flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } else { - return sourceBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public Builder setSource(flyteidl.artifact.Artifacts.ArtifactSource value) { - if (sourceBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - source_ = value; - onChanged(); - } else { - sourceBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public Builder setSource( - flyteidl.artifact.Artifacts.ArtifactSource.Builder builderForValue) { - if (sourceBuilder_ == null) { - source_ = builderForValue.build(); - onChanged(); - } else { - sourceBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public Builder mergeSource(flyteidl.artifact.Artifacts.ArtifactSource value) { - if (sourceBuilder_ == null) { - if (source_ != null) { - source_ = - flyteidl.artifact.Artifacts.ArtifactSource.newBuilder(source_).mergeFrom(value).buildPartial(); - } else { - source_ = value; - } - onChanged(); - } else { - sourceBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public Builder clearSource() { - if (sourceBuilder_ == null) { - source_ = null; - onChanged(); - } else { - source_ = null; - sourceBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public flyteidl.artifact.Artifacts.ArtifactSource.Builder getSourceBuilder() { - - onChanged(); - return getSourceFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - public flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder() { - if (sourceBuilder_ != null) { - return sourceBuilder_.getMessageOrBuilder(); - } else { - return source_ == null ? - flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } - } - /** - * .flyteidl.artifact.ArtifactSource source = 4; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder> - getSourceFieldBuilder() { - if (sourceBuilder_ == null) { - sourceBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder>( - getSource(), - getParentForChildren(), - isClean()); - source_ = null; - } - return sourceBuilder_; - } - @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.artifact.Artifact) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.Artifact) - private static final flyteidl.artifact.Artifacts.Artifact DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.Artifact(); - } - - public static flyteidl.artifact.Artifacts.Artifact getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public Artifact parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new Artifact(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.artifact.Artifacts.Artifact getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface CreateArtifactRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.CreateArtifactRequest) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - boolean hasArtifactKey(); - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - flyteidl.core.ArtifactId.ArtifactKey getArtifactKey(); - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder(); - - /** - * string version = 3; - */ - java.lang.String getVersion(); - /** - * string version = 3; - */ - com.google.protobuf.ByteString - getVersionBytes(); - - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - boolean hasSpec(); - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - flyteidl.artifact.Artifacts.ArtifactSpec getSpec(); - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder(); - - /** - * map<string, string> partitions = 4; - */ - int getPartitionsCount(); - /** - * map<string, string> partitions = 4; - */ - boolean containsPartitions( - java.lang.String key); - /** - * Use {@link #getPartitionsMap()} instead. - */ - @java.lang.Deprecated - java.util.Map - getPartitions(); - /** - * map<string, string> partitions = 4; - */ - java.util.Map - getPartitionsMap(); - /** - * map<string, string> partitions = 4; - */ - - java.lang.String getPartitionsOrDefault( - java.lang.String key, - java.lang.String defaultValue); - /** - * map<string, string> partitions = 4; - */ - - java.lang.String getPartitionsOrThrow( - java.lang.String key); - - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - boolean hasTimePartitionValue(); - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - com.google.protobuf.Timestamp getTimePartitionValue(); - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder(); - - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - boolean hasSource(); - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - flyteidl.artifact.Artifacts.ArtifactSource getSource(); - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.CreateArtifactRequest} - */ - public static final class CreateArtifactRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.CreateArtifactRequest) - CreateArtifactRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use CreateArtifactRequest.newBuilder() to construct. - private CreateArtifactRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private CreateArtifactRequest() { - version_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private CreateArtifactRequest( - 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.ArtifactId.ArtifactKey.Builder subBuilder = null; - if (artifactKey_ != null) { - subBuilder = artifactKey_.toBuilder(); - } - artifactKey_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactKey.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifactKey_); - artifactKey_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.artifact.Artifacts.ArtifactSpec.Builder subBuilder = null; - if (spec_ != null) { - subBuilder = spec_.toBuilder(); - } - spec_ = input.readMessage(flyteidl.artifact.Artifacts.ArtifactSpec.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(spec_); - spec_ = subBuilder.buildPartial(); - } - - break; - } - case 26: { - java.lang.String s = input.readStringRequireUtf8(); - - version_ = s; - break; - } - case 34: { - if (!((mutable_bitField0_ & 0x00000008) != 0)) { - partitions_ = com.google.protobuf.MapField.newMapField( - PartitionsDefaultEntryHolder.defaultEntry); - mutable_bitField0_ |= 0x00000008; - } - com.google.protobuf.MapEntry - partitions__ = input.readMessage( - PartitionsDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); - partitions_.getMutableMap().put( - partitions__.getKey(), partitions__.getValue()); - break; - } - case 42: { - com.google.protobuf.Timestamp.Builder subBuilder = null; - if (timePartitionValue_ != null) { - subBuilder = timePartitionValue_.toBuilder(); - } - timePartitionValue_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(timePartitionValue_); - timePartitionValue_ = subBuilder.buildPartial(); - } - - break; - } - case 50: { - flyteidl.artifact.Artifacts.ArtifactSource.Builder subBuilder = null; - if (source_ != null) { - subBuilder = source_.toBuilder(); - } - source_ = input.readMessage(flyteidl.artifact.Artifacts.ArtifactSource.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(source_); - source_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor; - } - - @SuppressWarnings({"rawtypes"}) - @java.lang.Override - protected com.google.protobuf.MapField internalGetMapField( - int number) { - switch (number) { - case 4: - return internalGetPartitions(); - default: - throw new RuntimeException( - "Invalid map field number: " + number); - } - } - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateArtifactRequest.class, flyteidl.artifact.Artifacts.CreateArtifactRequest.Builder.class); - } - - private int bitField0_; - public static final int ARTIFACT_KEY_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactKey artifactKey_; - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public boolean hasArtifactKey() { - return artifactKey_ != null; - } - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey getArtifactKey() { - return artifactKey_ == null ? flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } - /** - *
-     * Specify just project/domain on creation
-     * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder() { - return getArtifactKey(); - } - - public static final int VERSION_FIELD_NUMBER = 3; - private volatile java.lang.Object version_; - /** - * string version = 3; - */ - public java.lang.String getVersion() { - java.lang.Object ref = version_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - version_ = s; - return s; - } - } - /** - * string version = 3; - */ - public com.google.protobuf.ByteString - getVersionBytes() { - java.lang.Object ref = version_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - version_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int SPEC_FIELD_NUMBER = 2; - private flyteidl.artifact.Artifacts.ArtifactSpec spec_; - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public boolean hasSpec() { - return spec_ != null; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec getSpec() { - return spec_ == null ? flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder() { - return getSpec(); - } - - public static final int PARTITIONS_FIELD_NUMBER = 4; - private static final class PartitionsDefaultEntryHolder { - static final com.google.protobuf.MapEntry< - java.lang.String, java.lang.String> defaultEntry = - com.google.protobuf.MapEntry - .newDefaultInstance( - flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_descriptor, - com.google.protobuf.WireFormat.FieldType.STRING, - "", - com.google.protobuf.WireFormat.FieldType.STRING, - ""); - } - private com.google.protobuf.MapField< - java.lang.String, java.lang.String> partitions_; - private com.google.protobuf.MapField - internalGetPartitions() { - if (partitions_ == null) { - return com.google.protobuf.MapField.emptyMapField( - PartitionsDefaultEntryHolder.defaultEntry); - } - return partitions_; - } - - public int getPartitionsCount() { - return internalGetPartitions().getMap().size(); - } - /** - * map<string, string> partitions = 4; - */ - - public boolean containsPartitions( - java.lang.String key) { - if (key == null) { throw new java.lang.NullPointerException(); } - return internalGetPartitions().getMap().containsKey(key); - } - /** - * Use {@link #getPartitionsMap()} instead. - */ - @java.lang.Deprecated - public java.util.Map getPartitions() { - return getPartitionsMap(); - } - /** - * map<string, string> partitions = 4; - */ - - public java.util.Map getPartitionsMap() { - return internalGetPartitions().getMap(); - } - /** - * map<string, string> partitions = 4; - */ - - public java.lang.String getPartitionsOrDefault( - java.lang.String key, - java.lang.String defaultValue) { - if (key == null) { throw new java.lang.NullPointerException(); } - java.util.Map map = - internalGetPartitions().getMap(); - return map.containsKey(key) ? map.get(key) : defaultValue; - } - /** - * map<string, string> partitions = 4; - */ - - public java.lang.String getPartitionsOrThrow( - java.lang.String key) { - if (key == null) { throw new java.lang.NullPointerException(); } - java.util.Map map = - internalGetPartitions().getMap(); - if (!map.containsKey(key)) { - throw new java.lang.IllegalArgumentException(); - } - return map.get(key); - } - - public static final int TIME_PARTITION_VALUE_FIELD_NUMBER = 5; - private com.google.protobuf.Timestamp timePartitionValue_; - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public boolean hasTimePartitionValue() { - return timePartitionValue_ != null; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public com.google.protobuf.Timestamp getTimePartitionValue() { - return timePartitionValue_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder() { - return getTimePartitionValue(); - } - - public static final int SOURCE_FIELD_NUMBER = 6; - private flyteidl.artifact.Artifacts.ArtifactSource source_; - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public boolean hasSource() { - return source_ != null; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public flyteidl.artifact.Artifacts.ArtifactSource getSource() { - return source_ == null ? flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder() { - return getSource(); - } - - 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 (artifactKey_ != null) { - output.writeMessage(1, getArtifactKey()); - } - if (spec_ != null) { - output.writeMessage(2, getSpec()); - } - if (!getVersionBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 3, version_); - } - com.google.protobuf.GeneratedMessageV3 - .serializeStringMapTo( - output, - internalGetPartitions(), - PartitionsDefaultEntryHolder.defaultEntry, - 4); - if (timePartitionValue_ != null) { - output.writeMessage(5, getTimePartitionValue()); - } - if (source_ != null) { - output.writeMessage(6, getSource()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifactKey_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifactKey()); - } - if (spec_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getSpec()); - } - if (!getVersionBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, version_); - } - for (java.util.Map.Entry entry - : internalGetPartitions().getMap().entrySet()) { - com.google.protobuf.MapEntry - partitions__ = PartitionsDefaultEntryHolder.defaultEntry.newBuilderForType() - .setKey(entry.getKey()) - .setValue(entry.getValue()) - .build(); - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, partitions__); - } - if (timePartitionValue_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(5, getTimePartitionValue()); - } - if (source_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, getSource()); - } - 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.artifact.Artifacts.CreateArtifactRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.CreateArtifactRequest other = (flyteidl.artifact.Artifacts.CreateArtifactRequest) obj; - - if (hasArtifactKey() != other.hasArtifactKey()) return false; - if (hasArtifactKey()) { - if (!getArtifactKey() - .equals(other.getArtifactKey())) return false; - } - if (!getVersion() - .equals(other.getVersion())) return false; - if (hasSpec() != other.hasSpec()) return false; - if (hasSpec()) { - if (!getSpec() - .equals(other.getSpec())) return false; - } - if (!internalGetPartitions().equals( - other.internalGetPartitions())) return false; - if (hasTimePartitionValue() != other.hasTimePartitionValue()) return false; - if (hasTimePartitionValue()) { - if (!getTimePartitionValue() - .equals(other.getTimePartitionValue())) return false; - } - if (hasSource() != other.hasSource()) return false; - if (hasSource()) { - if (!getSource() - .equals(other.getSource())) 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 (hasArtifactKey()) { - hash = (37 * hash) + ARTIFACT_KEY_FIELD_NUMBER; - hash = (53 * hash) + getArtifactKey().hashCode(); - } - hash = (37 * hash) + VERSION_FIELD_NUMBER; - hash = (53 * hash) + getVersion().hashCode(); - if (hasSpec()) { - hash = (37 * hash) + SPEC_FIELD_NUMBER; - hash = (53 * hash) + getSpec().hashCode(); - } - if (!internalGetPartitions().getMap().isEmpty()) { - hash = (37 * hash) + PARTITIONS_FIELD_NUMBER; - hash = (53 * hash) + internalGetPartitions().hashCode(); - } - if (hasTimePartitionValue()) { - hash = (37 * hash) + TIME_PARTITION_VALUE_FIELD_NUMBER; - hash = (53 * hash) + getTimePartitionValue().hashCode(); - } - if (hasSource()) { - hash = (37 * hash) + SOURCE_FIELD_NUMBER; - hash = (53 * hash) + getSource().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest 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.artifact.Artifacts.CreateArtifactRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest 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.artifact.Artifacts.CreateArtifactRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactRequest 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.artifact.Artifacts.CreateArtifactRequest 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.artifact.CreateArtifactRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.CreateArtifactRequest) - flyteidl.artifact.Artifacts.CreateArtifactRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor; - } - - @SuppressWarnings({"rawtypes"}) - protected com.google.protobuf.MapField internalGetMapField( - int number) { - switch (number) { - case 4: - return internalGetPartitions(); - default: - throw new RuntimeException( - "Invalid map field number: " + number); - } - } - @SuppressWarnings({"rawtypes"}) - protected com.google.protobuf.MapField internalGetMutableMapField( - int number) { - switch (number) { - case 4: - return internalGetMutablePartitions(); - default: - throw new RuntimeException( - "Invalid map field number: " + number); - } - } - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateArtifactRequest.class, flyteidl.artifact.Artifacts.CreateArtifactRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.CreateArtifactRequest.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 (artifactKeyBuilder_ == null) { - artifactKey_ = null; - } else { - artifactKey_ = null; - artifactKeyBuilder_ = null; - } - version_ = ""; - - if (specBuilder_ == null) { - spec_ = null; - } else { - spec_ = null; - specBuilder_ = null; - } - internalGetMutablePartitions().clear(); - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = null; - } else { - timePartitionValue_ = null; - timePartitionValueBuilder_ = null; - } - if (sourceBuilder_ == null) { - source_ = null; - } else { - source_ = null; - sourceBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.CreateArtifactRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactRequest build() { - flyteidl.artifact.Artifacts.CreateArtifactRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactRequest buildPartial() { - flyteidl.artifact.Artifacts.CreateArtifactRequest result = new flyteidl.artifact.Artifacts.CreateArtifactRequest(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (artifactKeyBuilder_ == null) { - result.artifactKey_ = artifactKey_; - } else { - result.artifactKey_ = artifactKeyBuilder_.build(); - } - result.version_ = version_; - if (specBuilder_ == null) { - result.spec_ = spec_; - } else { - result.spec_ = specBuilder_.build(); - } - result.partitions_ = internalGetPartitions(); - result.partitions_.makeImmutable(); - if (timePartitionValueBuilder_ == null) { - result.timePartitionValue_ = timePartitionValue_; - } else { - result.timePartitionValue_ = timePartitionValueBuilder_.build(); - } - if (sourceBuilder_ == null) { - result.source_ = source_; - } else { - result.source_ = sourceBuilder_.build(); - } - result.bitField0_ = to_bitField0_; - 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.artifact.Artifacts.CreateArtifactRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.CreateArtifactRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.CreateArtifactRequest other) { - if (other == flyteidl.artifact.Artifacts.CreateArtifactRequest.getDefaultInstance()) return this; - if (other.hasArtifactKey()) { - mergeArtifactKey(other.getArtifactKey()); - } - if (!other.getVersion().isEmpty()) { - version_ = other.version_; - onChanged(); - } - if (other.hasSpec()) { - mergeSpec(other.getSpec()); - } - internalGetMutablePartitions().mergeFrom( - other.internalGetPartitions()); - if (other.hasTimePartitionValue()) { - mergeTimePartitionValue(other.getTimePartitionValue()); - } - if (other.hasSource()) { - mergeSource(other.getSource()); - } - 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.artifact.Artifacts.CreateArtifactRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.CreateArtifactRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private flyteidl.core.ArtifactId.ArtifactKey artifactKey_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder> artifactKeyBuilder_; - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public boolean hasArtifactKey() { - return artifactKeyBuilder_ != null || artifactKey_ != null; - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey getArtifactKey() { - if (artifactKeyBuilder_ == null) { - return artifactKey_ == null ? flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } else { - return artifactKeyBuilder_.getMessage(); - } - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder setArtifactKey(flyteidl.core.ArtifactId.ArtifactKey value) { - if (artifactKeyBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifactKey_ = value; - onChanged(); - } else { - artifactKeyBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder setArtifactKey( - flyteidl.core.ArtifactId.ArtifactKey.Builder builderForValue) { - if (artifactKeyBuilder_ == null) { - artifactKey_ = builderForValue.build(); - onChanged(); - } else { - artifactKeyBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder mergeArtifactKey(flyteidl.core.ArtifactId.ArtifactKey value) { - if (artifactKeyBuilder_ == null) { - if (artifactKey_ != null) { - artifactKey_ = - flyteidl.core.ArtifactId.ArtifactKey.newBuilder(artifactKey_).mergeFrom(value).buildPartial(); - } else { - artifactKey_ = value; - } - onChanged(); - } else { - artifactKeyBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder clearArtifactKey() { - if (artifactKeyBuilder_ == null) { - artifactKey_ = null; - onChanged(); - } else { - artifactKey_ = null; - artifactKeyBuilder_ = null; - } - - return this; - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey.Builder getArtifactKeyBuilder() { - - onChanged(); - return getArtifactKeyFieldBuilder().getBuilder(); - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder() { - if (artifactKeyBuilder_ != null) { - return artifactKeyBuilder_.getMessageOrBuilder(); - } else { - return artifactKey_ == null ? - flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } - } - /** - *
-       * Specify just project/domain on creation
-       * 
- * - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder> - getArtifactKeyFieldBuilder() { - if (artifactKeyBuilder_ == null) { - artifactKeyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder>( - getArtifactKey(), - getParentForChildren(), - isClean()); - artifactKey_ = null; - } - return artifactKeyBuilder_; - } - - private java.lang.Object version_ = ""; - /** - * string version = 3; - */ - public java.lang.String getVersion() { - java.lang.Object ref = version_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - version_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string version = 3; - */ - public com.google.protobuf.ByteString - getVersionBytes() { - java.lang.Object ref = version_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - version_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string version = 3; - */ - public Builder setVersion( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - version_ = value; - onChanged(); - return this; - } - /** - * string version = 3; - */ - public Builder clearVersion() { - - version_ = getDefaultInstance().getVersion(); - onChanged(); - return this; - } - /** - * string version = 3; - */ - public Builder setVersionBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - version_ = value; - onChanged(); - return this; - } - - private flyteidl.artifact.Artifacts.ArtifactSpec spec_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder> specBuilder_; - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public boolean hasSpec() { - return specBuilder_ != null || spec_ != null; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec getSpec() { - if (specBuilder_ == null) { - return spec_ == null ? flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } else { - return specBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder setSpec(flyteidl.artifact.Artifacts.ArtifactSpec value) { - if (specBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - spec_ = value; - onChanged(); - } else { - specBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder setSpec( - flyteidl.artifact.Artifacts.ArtifactSpec.Builder builderForValue) { - if (specBuilder_ == null) { - spec_ = builderForValue.build(); - onChanged(); - } else { - specBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder mergeSpec(flyteidl.artifact.Artifacts.ArtifactSpec value) { - if (specBuilder_ == null) { - if (spec_ != null) { - spec_ = - flyteidl.artifact.Artifacts.ArtifactSpec.newBuilder(spec_).mergeFrom(value).buildPartial(); - } else { - spec_ = value; - } - onChanged(); - } else { - specBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public Builder clearSpec() { - if (specBuilder_ == null) { - spec_ = null; - onChanged(); - } else { - spec_ = null; - specBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpec.Builder getSpecBuilder() { - - onChanged(); - return getSpecFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - public flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder getSpecOrBuilder() { - if (specBuilder_ != null) { - return specBuilder_.getMessageOrBuilder(); - } else { - return spec_ == null ? - flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance() : spec_; - } - } - /** - * .flyteidl.artifact.ArtifactSpec spec = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder> - getSpecFieldBuilder() { - if (specBuilder_ == null) { - specBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSpec, flyteidl.artifact.Artifacts.ArtifactSpec.Builder, flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder>( - getSpec(), - getParentForChildren(), - isClean()); - spec_ = null; - } - return specBuilder_; - } - - private com.google.protobuf.MapField< - java.lang.String, java.lang.String> partitions_; - private com.google.protobuf.MapField - internalGetPartitions() { - if (partitions_ == null) { - return com.google.protobuf.MapField.emptyMapField( - PartitionsDefaultEntryHolder.defaultEntry); - } - return partitions_; - } - private com.google.protobuf.MapField - internalGetMutablePartitions() { - onChanged();; - if (partitions_ == null) { - partitions_ = com.google.protobuf.MapField.newMapField( - PartitionsDefaultEntryHolder.defaultEntry); - } - if (!partitions_.isMutable()) { - partitions_ = partitions_.copy(); - } - return partitions_; - } - - public int getPartitionsCount() { - return internalGetPartitions().getMap().size(); - } - /** - * map<string, string> partitions = 4; - */ - - public boolean containsPartitions( - java.lang.String key) { - if (key == null) { throw new java.lang.NullPointerException(); } - return internalGetPartitions().getMap().containsKey(key); - } - /** - * Use {@link #getPartitionsMap()} instead. - */ - @java.lang.Deprecated - public java.util.Map getPartitions() { - return getPartitionsMap(); - } - /** - * map<string, string> partitions = 4; - */ - - public java.util.Map getPartitionsMap() { - return internalGetPartitions().getMap(); - } - /** - * map<string, string> partitions = 4; - */ - - public java.lang.String getPartitionsOrDefault( - java.lang.String key, - java.lang.String defaultValue) { - if (key == null) { throw new java.lang.NullPointerException(); } - java.util.Map map = - internalGetPartitions().getMap(); - return map.containsKey(key) ? map.get(key) : defaultValue; - } - /** - * map<string, string> partitions = 4; - */ - - public java.lang.String getPartitionsOrThrow( - java.lang.String key) { - if (key == null) { throw new java.lang.NullPointerException(); } - java.util.Map map = - internalGetPartitions().getMap(); - if (!map.containsKey(key)) { - throw new java.lang.IllegalArgumentException(); - } - return map.get(key); - } - - public Builder clearPartitions() { - internalGetMutablePartitions().getMutableMap() - .clear(); - return this; - } - /** - * map<string, string> partitions = 4; - */ - - public Builder removePartitions( - java.lang.String key) { - if (key == null) { throw new java.lang.NullPointerException(); } - internalGetMutablePartitions().getMutableMap() - .remove(key); - return this; - } - /** - * Use alternate mutation accessors instead. - */ - @java.lang.Deprecated - public java.util.Map - getMutablePartitions() { - return internalGetMutablePartitions().getMutableMap(); - } - /** - * map<string, string> partitions = 4; - */ - public Builder putPartitions( - java.lang.String key, - java.lang.String value) { - if (key == null) { throw new java.lang.NullPointerException(); } - if (value == null) { throw new java.lang.NullPointerException(); } - internalGetMutablePartitions().getMutableMap() - .put(key, value); - return this; - } - /** - * map<string, string> partitions = 4; - */ - - public Builder putAllPartitions( - java.util.Map values) { - internalGetMutablePartitions().getMutableMap() - .putAll(values); - return this; - } - - private com.google.protobuf.Timestamp timePartitionValue_; - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> timePartitionValueBuilder_; - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public boolean hasTimePartitionValue() { - return timePartitionValueBuilder_ != null || timePartitionValue_ != null; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public com.google.protobuf.Timestamp getTimePartitionValue() { - if (timePartitionValueBuilder_ == null) { - return timePartitionValue_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } else { - return timePartitionValueBuilder_.getMessage(); - } - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public Builder setTimePartitionValue(com.google.protobuf.Timestamp value) { - if (timePartitionValueBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - timePartitionValue_ = value; - onChanged(); - } else { - timePartitionValueBuilder_.setMessage(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public Builder setTimePartitionValue( - com.google.protobuf.Timestamp.Builder builderForValue) { - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = builderForValue.build(); - onChanged(); - } else { - timePartitionValueBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public Builder mergeTimePartitionValue(com.google.protobuf.Timestamp value) { - if (timePartitionValueBuilder_ == null) { - if (timePartitionValue_ != null) { - timePartitionValue_ = - com.google.protobuf.Timestamp.newBuilder(timePartitionValue_).mergeFrom(value).buildPartial(); - } else { - timePartitionValue_ = value; - } - onChanged(); - } else { - timePartitionValueBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public Builder clearTimePartitionValue() { - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = null; - onChanged(); - } else { - timePartitionValue_ = null; - timePartitionValueBuilder_ = null; - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public com.google.protobuf.Timestamp.Builder getTimePartitionValueBuilder() { - - onChanged(); - return getTimePartitionValueFieldBuilder().getBuilder(); - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - public com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder() { - if (timePartitionValueBuilder_ != null) { - return timePartitionValueBuilder_.getMessageOrBuilder(); - } else { - return timePartitionValue_ == null ? - com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } - } - /** - * .google.protobuf.Timestamp time_partition_value = 5; - */ - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> - getTimePartitionValueFieldBuilder() { - if (timePartitionValueBuilder_ == null) { - timePartitionValueBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>( - getTimePartitionValue(), - getParentForChildren(), - isClean()); - timePartitionValue_ = null; - } - return timePartitionValueBuilder_; - } - - private flyteidl.artifact.Artifacts.ArtifactSource source_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder> sourceBuilder_; - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public boolean hasSource() { - return sourceBuilder_ != null || source_ != null; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public flyteidl.artifact.Artifacts.ArtifactSource getSource() { - if (sourceBuilder_ == null) { - return source_ == null ? flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } else { - return sourceBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public Builder setSource(flyteidl.artifact.Artifacts.ArtifactSource value) { - if (sourceBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - source_ = value; - onChanged(); - } else { - sourceBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public Builder setSource( - flyteidl.artifact.Artifacts.ArtifactSource.Builder builderForValue) { - if (sourceBuilder_ == null) { - source_ = builderForValue.build(); - onChanged(); - } else { - sourceBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public Builder mergeSource(flyteidl.artifact.Artifacts.ArtifactSource value) { - if (sourceBuilder_ == null) { - if (source_ != null) { - source_ = - flyteidl.artifact.Artifacts.ArtifactSource.newBuilder(source_).mergeFrom(value).buildPartial(); - } else { - source_ = value; - } - onChanged(); - } else { - sourceBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public Builder clearSource() { - if (sourceBuilder_ == null) { - source_ = null; - onChanged(); - } else { - source_ = null; - sourceBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public flyteidl.artifact.Artifacts.ArtifactSource.Builder getSourceBuilder() { - - onChanged(); - return getSourceFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - public flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder getSourceOrBuilder() { - if (sourceBuilder_ != null) { - return sourceBuilder_.getMessageOrBuilder(); - } else { - return source_ == null ? - flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance() : source_; - } - } - /** - * .flyteidl.artifact.ArtifactSource source = 6; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder> - getSourceFieldBuilder() { - if (sourceBuilder_ == null) { - sourceBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactSource, flyteidl.artifact.Artifacts.ArtifactSource.Builder, flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder>( - getSource(), - getParentForChildren(), - isClean()); - source_ = null; - } - return sourceBuilder_; - } - @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.artifact.CreateArtifactRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateArtifactRequest) - private static final flyteidl.artifact.Artifacts.CreateArtifactRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.CreateArtifactRequest(); - } - - public static flyteidl.artifact.Artifacts.CreateArtifactRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public CreateArtifactRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new CreateArtifactRequest(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.artifact.Artifacts.CreateArtifactRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ArtifactSourceOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ArtifactSource) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - boolean hasWorkflowExecution(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getWorkflowExecution(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getWorkflowExecutionOrBuilder(); - - /** - * string node_id = 2; - */ - java.lang.String getNodeId(); - /** - * string node_id = 2; - */ - com.google.protobuf.ByteString - getNodeIdBytes(); - - /** - * .flyteidl.core.Identifier task_id = 3; - */ - boolean hasTaskId(); - /** - * .flyteidl.core.Identifier task_id = 3; - */ - flyteidl.core.IdentifierOuterClass.Identifier getTaskId(); - /** - * .flyteidl.core.Identifier task_id = 3; - */ - flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTaskIdOrBuilder(); - - /** - * uint32 retry_attempt = 4; - */ - int getRetryAttempt(); - - /** - *
-     * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-     * 
- * - * string principal = 5; - */ - java.lang.String getPrincipal(); - /** - *
-     * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-     * 
- * - * string principal = 5; - */ - com.google.protobuf.ByteString - getPrincipalBytes(); - } - /** - * Protobuf type {@code flyteidl.artifact.ArtifactSource} - */ - public static final class ArtifactSource extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ArtifactSource) - ArtifactSourceOrBuilder { - private static final long serialVersionUID = 0L; - // Use ArtifactSource.newBuilder() to construct. - private ArtifactSource(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ArtifactSource() { - nodeId_ = ""; - principal_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ArtifactSource( - 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.WorkflowExecutionIdentifier.Builder subBuilder = null; - if (workflowExecution_ != null) { - subBuilder = workflowExecution_.toBuilder(); - } - workflowExecution_ = input.readMessage(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(workflowExecution_); - workflowExecution_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - java.lang.String s = input.readStringRequireUtf8(); - - nodeId_ = s; - break; - } - case 26: { - flyteidl.core.IdentifierOuterClass.Identifier.Builder subBuilder = null; - if (taskId_ != null) { - subBuilder = taskId_.toBuilder(); - } - taskId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.Identifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(taskId_); - taskId_ = subBuilder.buildPartial(); - } - - break; - } - case 32: { - - retryAttempt_ = input.readUInt32(); - break; - } - case 42: { - java.lang.String s = input.readStringRequireUtf8(); - - principal_ = s; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSource_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSource_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactSource.class, flyteidl.artifact.Artifacts.ArtifactSource.Builder.class); - } - - public static final int WORKFLOW_EXECUTION_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier workflowExecution_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public boolean hasWorkflowExecution() { - return workflowExecution_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getWorkflowExecution() { - return workflowExecution_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : workflowExecution_; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getWorkflowExecutionOrBuilder() { - return getWorkflowExecution(); - } - - public static final int NODE_ID_FIELD_NUMBER = 2; - private volatile java.lang.Object nodeId_; - /** - * string node_id = 2; - */ - public java.lang.String getNodeId() { - java.lang.Object ref = nodeId_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - nodeId_ = s; - return s; - } - } - /** - * string node_id = 2; - */ - public com.google.protobuf.ByteString - getNodeIdBytes() { - java.lang.Object ref = nodeId_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - nodeId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int TASK_ID_FIELD_NUMBER = 3; - private flyteidl.core.IdentifierOuterClass.Identifier taskId_; - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public boolean hasTaskId() { - return taskId_ != null; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTaskId() { - return taskId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : taskId_; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTaskIdOrBuilder() { - return getTaskId(); - } - - public static final int RETRY_ATTEMPT_FIELD_NUMBER = 4; - private int retryAttempt_; - /** - * uint32 retry_attempt = 4; - */ - public int getRetryAttempt() { - return retryAttempt_; - } - - public static final int PRINCIPAL_FIELD_NUMBER = 5; - private volatile java.lang.Object principal_; - /** - *
-     * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-     * 
- * - * string principal = 5; - */ - public java.lang.String getPrincipal() { - java.lang.Object ref = principal_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - principal_ = s; - return s; - } - } - /** - *
-     * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-     * 
- * - * string principal = 5; - */ - public com.google.protobuf.ByteString - getPrincipalBytes() { - java.lang.Object ref = principal_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - principal_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - 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 (workflowExecution_ != null) { - output.writeMessage(1, getWorkflowExecution()); - } - if (!getNodeIdBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 2, nodeId_); - } - if (taskId_ != null) { - output.writeMessage(3, getTaskId()); - } - if (retryAttempt_ != 0) { - output.writeUInt32(4, retryAttempt_); - } - if (!getPrincipalBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 5, principal_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (workflowExecution_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getWorkflowExecution()); - } - if (!getNodeIdBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, nodeId_); - } - if (taskId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getTaskId()); - } - if (retryAttempt_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(4, retryAttempt_); - } - if (!getPrincipalBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, principal_); - } - 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.artifact.Artifacts.ArtifactSource)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ArtifactSource other = (flyteidl.artifact.Artifacts.ArtifactSource) obj; - - if (hasWorkflowExecution() != other.hasWorkflowExecution()) return false; - if (hasWorkflowExecution()) { - if (!getWorkflowExecution() - .equals(other.getWorkflowExecution())) return false; - } - if (!getNodeId() - .equals(other.getNodeId())) return false; - if (hasTaskId() != other.hasTaskId()) return false; - if (hasTaskId()) { - if (!getTaskId() - .equals(other.getTaskId())) return false; - } - if (getRetryAttempt() - != other.getRetryAttempt()) return false; - if (!getPrincipal() - .equals(other.getPrincipal())) 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 (hasWorkflowExecution()) { - hash = (37 * hash) + WORKFLOW_EXECUTION_FIELD_NUMBER; - hash = (53 * hash) + getWorkflowExecution().hashCode(); - } - hash = (37 * hash) + NODE_ID_FIELD_NUMBER; - hash = (53 * hash) + getNodeId().hashCode(); - if (hasTaskId()) { - hash = (37 * hash) + TASK_ID_FIELD_NUMBER; - hash = (53 * hash) + getTaskId().hashCode(); - } - hash = (37 * hash) + RETRY_ATTEMPT_FIELD_NUMBER; - hash = (53 * hash) + getRetryAttempt(); - hash = (37 * hash) + PRINCIPAL_FIELD_NUMBER; - hash = (53 * hash) + getPrincipal().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSource parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSource 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.artifact.Artifacts.ArtifactSource parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSource 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.artifact.Artifacts.ArtifactSource parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSource 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.artifact.Artifacts.ArtifactSource 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.artifact.ArtifactSource} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ArtifactSource) - flyteidl.artifact.Artifacts.ArtifactSourceOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSource_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSource_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactSource.class, flyteidl.artifact.Artifacts.ArtifactSource.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ArtifactSource.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 (workflowExecutionBuilder_ == null) { - workflowExecution_ = null; - } else { - workflowExecution_ = null; - workflowExecutionBuilder_ = null; - } - nodeId_ = ""; - - if (taskIdBuilder_ == null) { - taskId_ = null; - } else { - taskId_ = null; - taskIdBuilder_ = null; - } - retryAttempt_ = 0; - - principal_ = ""; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSource_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSource getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSource build() { - flyteidl.artifact.Artifacts.ArtifactSource result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSource buildPartial() { - flyteidl.artifact.Artifacts.ArtifactSource result = new flyteidl.artifact.Artifacts.ArtifactSource(this); - if (workflowExecutionBuilder_ == null) { - result.workflowExecution_ = workflowExecution_; - } else { - result.workflowExecution_ = workflowExecutionBuilder_.build(); - } - result.nodeId_ = nodeId_; - if (taskIdBuilder_ == null) { - result.taskId_ = taskId_; - } else { - result.taskId_ = taskIdBuilder_.build(); - } - result.retryAttempt_ = retryAttempt_; - result.principal_ = principal_; - 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.artifact.Artifacts.ArtifactSource) { - return mergeFrom((flyteidl.artifact.Artifacts.ArtifactSource)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ArtifactSource other) { - if (other == flyteidl.artifact.Artifacts.ArtifactSource.getDefaultInstance()) return this; - if (other.hasWorkflowExecution()) { - mergeWorkflowExecution(other.getWorkflowExecution()); - } - if (!other.getNodeId().isEmpty()) { - nodeId_ = other.nodeId_; - onChanged(); - } - if (other.hasTaskId()) { - mergeTaskId(other.getTaskId()); - } - if (other.getRetryAttempt() != 0) { - setRetryAttempt(other.getRetryAttempt()); - } - if (!other.getPrincipal().isEmpty()) { - principal_ = other.principal_; - onChanged(); - } - 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.artifact.Artifacts.ArtifactSource parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ArtifactSource) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier workflowExecution_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> workflowExecutionBuilder_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public boolean hasWorkflowExecution() { - return workflowExecutionBuilder_ != null || workflowExecution_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getWorkflowExecution() { - if (workflowExecutionBuilder_ == null) { - return workflowExecution_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : workflowExecution_; - } else { - return workflowExecutionBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public Builder setWorkflowExecution(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (workflowExecutionBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - workflowExecution_ = value; - onChanged(); - } else { - workflowExecutionBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public Builder setWorkflowExecution( - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (workflowExecutionBuilder_ == null) { - workflowExecution_ = builderForValue.build(); - onChanged(); - } else { - workflowExecutionBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public Builder mergeWorkflowExecution(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (workflowExecutionBuilder_ == null) { - if (workflowExecution_ != null) { - workflowExecution_ = - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.newBuilder(workflowExecution_).mergeFrom(value).buildPartial(); - } else { - workflowExecution_ = value; - } - onChanged(); - } else { - workflowExecutionBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public Builder clearWorkflowExecution() { - if (workflowExecutionBuilder_ == null) { - workflowExecution_ = null; - onChanged(); - } else { - workflowExecution_ = null; - workflowExecutionBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder getWorkflowExecutionBuilder() { - - onChanged(); - return getWorkflowExecutionFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getWorkflowExecutionOrBuilder() { - if (workflowExecutionBuilder_ != null) { - return workflowExecutionBuilder_.getMessageOrBuilder(); - } else { - return workflowExecution_ == null ? - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : workflowExecution_; - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier workflow_execution = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> - getWorkflowExecutionFieldBuilder() { - if (workflowExecutionBuilder_ == null) { - workflowExecutionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder>( - getWorkflowExecution(), - getParentForChildren(), - isClean()); - workflowExecution_ = null; - } - return workflowExecutionBuilder_; - } - - private java.lang.Object nodeId_ = ""; - /** - * string node_id = 2; - */ - public java.lang.String getNodeId() { - java.lang.Object ref = nodeId_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - nodeId_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string node_id = 2; - */ - public com.google.protobuf.ByteString - getNodeIdBytes() { - java.lang.Object ref = nodeId_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - nodeId_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string node_id = 2; - */ - public Builder setNodeId( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - nodeId_ = value; - onChanged(); - return this; - } - /** - * string node_id = 2; - */ - public Builder clearNodeId() { - - nodeId_ = getDefaultInstance().getNodeId(); - onChanged(); - return this; - } - /** - * string node_id = 2; - */ - public Builder setNodeIdBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - nodeId_ = value; - onChanged(); - return this; - } - - private flyteidl.core.IdentifierOuterClass.Identifier taskId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> taskIdBuilder_; - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public boolean hasTaskId() { - return taskIdBuilder_ != null || taskId_ != null; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTaskId() { - if (taskIdBuilder_ == null) { - return taskId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : taskId_; - } else { - return taskIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public Builder setTaskId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (taskIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - taskId_ = value; - onChanged(); - } else { - taskIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public Builder setTaskId( - flyteidl.core.IdentifierOuterClass.Identifier.Builder builderForValue) { - if (taskIdBuilder_ == null) { - taskId_ = builderForValue.build(); - onChanged(); - } else { - taskIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public Builder mergeTaskId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (taskIdBuilder_ == null) { - if (taskId_ != null) { - taskId_ = - flyteidl.core.IdentifierOuterClass.Identifier.newBuilder(taskId_).mergeFrom(value).buildPartial(); - } else { - taskId_ = value; - } - onChanged(); - } else { - taskIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public Builder clearTaskId() { - if (taskIdBuilder_ == null) { - taskId_ = null; - onChanged(); - } else { - taskId_ = null; - taskIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public flyteidl.core.IdentifierOuterClass.Identifier.Builder getTaskIdBuilder() { - - onChanged(); - return getTaskIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTaskIdOrBuilder() { - if (taskIdBuilder_ != null) { - return taskIdBuilder_.getMessageOrBuilder(); - } else { - return taskId_ == null ? - flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : taskId_; - } - } - /** - * .flyteidl.core.Identifier task_id = 3; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> - getTaskIdFieldBuilder() { - if (taskIdBuilder_ == null) { - taskIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder>( - getTaskId(), - getParentForChildren(), - isClean()); - taskId_ = null; - } - return taskIdBuilder_; - } - - private int retryAttempt_ ; - /** - * uint32 retry_attempt = 4; - */ - public int getRetryAttempt() { - return retryAttempt_; - } - /** - * uint32 retry_attempt = 4; - */ - public Builder setRetryAttempt(int value) { - - retryAttempt_ = value; - onChanged(); - return this; - } - /** - * uint32 retry_attempt = 4; - */ - public Builder clearRetryAttempt() { - - retryAttempt_ = 0; - onChanged(); - return this; - } - - private java.lang.Object principal_ = ""; - /** - *
-       * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-       * 
- * - * string principal = 5; - */ - public java.lang.String getPrincipal() { - java.lang.Object ref = principal_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - principal_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - *
-       * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-       * 
- * - * string principal = 5; - */ - public com.google.protobuf.ByteString - getPrincipalBytes() { - java.lang.Object ref = principal_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - principal_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - *
-       * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-       * 
- * - * string principal = 5; - */ - public Builder setPrincipal( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - principal_ = value; - onChanged(); - return this; - } - /** - *
-       * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-       * 
- * - * string principal = 5; - */ - public Builder clearPrincipal() { - - principal_ = getDefaultInstance().getPrincipal(); - onChanged(); - return this; - } - /** - *
-       * Uploads, either from the UI or from the CLI, or FlyteRemote, will have this.
-       * 
- * - * string principal = 5; - */ - public Builder setPrincipalBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - principal_ = value; - onChanged(); - return this; - } - @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.artifact.ArtifactSource) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactSource) - private static final flyteidl.artifact.Artifacts.ArtifactSource DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ArtifactSource(); - } - - public static flyteidl.artifact.Artifacts.ArtifactSource getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ArtifactSource parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ArtifactSource(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.artifact.Artifacts.ArtifactSource getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ArtifactSpecOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ArtifactSpec) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.Literal value = 1; - */ - boolean hasValue(); - /** - * .flyteidl.core.Literal value = 1; - */ - flyteidl.core.Literals.Literal getValue(); - /** - * .flyteidl.core.Literal value = 1; - */ - flyteidl.core.Literals.LiteralOrBuilder getValueOrBuilder(); - - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - boolean hasType(); - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - flyteidl.core.Types.LiteralType getType(); - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - flyteidl.core.Types.LiteralTypeOrBuilder getTypeOrBuilder(); - - /** - * string short_description = 3; - */ - java.lang.String getShortDescription(); - /** - * string short_description = 3; - */ - com.google.protobuf.ByteString - getShortDescriptionBytes(); - - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - boolean hasUserMetadata(); - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - com.google.protobuf.Any getUserMetadata(); - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - com.google.protobuf.AnyOrBuilder getUserMetadataOrBuilder(); - - /** - * string metadata_type = 5; - */ - java.lang.String getMetadataType(); - /** - * string metadata_type = 5; - */ - com.google.protobuf.ByteString - getMetadataTypeBytes(); - - /** - * .google.protobuf.Timestamp created_at = 6; - */ - boolean hasCreatedAt(); - /** - * .google.protobuf.Timestamp created_at = 6; - */ - com.google.protobuf.Timestamp getCreatedAt(); - /** - * .google.protobuf.Timestamp created_at = 6; - */ - com.google.protobuf.TimestampOrBuilder getCreatedAtOrBuilder(); - - /** - * string file_format = 7; - */ - java.lang.String getFileFormat(); - /** - * string file_format = 7; - */ - com.google.protobuf.ByteString - getFileFormatBytes(); - } - /** - * Protobuf type {@code flyteidl.artifact.ArtifactSpec} - */ - public static final class ArtifactSpec extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ArtifactSpec) - ArtifactSpecOrBuilder { - private static final long serialVersionUID = 0L; - // Use ArtifactSpec.newBuilder() to construct. - private ArtifactSpec(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ArtifactSpec() { - shortDescription_ = ""; - metadataType_ = ""; - fileFormat_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ArtifactSpec( - 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.Literals.Literal.Builder subBuilder = null; - if (value_ != null) { - subBuilder = value_.toBuilder(); - } - value_ = input.readMessage(flyteidl.core.Literals.Literal.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(value_); - value_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.core.Types.LiteralType.Builder subBuilder = null; - if (type_ != null) { - subBuilder = type_.toBuilder(); - } - type_ = input.readMessage(flyteidl.core.Types.LiteralType.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(type_); - type_ = subBuilder.buildPartial(); - } - - break; - } - case 26: { - java.lang.String s = input.readStringRequireUtf8(); - - shortDescription_ = s; - break; - } - case 34: { - com.google.protobuf.Any.Builder subBuilder = null; - if (userMetadata_ != null) { - subBuilder = userMetadata_.toBuilder(); - } - userMetadata_ = input.readMessage(com.google.protobuf.Any.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(userMetadata_); - userMetadata_ = subBuilder.buildPartial(); - } - - break; - } - case 42: { - java.lang.String s = input.readStringRequireUtf8(); - - metadataType_ = s; - break; - } - case 50: { - com.google.protobuf.Timestamp.Builder subBuilder = null; - if (createdAt_ != null) { - subBuilder = createdAt_.toBuilder(); - } - createdAt_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(createdAt_); - createdAt_ = subBuilder.buildPartial(); - } - - break; - } - case 58: { - java.lang.String s = input.readStringRequireUtf8(); - - fileFormat_ = s; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSpec_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSpec_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactSpec.class, flyteidl.artifact.Artifacts.ArtifactSpec.Builder.class); - } - - public static final int VALUE_FIELD_NUMBER = 1; - private flyteidl.core.Literals.Literal value_; - /** - * .flyteidl.core.Literal value = 1; - */ - public boolean hasValue() { - return value_ != null; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public flyteidl.core.Literals.Literal getValue() { - return value_ == null ? flyteidl.core.Literals.Literal.getDefaultInstance() : value_; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public flyteidl.core.Literals.LiteralOrBuilder getValueOrBuilder() { - return getValue(); - } - - public static final int TYPE_FIELD_NUMBER = 2; - private flyteidl.core.Types.LiteralType type_; - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public boolean hasType() { - return type_ != null; - } - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public flyteidl.core.Types.LiteralType getType() { - return type_ == null ? flyteidl.core.Types.LiteralType.getDefaultInstance() : type_; - } - /** - *
-     * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-     * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-     * type to all Literals is a lot of work.
-     * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public flyteidl.core.Types.LiteralTypeOrBuilder getTypeOrBuilder() { - return getType(); - } - - public static final int SHORT_DESCRIPTION_FIELD_NUMBER = 3; - private volatile java.lang.Object shortDescription_; - /** - * string short_description = 3; - */ - public java.lang.String getShortDescription() { - java.lang.Object ref = shortDescription_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - shortDescription_ = s; - return s; - } - } - /** - * string short_description = 3; - */ - public com.google.protobuf.ByteString - getShortDescriptionBytes() { - java.lang.Object ref = shortDescription_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - shortDescription_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int USER_METADATA_FIELD_NUMBER = 4; - private com.google.protobuf.Any userMetadata_; - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public boolean hasUserMetadata() { - return userMetadata_ != null; - } - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public com.google.protobuf.Any getUserMetadata() { - return userMetadata_ == null ? com.google.protobuf.Any.getDefaultInstance() : userMetadata_; - } - /** - *
-     * Additional user metadata
-     * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public com.google.protobuf.AnyOrBuilder getUserMetadataOrBuilder() { - return getUserMetadata(); - } - - public static final int METADATA_TYPE_FIELD_NUMBER = 5; - private volatile java.lang.Object metadataType_; - /** - * string metadata_type = 5; - */ - public java.lang.String getMetadataType() { - java.lang.Object ref = metadataType_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - metadataType_ = s; - return s; - } - } - /** - * string metadata_type = 5; - */ - public com.google.protobuf.ByteString - getMetadataTypeBytes() { - java.lang.Object ref = metadataType_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - metadataType_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int CREATED_AT_FIELD_NUMBER = 6; - private com.google.protobuf.Timestamp createdAt_; - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public boolean hasCreatedAt() { - return createdAt_ != null; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public com.google.protobuf.Timestamp getCreatedAt() { - return createdAt_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : createdAt_; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public com.google.protobuf.TimestampOrBuilder getCreatedAtOrBuilder() { - return getCreatedAt(); - } - - public static final int FILE_FORMAT_FIELD_NUMBER = 7; - private volatile java.lang.Object fileFormat_; - /** - * string file_format = 7; - */ - public java.lang.String getFileFormat() { - java.lang.Object ref = fileFormat_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - fileFormat_ = s; - return s; - } - } - /** - * string file_format = 7; - */ - public com.google.protobuf.ByteString - getFileFormatBytes() { - java.lang.Object ref = fileFormat_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - fileFormat_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - 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 (value_ != null) { - output.writeMessage(1, getValue()); - } - if (type_ != null) { - output.writeMessage(2, getType()); - } - if (!getShortDescriptionBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 3, shortDescription_); - } - if (userMetadata_ != null) { - output.writeMessage(4, getUserMetadata()); - } - if (!getMetadataTypeBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 5, metadataType_); - } - if (createdAt_ != null) { - output.writeMessage(6, getCreatedAt()); - } - if (!getFileFormatBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 7, fileFormat_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (value_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getValue()); - } - if (type_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getType()); - } - if (!getShortDescriptionBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, shortDescription_); - } - if (userMetadata_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, getUserMetadata()); - } - if (!getMetadataTypeBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, metadataType_); - } - if (createdAt_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, getCreatedAt()); - } - if (!getFileFormatBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(7, fileFormat_); - } - 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.artifact.Artifacts.ArtifactSpec)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ArtifactSpec other = (flyteidl.artifact.Artifacts.ArtifactSpec) obj; - - if (hasValue() != other.hasValue()) return false; - if (hasValue()) { - if (!getValue() - .equals(other.getValue())) return false; - } - if (hasType() != other.hasType()) return false; - if (hasType()) { - if (!getType() - .equals(other.getType())) return false; - } - if (!getShortDescription() - .equals(other.getShortDescription())) return false; - if (hasUserMetadata() != other.hasUserMetadata()) return false; - if (hasUserMetadata()) { - if (!getUserMetadata() - .equals(other.getUserMetadata())) return false; - } - if (!getMetadataType() - .equals(other.getMetadataType())) return false; - if (hasCreatedAt() != other.hasCreatedAt()) return false; - if (hasCreatedAt()) { - if (!getCreatedAt() - .equals(other.getCreatedAt())) return false; - } - if (!getFileFormat() - .equals(other.getFileFormat())) 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 (hasValue()) { - hash = (37 * hash) + VALUE_FIELD_NUMBER; - hash = (53 * hash) + getValue().hashCode(); - } - if (hasType()) { - hash = (37 * hash) + TYPE_FIELD_NUMBER; - hash = (53 * hash) + getType().hashCode(); - } - hash = (37 * hash) + SHORT_DESCRIPTION_FIELD_NUMBER; - hash = (53 * hash) + getShortDescription().hashCode(); - if (hasUserMetadata()) { - hash = (37 * hash) + USER_METADATA_FIELD_NUMBER; - hash = (53 * hash) + getUserMetadata().hashCode(); - } - hash = (37 * hash) + METADATA_TYPE_FIELD_NUMBER; - hash = (53 * hash) + getMetadataType().hashCode(); - if (hasCreatedAt()) { - hash = (37 * hash) + CREATED_AT_FIELD_NUMBER; - hash = (53 * hash) + getCreatedAt().hashCode(); - } - hash = (37 * hash) + FILE_FORMAT_FIELD_NUMBER; - hash = (53 * hash) + getFileFormat().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec 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.artifact.Artifacts.ArtifactSpec parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec 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.artifact.Artifacts.ArtifactSpec parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactSpec 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.artifact.Artifacts.ArtifactSpec 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.artifact.ArtifactSpec} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ArtifactSpec) - flyteidl.artifact.Artifacts.ArtifactSpecOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSpec_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSpec_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactSpec.class, flyteidl.artifact.Artifacts.ArtifactSpec.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ArtifactSpec.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 (valueBuilder_ == null) { - value_ = null; - } else { - value_ = null; - valueBuilder_ = null; - } - if (typeBuilder_ == null) { - type_ = null; - } else { - type_ = null; - typeBuilder_ = null; - } - shortDescription_ = ""; - - if (userMetadataBuilder_ == null) { - userMetadata_ = null; - } else { - userMetadata_ = null; - userMetadataBuilder_ = null; - } - metadataType_ = ""; - - if (createdAtBuilder_ == null) { - createdAt_ = null; - } else { - createdAt_ = null; - createdAtBuilder_ = null; - } - fileFormat_ = ""; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactSpec_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSpec getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSpec build() { - flyteidl.artifact.Artifacts.ArtifactSpec result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactSpec buildPartial() { - flyteidl.artifact.Artifacts.ArtifactSpec result = new flyteidl.artifact.Artifacts.ArtifactSpec(this); - if (valueBuilder_ == null) { - result.value_ = value_; - } else { - result.value_ = valueBuilder_.build(); - } - if (typeBuilder_ == null) { - result.type_ = type_; - } else { - result.type_ = typeBuilder_.build(); - } - result.shortDescription_ = shortDescription_; - if (userMetadataBuilder_ == null) { - result.userMetadata_ = userMetadata_; - } else { - result.userMetadata_ = userMetadataBuilder_.build(); - } - result.metadataType_ = metadataType_; - if (createdAtBuilder_ == null) { - result.createdAt_ = createdAt_; - } else { - result.createdAt_ = createdAtBuilder_.build(); - } - result.fileFormat_ = fileFormat_; - 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.artifact.Artifacts.ArtifactSpec) { - return mergeFrom((flyteidl.artifact.Artifacts.ArtifactSpec)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ArtifactSpec other) { - if (other == flyteidl.artifact.Artifacts.ArtifactSpec.getDefaultInstance()) return this; - if (other.hasValue()) { - mergeValue(other.getValue()); - } - if (other.hasType()) { - mergeType(other.getType()); - } - if (!other.getShortDescription().isEmpty()) { - shortDescription_ = other.shortDescription_; - onChanged(); - } - if (other.hasUserMetadata()) { - mergeUserMetadata(other.getUserMetadata()); - } - if (!other.getMetadataType().isEmpty()) { - metadataType_ = other.metadataType_; - onChanged(); - } - if (other.hasCreatedAt()) { - mergeCreatedAt(other.getCreatedAt()); - } - if (!other.getFileFormat().isEmpty()) { - fileFormat_ = other.fileFormat_; - onChanged(); - } - 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.artifact.Artifacts.ArtifactSpec parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ArtifactSpec) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.Literals.Literal value_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Literals.Literal, flyteidl.core.Literals.Literal.Builder, flyteidl.core.Literals.LiteralOrBuilder> valueBuilder_; - /** - * .flyteidl.core.Literal value = 1; - */ - public boolean hasValue() { - return valueBuilder_ != null || value_ != null; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public flyteidl.core.Literals.Literal getValue() { - if (valueBuilder_ == null) { - return value_ == null ? flyteidl.core.Literals.Literal.getDefaultInstance() : value_; - } else { - return valueBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.Literal value = 1; - */ - public Builder setValue(flyteidl.core.Literals.Literal value) { - if (valueBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - value_ = value; - onChanged(); - } else { - valueBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public Builder setValue( - flyteidl.core.Literals.Literal.Builder builderForValue) { - if (valueBuilder_ == null) { - value_ = builderForValue.build(); - onChanged(); - } else { - valueBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public Builder mergeValue(flyteidl.core.Literals.Literal value) { - if (valueBuilder_ == null) { - if (value_ != null) { - value_ = - flyteidl.core.Literals.Literal.newBuilder(value_).mergeFrom(value).buildPartial(); - } else { - value_ = value; - } - onChanged(); - } else { - valueBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public Builder clearValue() { - if (valueBuilder_ == null) { - value_ = null; - onChanged(); - } else { - value_ = null; - valueBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.Literal value = 1; - */ - public flyteidl.core.Literals.Literal.Builder getValueBuilder() { - - onChanged(); - return getValueFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.Literal value = 1; - */ - public flyteidl.core.Literals.LiteralOrBuilder getValueOrBuilder() { - if (valueBuilder_ != null) { - return valueBuilder_.getMessageOrBuilder(); - } else { - return value_ == null ? - flyteidl.core.Literals.Literal.getDefaultInstance() : value_; - } - } - /** - * .flyteidl.core.Literal value = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Literals.Literal, flyteidl.core.Literals.Literal.Builder, flyteidl.core.Literals.LiteralOrBuilder> - getValueFieldBuilder() { - if (valueBuilder_ == null) { - valueBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Literals.Literal, flyteidl.core.Literals.Literal.Builder, flyteidl.core.Literals.LiteralOrBuilder>( - getValue(), - getParentForChildren(), - isClean()); - value_ = null; - } - return valueBuilder_; - } - - private flyteidl.core.Types.LiteralType type_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Types.LiteralType, flyteidl.core.Types.LiteralType.Builder, flyteidl.core.Types.LiteralTypeOrBuilder> typeBuilder_; - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public boolean hasType() { - return typeBuilder_ != null || type_ != null; - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public flyteidl.core.Types.LiteralType getType() { - if (typeBuilder_ == null) { - return type_ == null ? flyteidl.core.Types.LiteralType.getDefaultInstance() : type_; - } else { - return typeBuilder_.getMessage(); - } - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public Builder setType(flyteidl.core.Types.LiteralType value) { - if (typeBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - type_ = value; - onChanged(); - } else { - typeBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public Builder setType( - flyteidl.core.Types.LiteralType.Builder builderForValue) { - if (typeBuilder_ == null) { - type_ = builderForValue.build(); - onChanged(); - } else { - typeBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public Builder mergeType(flyteidl.core.Types.LiteralType value) { - if (typeBuilder_ == null) { - if (type_ != null) { - type_ = - flyteidl.core.Types.LiteralType.newBuilder(type_).mergeFrom(value).buildPartial(); - } else { - type_ = value; - } - onChanged(); - } else { - typeBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public Builder clearType() { - if (typeBuilder_ == null) { - type_ = null; - onChanged(); - } else { - type_ = null; - typeBuilder_ = null; - } - - return this; - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public flyteidl.core.Types.LiteralType.Builder getTypeBuilder() { - - onChanged(); - return getTypeFieldBuilder().getBuilder(); - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - public flyteidl.core.Types.LiteralTypeOrBuilder getTypeOrBuilder() { - if (typeBuilder_ != null) { - return typeBuilder_.getMessageOrBuilder(); - } else { - return type_ == null ? - flyteidl.core.Types.LiteralType.getDefaultInstance() : type_; - } - } - /** - *
-       * This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but
-       * forgets to change the name, that is okay. And the reason why this is a separate field is because adding the
-       * type to all Literals is a lot of work.
-       * 
- * - * .flyteidl.core.LiteralType type = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Types.LiteralType, flyteidl.core.Types.LiteralType.Builder, flyteidl.core.Types.LiteralTypeOrBuilder> - getTypeFieldBuilder() { - if (typeBuilder_ == null) { - typeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Types.LiteralType, flyteidl.core.Types.LiteralType.Builder, flyteidl.core.Types.LiteralTypeOrBuilder>( - getType(), - getParentForChildren(), - isClean()); - type_ = null; - } - return typeBuilder_; - } - - private java.lang.Object shortDescription_ = ""; - /** - * string short_description = 3; - */ - public java.lang.String getShortDescription() { - java.lang.Object ref = shortDescription_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - shortDescription_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string short_description = 3; - */ - public com.google.protobuf.ByteString - getShortDescriptionBytes() { - java.lang.Object ref = shortDescription_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - shortDescription_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string short_description = 3; - */ - public Builder setShortDescription( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - shortDescription_ = value; - onChanged(); - return this; - } - /** - * string short_description = 3; - */ - public Builder clearShortDescription() { - - shortDescription_ = getDefaultInstance().getShortDescription(); - onChanged(); - return this; - } - /** - * string short_description = 3; - */ - public Builder setShortDescriptionBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - shortDescription_ = value; - onChanged(); - return this; - } - - private com.google.protobuf.Any userMetadata_; - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Any, com.google.protobuf.Any.Builder, com.google.protobuf.AnyOrBuilder> userMetadataBuilder_; - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public boolean hasUserMetadata() { - return userMetadataBuilder_ != null || userMetadata_ != null; - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public com.google.protobuf.Any getUserMetadata() { - if (userMetadataBuilder_ == null) { - return userMetadata_ == null ? com.google.protobuf.Any.getDefaultInstance() : userMetadata_; - } else { - return userMetadataBuilder_.getMessage(); - } - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public Builder setUserMetadata(com.google.protobuf.Any value) { - if (userMetadataBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - userMetadata_ = value; - onChanged(); - } else { - userMetadataBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public Builder setUserMetadata( - com.google.protobuf.Any.Builder builderForValue) { - if (userMetadataBuilder_ == null) { - userMetadata_ = builderForValue.build(); - onChanged(); - } else { - userMetadataBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public Builder mergeUserMetadata(com.google.protobuf.Any value) { - if (userMetadataBuilder_ == null) { - if (userMetadata_ != null) { - userMetadata_ = - com.google.protobuf.Any.newBuilder(userMetadata_).mergeFrom(value).buildPartial(); - } else { - userMetadata_ = value; - } - onChanged(); - } else { - userMetadataBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public Builder clearUserMetadata() { - if (userMetadataBuilder_ == null) { - userMetadata_ = null; - onChanged(); - } else { - userMetadata_ = null; - userMetadataBuilder_ = null; - } - - return this; - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public com.google.protobuf.Any.Builder getUserMetadataBuilder() { - - onChanged(); - return getUserMetadataFieldBuilder().getBuilder(); - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - public com.google.protobuf.AnyOrBuilder getUserMetadataOrBuilder() { - if (userMetadataBuilder_ != null) { - return userMetadataBuilder_.getMessageOrBuilder(); - } else { - return userMetadata_ == null ? - com.google.protobuf.Any.getDefaultInstance() : userMetadata_; - } - } - /** - *
-       * Additional user metadata
-       * 
- * - * .google.protobuf.Any user_metadata = 4; - */ - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Any, com.google.protobuf.Any.Builder, com.google.protobuf.AnyOrBuilder> - getUserMetadataFieldBuilder() { - if (userMetadataBuilder_ == null) { - userMetadataBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Any, com.google.protobuf.Any.Builder, com.google.protobuf.AnyOrBuilder>( - getUserMetadata(), - getParentForChildren(), - isClean()); - userMetadata_ = null; - } - return userMetadataBuilder_; - } - - private java.lang.Object metadataType_ = ""; - /** - * string metadata_type = 5; - */ - public java.lang.String getMetadataType() { - java.lang.Object ref = metadataType_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - metadataType_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string metadata_type = 5; - */ - public com.google.protobuf.ByteString - getMetadataTypeBytes() { - java.lang.Object ref = metadataType_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - metadataType_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string metadata_type = 5; - */ - public Builder setMetadataType( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - metadataType_ = value; - onChanged(); - return this; - } - /** - * string metadata_type = 5; - */ - public Builder clearMetadataType() { - - metadataType_ = getDefaultInstance().getMetadataType(); - onChanged(); - return this; - } - /** - * string metadata_type = 5; - */ - public Builder setMetadataTypeBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - metadataType_ = value; - onChanged(); - return this; - } - - private com.google.protobuf.Timestamp createdAt_; - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> createdAtBuilder_; - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public boolean hasCreatedAt() { - return createdAtBuilder_ != null || createdAt_ != null; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public com.google.protobuf.Timestamp getCreatedAt() { - if (createdAtBuilder_ == null) { - return createdAt_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : createdAt_; - } else { - return createdAtBuilder_.getMessage(); - } - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public Builder setCreatedAt(com.google.protobuf.Timestamp value) { - if (createdAtBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - createdAt_ = value; - onChanged(); - } else { - createdAtBuilder_.setMessage(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public Builder setCreatedAt( - com.google.protobuf.Timestamp.Builder builderForValue) { - if (createdAtBuilder_ == null) { - createdAt_ = builderForValue.build(); - onChanged(); - } else { - createdAtBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public Builder mergeCreatedAt(com.google.protobuf.Timestamp value) { - if (createdAtBuilder_ == null) { - if (createdAt_ != null) { - createdAt_ = - com.google.protobuf.Timestamp.newBuilder(createdAt_).mergeFrom(value).buildPartial(); - } else { - createdAt_ = value; - } - onChanged(); - } else { - createdAtBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public Builder clearCreatedAt() { - if (createdAtBuilder_ == null) { - createdAt_ = null; - onChanged(); - } else { - createdAt_ = null; - createdAtBuilder_ = null; - } - - return this; - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public com.google.protobuf.Timestamp.Builder getCreatedAtBuilder() { - - onChanged(); - return getCreatedAtFieldBuilder().getBuilder(); - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - public com.google.protobuf.TimestampOrBuilder getCreatedAtOrBuilder() { - if (createdAtBuilder_ != null) { - return createdAtBuilder_.getMessageOrBuilder(); - } else { - return createdAt_ == null ? - com.google.protobuf.Timestamp.getDefaultInstance() : createdAt_; - } - } - /** - * .google.protobuf.Timestamp created_at = 6; - */ - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> - getCreatedAtFieldBuilder() { - if (createdAtBuilder_ == null) { - createdAtBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>( - getCreatedAt(), - getParentForChildren(), - isClean()); - createdAt_ = null; - } - return createdAtBuilder_; - } - - private java.lang.Object fileFormat_ = ""; - /** - * string file_format = 7; - */ - public java.lang.String getFileFormat() { - java.lang.Object ref = fileFormat_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - fileFormat_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string file_format = 7; - */ - public com.google.protobuf.ByteString - getFileFormatBytes() { - java.lang.Object ref = fileFormat_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - fileFormat_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string file_format = 7; - */ - public Builder setFileFormat( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - fileFormat_ = value; - onChanged(); - return this; - } - /** - * string file_format = 7; - */ - public Builder clearFileFormat() { - - fileFormat_ = getDefaultInstance().getFileFormat(); - onChanged(); - return this; - } - /** - * string file_format = 7; - */ - public Builder setFileFormatBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - fileFormat_ = value; - onChanged(); - return this; - } - @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.artifact.ArtifactSpec) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactSpec) - private static final flyteidl.artifact.Artifacts.ArtifactSpec DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ArtifactSpec(); - } - - public static flyteidl.artifact.Artifacts.ArtifactSpec getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ArtifactSpec parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ArtifactSpec(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.artifact.Artifacts.ArtifactSpec getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface CreateArtifactResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.CreateArtifactResponse) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - boolean hasArtifact(); - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - flyteidl.artifact.Artifacts.Artifact getArtifact(); - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.CreateArtifactResponse} - */ - public static final class CreateArtifactResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.CreateArtifactResponse) - CreateArtifactResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use CreateArtifactResponse.newBuilder() to construct. - private CreateArtifactResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private CreateArtifactResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private CreateArtifactResponse( - 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.artifact.Artifacts.Artifact.Builder subBuilder = null; - if (artifact_ != null) { - subBuilder = artifact_.toBuilder(); - } - artifact_ = input.readMessage(flyteidl.artifact.Artifacts.Artifact.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifact_); - artifact_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateArtifactResponse.class, flyteidl.artifact.Artifacts.CreateArtifactResponse.Builder.class); - } - - public static final int ARTIFACT_FIELD_NUMBER = 1; - private flyteidl.artifact.Artifacts.Artifact artifact_; - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public boolean hasArtifact() { - return artifact_ != null; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifact() { - return artifact_ == null ? flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder() { - return getArtifact(); - } - - 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 (artifact_ != null) { - output.writeMessage(1, getArtifact()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifact_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifact()); - } - 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.artifact.Artifacts.CreateArtifactResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.CreateArtifactResponse other = (flyteidl.artifact.Artifacts.CreateArtifactResponse) obj; - - if (hasArtifact() != other.hasArtifact()) return false; - if (hasArtifact()) { - if (!getArtifact() - .equals(other.getArtifact())) 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 (hasArtifact()) { - hash = (37 * hash) + ARTIFACT_FIELD_NUMBER; - hash = (53 * hash) + getArtifact().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse 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.artifact.Artifacts.CreateArtifactResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse 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.artifact.Artifacts.CreateArtifactResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateArtifactResponse 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.artifact.Artifacts.CreateArtifactResponse 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.artifact.CreateArtifactResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.CreateArtifactResponse) - flyteidl.artifact.Artifacts.CreateArtifactResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateArtifactResponse.class, flyteidl.artifact.Artifacts.CreateArtifactResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.CreateArtifactResponse.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 (artifactBuilder_ == null) { - artifact_ = null; - } else { - artifact_ = null; - artifactBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.CreateArtifactResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactResponse build() { - flyteidl.artifact.Artifacts.CreateArtifactResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateArtifactResponse buildPartial() { - flyteidl.artifact.Artifacts.CreateArtifactResponse result = new flyteidl.artifact.Artifacts.CreateArtifactResponse(this); - if (artifactBuilder_ == null) { - result.artifact_ = artifact_; - } else { - result.artifact_ = artifactBuilder_.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.artifact.Artifacts.CreateArtifactResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.CreateArtifactResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.CreateArtifactResponse other) { - if (other == flyteidl.artifact.Artifacts.CreateArtifactResponse.getDefaultInstance()) return this; - if (other.hasArtifact()) { - mergeArtifact(other.getArtifact()); - } - 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.artifact.Artifacts.CreateArtifactResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.CreateArtifactResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.artifact.Artifacts.Artifact artifact_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> artifactBuilder_; - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public boolean hasArtifact() { - return artifactBuilder_ != null || artifact_ != null; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifact() { - if (artifactBuilder_ == null) { - return artifact_ == null ? flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } else { - return artifactBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder setArtifact(flyteidl.artifact.Artifacts.Artifact value) { - if (artifactBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifact_ = value; - onChanged(); - } else { - artifactBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder setArtifact( - flyteidl.artifact.Artifacts.Artifact.Builder builderForValue) { - if (artifactBuilder_ == null) { - artifact_ = builderForValue.build(); - onChanged(); - } else { - artifactBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder mergeArtifact(flyteidl.artifact.Artifacts.Artifact value) { - if (artifactBuilder_ == null) { - if (artifact_ != null) { - artifact_ = - flyteidl.artifact.Artifacts.Artifact.newBuilder(artifact_).mergeFrom(value).buildPartial(); - } else { - artifact_ = value; - } - onChanged(); - } else { - artifactBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder clearArtifact() { - if (artifactBuilder_ == null) { - artifact_ = null; - onChanged(); - } else { - artifact_ = null; - artifactBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact.Builder getArtifactBuilder() { - - onChanged(); - return getArtifactFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder() { - if (artifactBuilder_ != null) { - return artifactBuilder_.getMessageOrBuilder(); - } else { - return artifact_ == null ? - flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> - getArtifactFieldBuilder() { - if (artifactBuilder_ == null) { - artifactBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder>( - getArtifact(), - getParentForChildren(), - isClean()); - artifact_ = null; - } - return artifactBuilder_; - } - @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.artifact.CreateArtifactResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateArtifactResponse) - private static final flyteidl.artifact.Artifacts.CreateArtifactResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.CreateArtifactResponse(); - } - - public static flyteidl.artifact.Artifacts.CreateArtifactResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public CreateArtifactResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new CreateArtifactResponse(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.artifact.Artifacts.CreateArtifactResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface GetArtifactRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.GetArtifactRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - boolean hasQuery(); - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - flyteidl.core.ArtifactId.ArtifactQuery getQuery(); - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - flyteidl.core.ArtifactId.ArtifactQueryOrBuilder getQueryOrBuilder(); - - /** - *
-     * If false, then long_description is not returned.
-     * 
- * - * bool details = 2; - */ - boolean getDetails(); - } - /** - * Protobuf type {@code flyteidl.artifact.GetArtifactRequest} - */ - public static final class GetArtifactRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.GetArtifactRequest) - GetArtifactRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use GetArtifactRequest.newBuilder() to construct. - private GetArtifactRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private GetArtifactRequest() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private GetArtifactRequest( - 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.ArtifactId.ArtifactQuery.Builder subBuilder = null; - if (query_ != null) { - subBuilder = query_.toBuilder(); - } - query_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactQuery.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(query_); - query_ = subBuilder.buildPartial(); - } - - break; - } - case 16: { - - details_ = input.readBool(); - 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.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.GetArtifactRequest.class, flyteidl.artifact.Artifacts.GetArtifactRequest.Builder.class); - } - - public static final int QUERY_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactQuery query_; - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public boolean hasQuery() { - return query_ != null; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public flyteidl.core.ArtifactId.ArtifactQuery getQuery() { - return query_ == null ? flyteidl.core.ArtifactId.ArtifactQuery.getDefaultInstance() : query_; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public flyteidl.core.ArtifactId.ArtifactQueryOrBuilder getQueryOrBuilder() { - return getQuery(); - } - - public static final int DETAILS_FIELD_NUMBER = 2; - private boolean details_; - /** - *
-     * If false, then long_description is not returned.
-     * 
- * - * bool details = 2; - */ - public boolean getDetails() { - return details_; - } - - 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 (query_ != null) { - output.writeMessage(1, getQuery()); - } - if (details_ != false) { - output.writeBool(2, details_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (query_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getQuery()); - } - if (details_ != false) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(2, details_); - } - 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.artifact.Artifacts.GetArtifactRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.GetArtifactRequest other = (flyteidl.artifact.Artifacts.GetArtifactRequest) obj; - - if (hasQuery() != other.hasQuery()) return false; - if (hasQuery()) { - if (!getQuery() - .equals(other.getQuery())) return false; - } - if (getDetails() - != other.getDetails()) 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 (hasQuery()) { - hash = (37 * hash) + QUERY_FIELD_NUMBER; - hash = (53 * hash) + getQuery().hashCode(); - } - hash = (37 * hash) + DETAILS_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( - getDetails()); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest 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.artifact.Artifacts.GetArtifactRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest 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.artifact.Artifacts.GetArtifactRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactRequest 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.artifact.Artifacts.GetArtifactRequest 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.artifact.GetArtifactRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.GetArtifactRequest) - flyteidl.artifact.Artifacts.GetArtifactRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.GetArtifactRequest.class, flyteidl.artifact.Artifacts.GetArtifactRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.GetArtifactRequest.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 (queryBuilder_ == null) { - query_ = null; - } else { - query_ = null; - queryBuilder_ = null; - } - details_ = false; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.GetArtifactRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactRequest build() { - flyteidl.artifact.Artifacts.GetArtifactRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactRequest buildPartial() { - flyteidl.artifact.Artifacts.GetArtifactRequest result = new flyteidl.artifact.Artifacts.GetArtifactRequest(this); - if (queryBuilder_ == null) { - result.query_ = query_; - } else { - result.query_ = queryBuilder_.build(); - } - result.details_ = details_; - 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.artifact.Artifacts.GetArtifactRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.GetArtifactRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.GetArtifactRequest other) { - if (other == flyteidl.artifact.Artifacts.GetArtifactRequest.getDefaultInstance()) return this; - if (other.hasQuery()) { - mergeQuery(other.getQuery()); - } - if (other.getDetails() != false) { - setDetails(other.getDetails()); - } - 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.artifact.Artifacts.GetArtifactRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.GetArtifactRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.ArtifactId.ArtifactQuery query_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactQuery, flyteidl.core.ArtifactId.ArtifactQuery.Builder, flyteidl.core.ArtifactId.ArtifactQueryOrBuilder> queryBuilder_; - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public boolean hasQuery() { - return queryBuilder_ != null || query_ != null; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public flyteidl.core.ArtifactId.ArtifactQuery getQuery() { - if (queryBuilder_ == null) { - return query_ == null ? flyteidl.core.ArtifactId.ArtifactQuery.getDefaultInstance() : query_; - } else { - return queryBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public Builder setQuery(flyteidl.core.ArtifactId.ArtifactQuery value) { - if (queryBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - query_ = value; - onChanged(); - } else { - queryBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public Builder setQuery( - flyteidl.core.ArtifactId.ArtifactQuery.Builder builderForValue) { - if (queryBuilder_ == null) { - query_ = builderForValue.build(); - onChanged(); - } else { - queryBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public Builder mergeQuery(flyteidl.core.ArtifactId.ArtifactQuery value) { - if (queryBuilder_ == null) { - if (query_ != null) { - query_ = - flyteidl.core.ArtifactId.ArtifactQuery.newBuilder(query_).mergeFrom(value).buildPartial(); - } else { - query_ = value; - } - onChanged(); - } else { - queryBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public Builder clearQuery() { - if (queryBuilder_ == null) { - query_ = null; - onChanged(); - } else { - query_ = null; - queryBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public flyteidl.core.ArtifactId.ArtifactQuery.Builder getQueryBuilder() { - - onChanged(); - return getQueryFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - public flyteidl.core.ArtifactId.ArtifactQueryOrBuilder getQueryOrBuilder() { - if (queryBuilder_ != null) { - return queryBuilder_.getMessageOrBuilder(); - } else { - return query_ == null ? - flyteidl.core.ArtifactId.ArtifactQuery.getDefaultInstance() : query_; - } - } - /** - * .flyteidl.core.ArtifactQuery query = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactQuery, flyteidl.core.ArtifactId.ArtifactQuery.Builder, flyteidl.core.ArtifactId.ArtifactQueryOrBuilder> - getQueryFieldBuilder() { - if (queryBuilder_ == null) { - queryBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactQuery, flyteidl.core.ArtifactId.ArtifactQuery.Builder, flyteidl.core.ArtifactId.ArtifactQueryOrBuilder>( - getQuery(), - getParentForChildren(), - isClean()); - query_ = null; - } - return queryBuilder_; - } - - private boolean details_ ; - /** - *
-       * If false, then long_description is not returned.
-       * 
- * - * bool details = 2; - */ - public boolean getDetails() { - return details_; - } - /** - *
-       * If false, then long_description is not returned.
-       * 
- * - * bool details = 2; - */ - public Builder setDetails(boolean value) { - - details_ = value; - onChanged(); - return this; - } - /** - *
-       * If false, then long_description is not returned.
-       * 
- * - * bool details = 2; - */ - public Builder clearDetails() { - - details_ = false; - onChanged(); - return this; - } - @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.artifact.GetArtifactRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.GetArtifactRequest) - private static final flyteidl.artifact.Artifacts.GetArtifactRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.GetArtifactRequest(); - } - - public static flyteidl.artifact.Artifacts.GetArtifactRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public GetArtifactRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new GetArtifactRequest(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.artifact.Artifacts.GetArtifactRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface GetArtifactResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.GetArtifactResponse) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - boolean hasArtifact(); - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - flyteidl.artifact.Artifacts.Artifact getArtifact(); - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.GetArtifactResponse} - */ - public static final class GetArtifactResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.GetArtifactResponse) - GetArtifactResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use GetArtifactResponse.newBuilder() to construct. - private GetArtifactResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private GetArtifactResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private GetArtifactResponse( - 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.artifact.Artifacts.Artifact.Builder subBuilder = null; - if (artifact_ != null) { - subBuilder = artifact_.toBuilder(); - } - artifact_ = input.readMessage(flyteidl.artifact.Artifacts.Artifact.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifact_); - artifact_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.GetArtifactResponse.class, flyteidl.artifact.Artifacts.GetArtifactResponse.Builder.class); - } - - public static final int ARTIFACT_FIELD_NUMBER = 1; - private flyteidl.artifact.Artifacts.Artifact artifact_; - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public boolean hasArtifact() { - return artifact_ != null; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifact() { - return artifact_ == null ? flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder() { - return getArtifact(); - } - - 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 (artifact_ != null) { - output.writeMessage(1, getArtifact()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifact_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifact()); - } - 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.artifact.Artifacts.GetArtifactResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.GetArtifactResponse other = (flyteidl.artifact.Artifacts.GetArtifactResponse) obj; - - if (hasArtifact() != other.hasArtifact()) return false; - if (hasArtifact()) { - if (!getArtifact() - .equals(other.getArtifact())) 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 (hasArtifact()) { - hash = (37 * hash) + ARTIFACT_FIELD_NUMBER; - hash = (53 * hash) + getArtifact().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse 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.artifact.Artifacts.GetArtifactResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse 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.artifact.Artifacts.GetArtifactResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.GetArtifactResponse 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.artifact.Artifacts.GetArtifactResponse 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.artifact.GetArtifactResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.GetArtifactResponse) - flyteidl.artifact.Artifacts.GetArtifactResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.GetArtifactResponse.class, flyteidl.artifact.Artifacts.GetArtifactResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.GetArtifactResponse.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 (artifactBuilder_ == null) { - artifact_ = null; - } else { - artifact_ = null; - artifactBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_GetArtifactResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.GetArtifactResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactResponse build() { - flyteidl.artifact.Artifacts.GetArtifactResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.GetArtifactResponse buildPartial() { - flyteidl.artifact.Artifacts.GetArtifactResponse result = new flyteidl.artifact.Artifacts.GetArtifactResponse(this); - if (artifactBuilder_ == null) { - result.artifact_ = artifact_; - } else { - result.artifact_ = artifactBuilder_.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.artifact.Artifacts.GetArtifactResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.GetArtifactResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.GetArtifactResponse other) { - if (other == flyteidl.artifact.Artifacts.GetArtifactResponse.getDefaultInstance()) return this; - if (other.hasArtifact()) { - mergeArtifact(other.getArtifact()); - } - 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.artifact.Artifacts.GetArtifactResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.GetArtifactResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.artifact.Artifacts.Artifact artifact_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> artifactBuilder_; - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public boolean hasArtifact() { - return artifactBuilder_ != null || artifact_ != null; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifact() { - if (artifactBuilder_ == null) { - return artifact_ == null ? flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } else { - return artifactBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder setArtifact(flyteidl.artifact.Artifacts.Artifact value) { - if (artifactBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifact_ = value; - onChanged(); - } else { - artifactBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder setArtifact( - flyteidl.artifact.Artifacts.Artifact.Builder builderForValue) { - if (artifactBuilder_ == null) { - artifact_ = builderForValue.build(); - onChanged(); - } else { - artifactBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder mergeArtifact(flyteidl.artifact.Artifacts.Artifact value) { - if (artifactBuilder_ == null) { - if (artifact_ != null) { - artifact_ = - flyteidl.artifact.Artifacts.Artifact.newBuilder(artifact_).mergeFrom(value).buildPartial(); - } else { - artifact_ = value; - } - onChanged(); - } else { - artifactBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public Builder clearArtifact() { - if (artifactBuilder_ == null) { - artifact_ = null; - onChanged(); - } else { - artifact_ = null; - artifactBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.Artifact.Builder getArtifactBuilder() { - - onChanged(); - return getArtifactFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactOrBuilder() { - if (artifactBuilder_ != null) { - return artifactBuilder_.getMessageOrBuilder(); - } else { - return artifact_ == null ? - flyteidl.artifact.Artifacts.Artifact.getDefaultInstance() : artifact_; - } - } - /** - * .flyteidl.artifact.Artifact artifact = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> - getArtifactFieldBuilder() { - if (artifactBuilder_ == null) { - artifactBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder>( - getArtifact(), - getParentForChildren(), - isClean()); - artifact_ = null; - } - return artifactBuilder_; - } - @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.artifact.GetArtifactResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.GetArtifactResponse) - private static final flyteidl.artifact.Artifacts.GetArtifactResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.GetArtifactResponse(); - } - - public static flyteidl.artifact.Artifacts.GetArtifactResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public GetArtifactResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new GetArtifactResponse(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.artifact.Artifacts.GetArtifactResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface SearchOptionsOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.SearchOptions) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * If true, this means a strict partition search. meaning if you don't specify the partition
-     * field, that will mean, non-partitioned, rather than any partition.
-     * 
- * - * bool strict_partitions = 1; - */ - boolean getStrictPartitions(); - - /** - *
-     * If true, only one artifact per key will be returned. It will be the latest one by creation time.
-     * 
- * - * bool latest_by_key = 2; - */ - boolean getLatestByKey(); - } - /** - * Protobuf type {@code flyteidl.artifact.SearchOptions} - */ - public static final class SearchOptions extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.SearchOptions) - SearchOptionsOrBuilder { - private static final long serialVersionUID = 0L; - // Use SearchOptions.newBuilder() to construct. - private SearchOptions(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private SearchOptions() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private SearchOptions( - 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 8: { - - strictPartitions_ = input.readBool(); - break; - } - case 16: { - - latestByKey_ = input.readBool(); - 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.artifact.Artifacts.internal_static_flyteidl_artifact_SearchOptions_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchOptions_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchOptions.class, flyteidl.artifact.Artifacts.SearchOptions.Builder.class); - } - - public static final int STRICT_PARTITIONS_FIELD_NUMBER = 1; - private boolean strictPartitions_; - /** - *
-     * If true, this means a strict partition search. meaning if you don't specify the partition
-     * field, that will mean, non-partitioned, rather than any partition.
-     * 
- * - * bool strict_partitions = 1; - */ - public boolean getStrictPartitions() { - return strictPartitions_; - } - - public static final int LATEST_BY_KEY_FIELD_NUMBER = 2; - private boolean latestByKey_; - /** - *
-     * If true, only one artifact per key will be returned. It will be the latest one by creation time.
-     * 
- * - * bool latest_by_key = 2; - */ - public boolean getLatestByKey() { - return latestByKey_; - } - - 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 (strictPartitions_ != false) { - output.writeBool(1, strictPartitions_); - } - if (latestByKey_ != false) { - output.writeBool(2, latestByKey_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (strictPartitions_ != false) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(1, strictPartitions_); - } - if (latestByKey_ != false) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(2, latestByKey_); - } - 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.artifact.Artifacts.SearchOptions)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.SearchOptions other = (flyteidl.artifact.Artifacts.SearchOptions) obj; - - if (getStrictPartitions() - != other.getStrictPartitions()) return false; - if (getLatestByKey() - != other.getLatestByKey()) 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(); - hash = (37 * hash) + STRICT_PARTITIONS_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( - getStrictPartitions()); - hash = (37 * hash) + LATEST_BY_KEY_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( - getLatestByKey()); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchOptions parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchOptions 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.artifact.Artifacts.SearchOptions parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchOptions 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.artifact.Artifacts.SearchOptions parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchOptions 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.artifact.Artifacts.SearchOptions 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.artifact.SearchOptions} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.SearchOptions) - flyteidl.artifact.Artifacts.SearchOptionsOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchOptions_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchOptions_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchOptions.class, flyteidl.artifact.Artifacts.SearchOptions.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.SearchOptions.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(); - strictPartitions_ = false; - - latestByKey_ = false; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchOptions_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchOptions getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.SearchOptions.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchOptions build() { - flyteidl.artifact.Artifacts.SearchOptions result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchOptions buildPartial() { - flyteidl.artifact.Artifacts.SearchOptions result = new flyteidl.artifact.Artifacts.SearchOptions(this); - result.strictPartitions_ = strictPartitions_; - result.latestByKey_ = latestByKey_; - 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.artifact.Artifacts.SearchOptions) { - return mergeFrom((flyteidl.artifact.Artifacts.SearchOptions)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.SearchOptions other) { - if (other == flyteidl.artifact.Artifacts.SearchOptions.getDefaultInstance()) return this; - if (other.getStrictPartitions() != false) { - setStrictPartitions(other.getStrictPartitions()); - } - if (other.getLatestByKey() != false) { - setLatestByKey(other.getLatestByKey()); - } - 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.artifact.Artifacts.SearchOptions parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.SearchOptions) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private boolean strictPartitions_ ; - /** - *
-       * If true, this means a strict partition search. meaning if you don't specify the partition
-       * field, that will mean, non-partitioned, rather than any partition.
-       * 
- * - * bool strict_partitions = 1; - */ - public boolean getStrictPartitions() { - return strictPartitions_; - } - /** - *
-       * If true, this means a strict partition search. meaning if you don't specify the partition
-       * field, that will mean, non-partitioned, rather than any partition.
-       * 
- * - * bool strict_partitions = 1; - */ - public Builder setStrictPartitions(boolean value) { - - strictPartitions_ = value; - onChanged(); - return this; - } - /** - *
-       * If true, this means a strict partition search. meaning if you don't specify the partition
-       * field, that will mean, non-partitioned, rather than any partition.
-       * 
- * - * bool strict_partitions = 1; - */ - public Builder clearStrictPartitions() { - - strictPartitions_ = false; - onChanged(); - return this; - } - - private boolean latestByKey_ ; - /** - *
-       * If true, only one artifact per key will be returned. It will be the latest one by creation time.
-       * 
- * - * bool latest_by_key = 2; - */ - public boolean getLatestByKey() { - return latestByKey_; - } - /** - *
-       * If true, only one artifact per key will be returned. It will be the latest one by creation time.
-       * 
- * - * bool latest_by_key = 2; - */ - public Builder setLatestByKey(boolean value) { - - latestByKey_ = value; - onChanged(); - return this; - } - /** - *
-       * If true, only one artifact per key will be returned. It will be the latest one by creation time.
-       * 
- * - * bool latest_by_key = 2; - */ - public Builder clearLatestByKey() { - - latestByKey_ = false; - onChanged(); - return this; - } - @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.artifact.SearchOptions) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchOptions) - private static final flyteidl.artifact.Artifacts.SearchOptions DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.SearchOptions(); - } - - public static flyteidl.artifact.Artifacts.SearchOptions getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SearchOptions parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new SearchOptions(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.artifact.Artifacts.SearchOptions getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface SearchArtifactsRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.SearchArtifactsRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - boolean hasArtifactKey(); - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - flyteidl.core.ArtifactId.ArtifactKey getArtifactKey(); - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder(); - - /** - * .flyteidl.core.Partitions partitions = 2; - */ - boolean hasPartitions(); - /** - * .flyteidl.core.Partitions partitions = 2; - */ - flyteidl.core.ArtifactId.Partitions getPartitions(); - /** - * .flyteidl.core.Partitions partitions = 2; - */ - flyteidl.core.ArtifactId.PartitionsOrBuilder getPartitionsOrBuilder(); - - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - boolean hasTimePartitionValue(); - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - com.google.protobuf.Timestamp getTimePartitionValue(); - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder(); - - /** - * string principal = 4; - */ - java.lang.String getPrincipal(); - /** - * string principal = 4; - */ - com.google.protobuf.ByteString - getPrincipalBytes(); - - /** - * string version = 5; - */ - java.lang.String getVersion(); - /** - * string version = 5; - */ - com.google.protobuf.ByteString - getVersionBytes(); - - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - boolean hasOptions(); - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - flyteidl.artifact.Artifacts.SearchOptions getOptions(); - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - flyteidl.artifact.Artifacts.SearchOptionsOrBuilder getOptionsOrBuilder(); - - /** - * string token = 7; - */ - java.lang.String getToken(); - /** - * string token = 7; - */ - com.google.protobuf.ByteString - getTokenBytes(); - - /** - * int32 limit = 8; - */ - int getLimit(); - } - /** - * Protobuf type {@code flyteidl.artifact.SearchArtifactsRequest} - */ - public static final class SearchArtifactsRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.SearchArtifactsRequest) - SearchArtifactsRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use SearchArtifactsRequest.newBuilder() to construct. - private SearchArtifactsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private SearchArtifactsRequest() { - principal_ = ""; - version_ = ""; - token_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private SearchArtifactsRequest( - 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.ArtifactId.ArtifactKey.Builder subBuilder = null; - if (artifactKey_ != null) { - subBuilder = artifactKey_.toBuilder(); - } - artifactKey_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactKey.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifactKey_); - artifactKey_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.core.ArtifactId.Partitions.Builder subBuilder = null; - if (partitions_ != null) { - subBuilder = partitions_.toBuilder(); - } - partitions_ = input.readMessage(flyteidl.core.ArtifactId.Partitions.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(partitions_); - partitions_ = subBuilder.buildPartial(); - } - - break; - } - case 26: { - com.google.protobuf.Timestamp.Builder subBuilder = null; - if (timePartitionValue_ != null) { - subBuilder = timePartitionValue_.toBuilder(); - } - timePartitionValue_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(timePartitionValue_); - timePartitionValue_ = subBuilder.buildPartial(); - } - - break; - } - case 34: { - java.lang.String s = input.readStringRequireUtf8(); - - principal_ = s; - break; - } - case 42: { - java.lang.String s = input.readStringRequireUtf8(); - - version_ = s; - break; - } - case 50: { - flyteidl.artifact.Artifacts.SearchOptions.Builder subBuilder = null; - if (options_ != null) { - subBuilder = options_.toBuilder(); - } - options_ = input.readMessage(flyteidl.artifact.Artifacts.SearchOptions.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(options_); - options_ = subBuilder.buildPartial(); - } - - break; - } - case 58: { - java.lang.String s = input.readStringRequireUtf8(); - - token_ = s; - break; - } - case 64: { - - limit_ = input.readInt32(); - 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.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchArtifactsRequest.class, flyteidl.artifact.Artifacts.SearchArtifactsRequest.Builder.class); - } - - public static final int ARTIFACT_KEY_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactKey artifactKey_; - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public boolean hasArtifactKey() { - return artifactKey_ != null; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey getArtifactKey() { - return artifactKey_ == null ? flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder() { - return getArtifactKey(); - } - - public static final int PARTITIONS_FIELD_NUMBER = 2; - private flyteidl.core.ArtifactId.Partitions partitions_; - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public boolean hasPartitions() { - return partitions_ != null; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public flyteidl.core.ArtifactId.Partitions getPartitions() { - return partitions_ == null ? flyteidl.core.ArtifactId.Partitions.getDefaultInstance() : partitions_; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public flyteidl.core.ArtifactId.PartitionsOrBuilder getPartitionsOrBuilder() { - return getPartitions(); - } - - public static final int TIME_PARTITION_VALUE_FIELD_NUMBER = 3; - private com.google.protobuf.Timestamp timePartitionValue_; - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public boolean hasTimePartitionValue() { - return timePartitionValue_ != null; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public com.google.protobuf.Timestamp getTimePartitionValue() { - return timePartitionValue_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder() { - return getTimePartitionValue(); - } - - public static final int PRINCIPAL_FIELD_NUMBER = 4; - private volatile java.lang.Object principal_; - /** - * string principal = 4; - */ - public java.lang.String getPrincipal() { - java.lang.Object ref = principal_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - principal_ = s; - return s; - } - } - /** - * string principal = 4; - */ - public com.google.protobuf.ByteString - getPrincipalBytes() { - java.lang.Object ref = principal_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - principal_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int VERSION_FIELD_NUMBER = 5; - private volatile java.lang.Object version_; - /** - * string version = 5; - */ - public java.lang.String getVersion() { - java.lang.Object ref = version_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - version_ = s; - return s; - } - } - /** - * string version = 5; - */ - public com.google.protobuf.ByteString - getVersionBytes() { - java.lang.Object ref = version_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - version_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int OPTIONS_FIELD_NUMBER = 6; - private flyteidl.artifact.Artifacts.SearchOptions options_; - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public boolean hasOptions() { - return options_ != null; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public flyteidl.artifact.Artifacts.SearchOptions getOptions() { - return options_ == null ? flyteidl.artifact.Artifacts.SearchOptions.getDefaultInstance() : options_; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public flyteidl.artifact.Artifacts.SearchOptionsOrBuilder getOptionsOrBuilder() { - return getOptions(); - } - - public static final int TOKEN_FIELD_NUMBER = 7; - private volatile java.lang.Object token_; - /** - * string token = 7; - */ - public java.lang.String getToken() { - java.lang.Object ref = token_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - token_ = s; - return s; - } - } - /** - * string token = 7; - */ - public com.google.protobuf.ByteString - getTokenBytes() { - java.lang.Object ref = token_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - token_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int LIMIT_FIELD_NUMBER = 8; - private int limit_; - /** - * int32 limit = 8; - */ - public int getLimit() { - return limit_; - } - - 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 (artifactKey_ != null) { - output.writeMessage(1, getArtifactKey()); - } - if (partitions_ != null) { - output.writeMessage(2, getPartitions()); - } - if (timePartitionValue_ != null) { - output.writeMessage(3, getTimePartitionValue()); - } - if (!getPrincipalBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 4, principal_); - } - if (!getVersionBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 5, version_); - } - if (options_ != null) { - output.writeMessage(6, getOptions()); - } - if (!getTokenBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 7, token_); - } - if (limit_ != 0) { - output.writeInt32(8, limit_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifactKey_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifactKey()); - } - if (partitions_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getPartitions()); - } - if (timePartitionValue_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getTimePartitionValue()); - } - if (!getPrincipalBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, principal_); - } - if (!getVersionBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, version_); - } - if (options_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, getOptions()); - } - if (!getTokenBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(7, token_); - } - if (limit_ != 0) { - size += com.google.protobuf.CodedOutputStream - .computeInt32Size(8, limit_); - } - 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.artifact.Artifacts.SearchArtifactsRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.SearchArtifactsRequest other = (flyteidl.artifact.Artifacts.SearchArtifactsRequest) obj; - - if (hasArtifactKey() != other.hasArtifactKey()) return false; - if (hasArtifactKey()) { - if (!getArtifactKey() - .equals(other.getArtifactKey())) return false; - } - if (hasPartitions() != other.hasPartitions()) return false; - if (hasPartitions()) { - if (!getPartitions() - .equals(other.getPartitions())) return false; - } - if (hasTimePartitionValue() != other.hasTimePartitionValue()) return false; - if (hasTimePartitionValue()) { - if (!getTimePartitionValue() - .equals(other.getTimePartitionValue())) return false; - } - if (!getPrincipal() - .equals(other.getPrincipal())) return false; - if (!getVersion() - .equals(other.getVersion())) return false; - if (hasOptions() != other.hasOptions()) return false; - if (hasOptions()) { - if (!getOptions() - .equals(other.getOptions())) return false; - } - if (!getToken() - .equals(other.getToken())) return false; - if (getLimit() - != other.getLimit()) 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 (hasArtifactKey()) { - hash = (37 * hash) + ARTIFACT_KEY_FIELD_NUMBER; - hash = (53 * hash) + getArtifactKey().hashCode(); - } - if (hasPartitions()) { - hash = (37 * hash) + PARTITIONS_FIELD_NUMBER; - hash = (53 * hash) + getPartitions().hashCode(); - } - if (hasTimePartitionValue()) { - hash = (37 * hash) + TIME_PARTITION_VALUE_FIELD_NUMBER; - hash = (53 * hash) + getTimePartitionValue().hashCode(); - } - hash = (37 * hash) + PRINCIPAL_FIELD_NUMBER; - hash = (53 * hash) + getPrincipal().hashCode(); - hash = (37 * hash) + VERSION_FIELD_NUMBER; - hash = (53 * hash) + getVersion().hashCode(); - if (hasOptions()) { - hash = (37 * hash) + OPTIONS_FIELD_NUMBER; - hash = (53 * hash) + getOptions().hashCode(); - } - hash = (37 * hash) + TOKEN_FIELD_NUMBER; - hash = (53 * hash) + getToken().hashCode(); - hash = (37 * hash) + LIMIT_FIELD_NUMBER; - hash = (53 * hash) + getLimit(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest 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.artifact.Artifacts.SearchArtifactsRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest 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.artifact.Artifacts.SearchArtifactsRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest 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.artifact.Artifacts.SearchArtifactsRequest 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.artifact.SearchArtifactsRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.SearchArtifactsRequest) - flyteidl.artifact.Artifacts.SearchArtifactsRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchArtifactsRequest.class, flyteidl.artifact.Artifacts.SearchArtifactsRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.SearchArtifactsRequest.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 (artifactKeyBuilder_ == null) { - artifactKey_ = null; - } else { - artifactKey_ = null; - artifactKeyBuilder_ = null; - } - if (partitionsBuilder_ == null) { - partitions_ = null; - } else { - partitions_ = null; - partitionsBuilder_ = null; - } - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = null; - } else { - timePartitionValue_ = null; - timePartitionValueBuilder_ = null; - } - principal_ = ""; - - version_ = ""; - - if (optionsBuilder_ == null) { - options_ = null; - } else { - options_ = null; - optionsBuilder_ = null; - } - token_ = ""; - - limit_ = 0; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.SearchArtifactsRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsRequest build() { - flyteidl.artifact.Artifacts.SearchArtifactsRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsRequest buildPartial() { - flyteidl.artifact.Artifacts.SearchArtifactsRequest result = new flyteidl.artifact.Artifacts.SearchArtifactsRequest(this); - if (artifactKeyBuilder_ == null) { - result.artifactKey_ = artifactKey_; - } else { - result.artifactKey_ = artifactKeyBuilder_.build(); - } - if (partitionsBuilder_ == null) { - result.partitions_ = partitions_; - } else { - result.partitions_ = partitionsBuilder_.build(); - } - if (timePartitionValueBuilder_ == null) { - result.timePartitionValue_ = timePartitionValue_; - } else { - result.timePartitionValue_ = timePartitionValueBuilder_.build(); - } - result.principal_ = principal_; - result.version_ = version_; - if (optionsBuilder_ == null) { - result.options_ = options_; - } else { - result.options_ = optionsBuilder_.build(); - } - result.token_ = token_; - result.limit_ = limit_; - 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.artifact.Artifacts.SearchArtifactsRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.SearchArtifactsRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.SearchArtifactsRequest other) { - if (other == flyteidl.artifact.Artifacts.SearchArtifactsRequest.getDefaultInstance()) return this; - if (other.hasArtifactKey()) { - mergeArtifactKey(other.getArtifactKey()); - } - if (other.hasPartitions()) { - mergePartitions(other.getPartitions()); - } - if (other.hasTimePartitionValue()) { - mergeTimePartitionValue(other.getTimePartitionValue()); - } - if (!other.getPrincipal().isEmpty()) { - principal_ = other.principal_; - onChanged(); - } - if (!other.getVersion().isEmpty()) { - version_ = other.version_; - onChanged(); - } - if (other.hasOptions()) { - mergeOptions(other.getOptions()); - } - if (!other.getToken().isEmpty()) { - token_ = other.token_; - onChanged(); - } - if (other.getLimit() != 0) { - setLimit(other.getLimit()); - } - 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.artifact.Artifacts.SearchArtifactsRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.SearchArtifactsRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.ArtifactId.ArtifactKey artifactKey_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder> artifactKeyBuilder_; - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public boolean hasArtifactKey() { - return artifactKeyBuilder_ != null || artifactKey_ != null; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey getArtifactKey() { - if (artifactKeyBuilder_ == null) { - return artifactKey_ == null ? flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } else { - return artifactKeyBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder setArtifactKey(flyteidl.core.ArtifactId.ArtifactKey value) { - if (artifactKeyBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifactKey_ = value; - onChanged(); - } else { - artifactKeyBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder setArtifactKey( - flyteidl.core.ArtifactId.ArtifactKey.Builder builderForValue) { - if (artifactKeyBuilder_ == null) { - artifactKey_ = builderForValue.build(); - onChanged(); - } else { - artifactKeyBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder mergeArtifactKey(flyteidl.core.ArtifactId.ArtifactKey value) { - if (artifactKeyBuilder_ == null) { - if (artifactKey_ != null) { - artifactKey_ = - flyteidl.core.ArtifactId.ArtifactKey.newBuilder(artifactKey_).mergeFrom(value).buildPartial(); - } else { - artifactKey_ = value; - } - onChanged(); - } else { - artifactKeyBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public Builder clearArtifactKey() { - if (artifactKeyBuilder_ == null) { - artifactKey_ = null; - onChanged(); - } else { - artifactKey_ = null; - artifactKeyBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKey.Builder getArtifactKeyBuilder() { - - onChanged(); - return getArtifactKeyFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - public flyteidl.core.ArtifactId.ArtifactKeyOrBuilder getArtifactKeyOrBuilder() { - if (artifactKeyBuilder_ != null) { - return artifactKeyBuilder_.getMessageOrBuilder(); - } else { - return artifactKey_ == null ? - flyteidl.core.ArtifactId.ArtifactKey.getDefaultInstance() : artifactKey_; - } - } - /** - * .flyteidl.core.ArtifactKey artifact_key = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder> - getArtifactKeyFieldBuilder() { - if (artifactKeyBuilder_ == null) { - artifactKeyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactKey, flyteidl.core.ArtifactId.ArtifactKey.Builder, flyteidl.core.ArtifactId.ArtifactKeyOrBuilder>( - getArtifactKey(), - getParentForChildren(), - isClean()); - artifactKey_ = null; - } - return artifactKeyBuilder_; - } - - private flyteidl.core.ArtifactId.Partitions partitions_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.Partitions, flyteidl.core.ArtifactId.Partitions.Builder, flyteidl.core.ArtifactId.PartitionsOrBuilder> partitionsBuilder_; - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public boolean hasPartitions() { - return partitionsBuilder_ != null || partitions_ != null; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public flyteidl.core.ArtifactId.Partitions getPartitions() { - if (partitionsBuilder_ == null) { - return partitions_ == null ? flyteidl.core.ArtifactId.Partitions.getDefaultInstance() : partitions_; - } else { - return partitionsBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public Builder setPartitions(flyteidl.core.ArtifactId.Partitions value) { - if (partitionsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - partitions_ = value; - onChanged(); - } else { - partitionsBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public Builder setPartitions( - flyteidl.core.ArtifactId.Partitions.Builder builderForValue) { - if (partitionsBuilder_ == null) { - partitions_ = builderForValue.build(); - onChanged(); - } else { - partitionsBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public Builder mergePartitions(flyteidl.core.ArtifactId.Partitions value) { - if (partitionsBuilder_ == null) { - if (partitions_ != null) { - partitions_ = - flyteidl.core.ArtifactId.Partitions.newBuilder(partitions_).mergeFrom(value).buildPartial(); - } else { - partitions_ = value; - } - onChanged(); - } else { - partitionsBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public Builder clearPartitions() { - if (partitionsBuilder_ == null) { - partitions_ = null; - onChanged(); - } else { - partitions_ = null; - partitionsBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public flyteidl.core.ArtifactId.Partitions.Builder getPartitionsBuilder() { - - onChanged(); - return getPartitionsFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - public flyteidl.core.ArtifactId.PartitionsOrBuilder getPartitionsOrBuilder() { - if (partitionsBuilder_ != null) { - return partitionsBuilder_.getMessageOrBuilder(); - } else { - return partitions_ == null ? - flyteidl.core.ArtifactId.Partitions.getDefaultInstance() : partitions_; - } - } - /** - * .flyteidl.core.Partitions partitions = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.Partitions, flyteidl.core.ArtifactId.Partitions.Builder, flyteidl.core.ArtifactId.PartitionsOrBuilder> - getPartitionsFieldBuilder() { - if (partitionsBuilder_ == null) { - partitionsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.Partitions, flyteidl.core.ArtifactId.Partitions.Builder, flyteidl.core.ArtifactId.PartitionsOrBuilder>( - getPartitions(), - getParentForChildren(), - isClean()); - partitions_ = null; - } - return partitionsBuilder_; - } - - private com.google.protobuf.Timestamp timePartitionValue_; - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> timePartitionValueBuilder_; - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public boolean hasTimePartitionValue() { - return timePartitionValueBuilder_ != null || timePartitionValue_ != null; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public com.google.protobuf.Timestamp getTimePartitionValue() { - if (timePartitionValueBuilder_ == null) { - return timePartitionValue_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } else { - return timePartitionValueBuilder_.getMessage(); - } - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public Builder setTimePartitionValue(com.google.protobuf.Timestamp value) { - if (timePartitionValueBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - timePartitionValue_ = value; - onChanged(); - } else { - timePartitionValueBuilder_.setMessage(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public Builder setTimePartitionValue( - com.google.protobuf.Timestamp.Builder builderForValue) { - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = builderForValue.build(); - onChanged(); - } else { - timePartitionValueBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public Builder mergeTimePartitionValue(com.google.protobuf.Timestamp value) { - if (timePartitionValueBuilder_ == null) { - if (timePartitionValue_ != null) { - timePartitionValue_ = - com.google.protobuf.Timestamp.newBuilder(timePartitionValue_).mergeFrom(value).buildPartial(); - } else { - timePartitionValue_ = value; - } - onChanged(); - } else { - timePartitionValueBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public Builder clearTimePartitionValue() { - if (timePartitionValueBuilder_ == null) { - timePartitionValue_ = null; - onChanged(); - } else { - timePartitionValue_ = null; - timePartitionValueBuilder_ = null; - } - - return this; - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public com.google.protobuf.Timestamp.Builder getTimePartitionValueBuilder() { - - onChanged(); - return getTimePartitionValueFieldBuilder().getBuilder(); - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - public com.google.protobuf.TimestampOrBuilder getTimePartitionValueOrBuilder() { - if (timePartitionValueBuilder_ != null) { - return timePartitionValueBuilder_.getMessageOrBuilder(); - } else { - return timePartitionValue_ == null ? - com.google.protobuf.Timestamp.getDefaultInstance() : timePartitionValue_; - } - } - /** - * .google.protobuf.Timestamp time_partition_value = 3; - */ - private com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> - getTimePartitionValueFieldBuilder() { - if (timePartitionValueBuilder_ == null) { - timePartitionValueBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>( - getTimePartitionValue(), - getParentForChildren(), - isClean()); - timePartitionValue_ = null; - } - return timePartitionValueBuilder_; - } - - private java.lang.Object principal_ = ""; - /** - * string principal = 4; - */ - public java.lang.String getPrincipal() { - java.lang.Object ref = principal_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - principal_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string principal = 4; - */ - public com.google.protobuf.ByteString - getPrincipalBytes() { - java.lang.Object ref = principal_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - principal_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string principal = 4; - */ - public Builder setPrincipal( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - principal_ = value; - onChanged(); - return this; - } - /** - * string principal = 4; - */ - public Builder clearPrincipal() { - - principal_ = getDefaultInstance().getPrincipal(); - onChanged(); - return this; - } - /** - * string principal = 4; - */ - public Builder setPrincipalBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - principal_ = value; - onChanged(); - return this; - } - - private java.lang.Object version_ = ""; - /** - * string version = 5; - */ - public java.lang.String getVersion() { - java.lang.Object ref = version_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - version_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string version = 5; - */ - public com.google.protobuf.ByteString - getVersionBytes() { - java.lang.Object ref = version_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - version_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string version = 5; - */ - public Builder setVersion( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - version_ = value; - onChanged(); - return this; - } - /** - * string version = 5; - */ - public Builder clearVersion() { - - version_ = getDefaultInstance().getVersion(); - onChanged(); - return this; - } - /** - * string version = 5; - */ - public Builder setVersionBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - version_ = value; - onChanged(); - return this; - } - - private flyteidl.artifact.Artifacts.SearchOptions options_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.SearchOptions, flyteidl.artifact.Artifacts.SearchOptions.Builder, flyteidl.artifact.Artifacts.SearchOptionsOrBuilder> optionsBuilder_; - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public boolean hasOptions() { - return optionsBuilder_ != null || options_ != null; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public flyteidl.artifact.Artifacts.SearchOptions getOptions() { - if (optionsBuilder_ == null) { - return options_ == null ? flyteidl.artifact.Artifacts.SearchOptions.getDefaultInstance() : options_; - } else { - return optionsBuilder_.getMessage(); - } - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public Builder setOptions(flyteidl.artifact.Artifacts.SearchOptions value) { - if (optionsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - options_ = value; - onChanged(); - } else { - optionsBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public Builder setOptions( - flyteidl.artifact.Artifacts.SearchOptions.Builder builderForValue) { - if (optionsBuilder_ == null) { - options_ = builderForValue.build(); - onChanged(); - } else { - optionsBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public Builder mergeOptions(flyteidl.artifact.Artifacts.SearchOptions value) { - if (optionsBuilder_ == null) { - if (options_ != null) { - options_ = - flyteidl.artifact.Artifacts.SearchOptions.newBuilder(options_).mergeFrom(value).buildPartial(); - } else { - options_ = value; - } - onChanged(); - } else { - optionsBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public Builder clearOptions() { - if (optionsBuilder_ == null) { - options_ = null; - onChanged(); - } else { - options_ = null; - optionsBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public flyteidl.artifact.Artifacts.SearchOptions.Builder getOptionsBuilder() { - - onChanged(); - return getOptionsFieldBuilder().getBuilder(); - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - public flyteidl.artifact.Artifacts.SearchOptionsOrBuilder getOptionsOrBuilder() { - if (optionsBuilder_ != null) { - return optionsBuilder_.getMessageOrBuilder(); - } else { - return options_ == null ? - flyteidl.artifact.Artifacts.SearchOptions.getDefaultInstance() : options_; - } - } - /** - * .flyteidl.artifact.SearchOptions options = 6; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.SearchOptions, flyteidl.artifact.Artifacts.SearchOptions.Builder, flyteidl.artifact.Artifacts.SearchOptionsOrBuilder> - getOptionsFieldBuilder() { - if (optionsBuilder_ == null) { - optionsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.artifact.Artifacts.SearchOptions, flyteidl.artifact.Artifacts.SearchOptions.Builder, flyteidl.artifact.Artifacts.SearchOptionsOrBuilder>( - getOptions(), - getParentForChildren(), - isClean()); - options_ = null; - } - return optionsBuilder_; - } - - private java.lang.Object token_ = ""; - /** - * string token = 7; - */ - public java.lang.String getToken() { - java.lang.Object ref = token_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - token_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string token = 7; - */ - public com.google.protobuf.ByteString - getTokenBytes() { - java.lang.Object ref = token_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - token_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string token = 7; - */ - public Builder setToken( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - token_ = value; - onChanged(); - return this; - } - /** - * string token = 7; - */ - public Builder clearToken() { - - token_ = getDefaultInstance().getToken(); - onChanged(); - return this; - } - /** - * string token = 7; - */ - public Builder setTokenBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - token_ = value; - onChanged(); - return this; - } - - private int limit_ ; - /** - * int32 limit = 8; - */ - public int getLimit() { - return limit_; - } - /** - * int32 limit = 8; - */ - public Builder setLimit(int value) { - - limit_ = value; - onChanged(); - return this; - } - /** - * int32 limit = 8; - */ - public Builder clearLimit() { - - limit_ = 0; - onChanged(); - return this; - } - @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.artifact.SearchArtifactsRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchArtifactsRequest) - private static final flyteidl.artifact.Artifacts.SearchArtifactsRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.SearchArtifactsRequest(); - } - - public static flyteidl.artifact.Artifacts.SearchArtifactsRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SearchArtifactsRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new SearchArtifactsRequest(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.artifact.Artifacts.SearchArtifactsRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface SearchArtifactsResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.SearchArtifactsResponse) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - java.util.List - getArtifactsList(); - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - flyteidl.artifact.Artifacts.Artifact getArtifacts(int index); - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - int getArtifactsCount(); - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - java.util.List - getArtifactsOrBuilderList(); - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactsOrBuilder( - int index); - - /** - *
-     * continuation token if relevant.
-     * 
- * - * string token = 2; - */ - java.lang.String getToken(); - /** - *
-     * continuation token if relevant.
-     * 
- * - * string token = 2; - */ - com.google.protobuf.ByteString - getTokenBytes(); - } - /** - * Protobuf type {@code flyteidl.artifact.SearchArtifactsResponse} - */ - public static final class SearchArtifactsResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.SearchArtifactsResponse) - SearchArtifactsResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use SearchArtifactsResponse.newBuilder() to construct. - private SearchArtifactsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private SearchArtifactsResponse() { - artifacts_ = java.util.Collections.emptyList(); - token_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private SearchArtifactsResponse( - 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: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - artifacts_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - artifacts_.add( - input.readMessage(flyteidl.artifact.Artifacts.Artifact.parser(), extensionRegistry)); - break; - } - case 18: { - java.lang.String s = input.readStringRequireUtf8(); - - token_ = s; - 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 { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - artifacts_ = java.util.Collections.unmodifiableList(artifacts_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchArtifactsResponse.class, flyteidl.artifact.Artifacts.SearchArtifactsResponse.Builder.class); - } - - private int bitField0_; - public static final int ARTIFACTS_FIELD_NUMBER = 1; - private java.util.List artifacts_; - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public java.util.List getArtifactsList() { - return artifacts_; - } - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public java.util.List - getArtifactsOrBuilderList() { - return artifacts_; - } - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public int getArtifactsCount() { - return artifacts_.size(); - } - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifacts(int index) { - return artifacts_.get(index); - } - /** - *
-     * If artifact specs are not requested, the resultant artifacts may be empty.
-     * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactsOrBuilder( - int index) { - return artifacts_.get(index); - } - - public static final int TOKEN_FIELD_NUMBER = 2; - private volatile java.lang.Object token_; - /** - *
-     * continuation token if relevant.
-     * 
- * - * string token = 2; - */ - public java.lang.String getToken() { - java.lang.Object ref = token_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - token_ = s; - return s; - } - } - /** - *
-     * continuation token if relevant.
-     * 
- * - * string token = 2; - */ - public com.google.protobuf.ByteString - getTokenBytes() { - java.lang.Object ref = token_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - token_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - 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 { - for (int i = 0; i < artifacts_.size(); i++) { - output.writeMessage(1, artifacts_.get(i)); - } - if (!getTokenBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 2, token_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < artifacts_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, artifacts_.get(i)); - } - if (!getTokenBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, token_); - } - 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.artifact.Artifacts.SearchArtifactsResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.SearchArtifactsResponse other = (flyteidl.artifact.Artifacts.SearchArtifactsResponse) obj; - - if (!getArtifactsList() - .equals(other.getArtifactsList())) return false; - if (!getToken() - .equals(other.getToken())) 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 (getArtifactsCount() > 0) { - hash = (37 * hash) + ARTIFACTS_FIELD_NUMBER; - hash = (53 * hash) + getArtifactsList().hashCode(); - } - hash = (37 * hash) + TOKEN_FIELD_NUMBER; - hash = (53 * hash) + getToken().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse 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.artifact.Artifacts.SearchArtifactsResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse 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.artifact.Artifacts.SearchArtifactsResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse 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.artifact.Artifacts.SearchArtifactsResponse 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.artifact.SearchArtifactsResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.SearchArtifactsResponse) - flyteidl.artifact.Artifacts.SearchArtifactsResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.SearchArtifactsResponse.class, flyteidl.artifact.Artifacts.SearchArtifactsResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.SearchArtifactsResponse.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getArtifactsFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (artifactsBuilder_ == null) { - artifacts_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - } else { - artifactsBuilder_.clear(); - } - token_ = ""; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.SearchArtifactsResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsResponse build() { - flyteidl.artifact.Artifacts.SearchArtifactsResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.SearchArtifactsResponse buildPartial() { - flyteidl.artifact.Artifacts.SearchArtifactsResponse result = new flyteidl.artifact.Artifacts.SearchArtifactsResponse(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (artifactsBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - artifacts_ = java.util.Collections.unmodifiableList(artifacts_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.artifacts_ = artifacts_; - } else { - result.artifacts_ = artifactsBuilder_.build(); - } - result.token_ = token_; - result.bitField0_ = to_bitField0_; - 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.artifact.Artifacts.SearchArtifactsResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.SearchArtifactsResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.SearchArtifactsResponse other) { - if (other == flyteidl.artifact.Artifacts.SearchArtifactsResponse.getDefaultInstance()) return this; - if (artifactsBuilder_ == null) { - if (!other.artifacts_.isEmpty()) { - if (artifacts_.isEmpty()) { - artifacts_ = other.artifacts_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureArtifactsIsMutable(); - artifacts_.addAll(other.artifacts_); - } - onChanged(); - } - } else { - if (!other.artifacts_.isEmpty()) { - if (artifactsBuilder_.isEmpty()) { - artifactsBuilder_.dispose(); - artifactsBuilder_ = null; - artifacts_ = other.artifacts_; - bitField0_ = (bitField0_ & ~0x00000001); - artifactsBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getArtifactsFieldBuilder() : null; - } else { - artifactsBuilder_.addAllMessages(other.artifacts_); - } - } - } - if (!other.getToken().isEmpty()) { - token_ = other.token_; - onChanged(); - } - 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.artifact.Artifacts.SearchArtifactsResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.SearchArtifactsResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private java.util.List artifacts_ = - java.util.Collections.emptyList(); - private void ensureArtifactsIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - artifacts_ = new java.util.ArrayList(artifacts_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> artifactsBuilder_; - - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public java.util.List getArtifactsList() { - if (artifactsBuilder_ == null) { - return java.util.Collections.unmodifiableList(artifacts_); - } else { - return artifactsBuilder_.getMessageList(); - } - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public int getArtifactsCount() { - if (artifactsBuilder_ == null) { - return artifacts_.size(); - } else { - return artifactsBuilder_.getCount(); - } - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.Artifact getArtifacts(int index) { - if (artifactsBuilder_ == null) { - return artifacts_.get(index); - } else { - return artifactsBuilder_.getMessage(index); - } - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder setArtifacts( - int index, flyteidl.artifact.Artifacts.Artifact value) { - if (artifactsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureArtifactsIsMutable(); - artifacts_.set(index, value); - onChanged(); - } else { - artifactsBuilder_.setMessage(index, value); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder setArtifacts( - int index, flyteidl.artifact.Artifacts.Artifact.Builder builderForValue) { - if (artifactsBuilder_ == null) { - ensureArtifactsIsMutable(); - artifacts_.set(index, builderForValue.build()); - onChanged(); - } else { - artifactsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder addArtifacts(flyteidl.artifact.Artifacts.Artifact value) { - if (artifactsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureArtifactsIsMutable(); - artifacts_.add(value); - onChanged(); - } else { - artifactsBuilder_.addMessage(value); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder addArtifacts( - int index, flyteidl.artifact.Artifacts.Artifact value) { - if (artifactsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureArtifactsIsMutable(); - artifacts_.add(index, value); - onChanged(); - } else { - artifactsBuilder_.addMessage(index, value); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder addArtifacts( - flyteidl.artifact.Artifacts.Artifact.Builder builderForValue) { - if (artifactsBuilder_ == null) { - ensureArtifactsIsMutable(); - artifacts_.add(builderForValue.build()); - onChanged(); - } else { - artifactsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder addArtifacts( - int index, flyteidl.artifact.Artifacts.Artifact.Builder builderForValue) { - if (artifactsBuilder_ == null) { - ensureArtifactsIsMutable(); - artifacts_.add(index, builderForValue.build()); - onChanged(); - } else { - artifactsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder addAllArtifacts( - java.lang.Iterable values) { - if (artifactsBuilder_ == null) { - ensureArtifactsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, artifacts_); - onChanged(); - } else { - artifactsBuilder_.addAllMessages(values); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder clearArtifacts() { - if (artifactsBuilder_ == null) { - artifacts_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - artifactsBuilder_.clear(); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public Builder removeArtifacts(int index) { - if (artifactsBuilder_ == null) { - ensureArtifactsIsMutable(); - artifacts_.remove(index); - onChanged(); - } else { - artifactsBuilder_.remove(index); - } - return this; - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.Artifact.Builder getArtifactsBuilder( - int index) { - return getArtifactsFieldBuilder().getBuilder(index); - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactOrBuilder getArtifactsOrBuilder( - int index) { - if (artifactsBuilder_ == null) { - return artifacts_.get(index); } else { - return artifactsBuilder_.getMessageOrBuilder(index); - } - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public java.util.List - getArtifactsOrBuilderList() { - if (artifactsBuilder_ != null) { - return artifactsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(artifacts_); - } - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.Artifact.Builder addArtifactsBuilder() { - return getArtifactsFieldBuilder().addBuilder( - flyteidl.artifact.Artifacts.Artifact.getDefaultInstance()); - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public flyteidl.artifact.Artifacts.Artifact.Builder addArtifactsBuilder( - int index) { - return getArtifactsFieldBuilder().addBuilder( - index, flyteidl.artifact.Artifacts.Artifact.getDefaultInstance()); - } - /** - *
-       * If artifact specs are not requested, the resultant artifacts may be empty.
-       * 
- * - * repeated .flyteidl.artifact.Artifact artifacts = 1; - */ - public java.util.List - getArtifactsBuilderList() { - return getArtifactsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder> - getArtifactsFieldBuilder() { - if (artifactsBuilder_ == null) { - artifactsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.Artifact, flyteidl.artifact.Artifacts.Artifact.Builder, flyteidl.artifact.Artifacts.ArtifactOrBuilder>( - artifacts_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - artifacts_ = null; - } - return artifactsBuilder_; - } - - private java.lang.Object token_ = ""; - /** - *
-       * continuation token if relevant.
-       * 
- * - * string token = 2; - */ - public java.lang.String getToken() { - java.lang.Object ref = token_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - token_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - *
-       * continuation token if relevant.
-       * 
- * - * string token = 2; - */ - public com.google.protobuf.ByteString - getTokenBytes() { - java.lang.Object ref = token_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - token_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - *
-       * continuation token if relevant.
-       * 
- * - * string token = 2; - */ - public Builder setToken( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - token_ = value; - onChanged(); - return this; - } - /** - *
-       * continuation token if relevant.
-       * 
- * - * string token = 2; - */ - public Builder clearToken() { - - token_ = getDefaultInstance().getToken(); - onChanged(); - return this; - } - /** - *
-       * continuation token if relevant.
-       * 
- * - * string token = 2; - */ - public Builder setTokenBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - token_ = value; - onChanged(); - return this; - } - @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.artifact.SearchArtifactsResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.SearchArtifactsResponse) - private static final flyteidl.artifact.Artifacts.SearchArtifactsResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.SearchArtifactsResponse(); - } - - public static flyteidl.artifact.Artifacts.SearchArtifactsResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public SearchArtifactsResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new SearchArtifactsResponse(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.artifact.Artifacts.SearchArtifactsResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface FindByWorkflowExecRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.FindByWorkflowExecRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - boolean hasExecId(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecId(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecIdOrBuilder(); - - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - int getDirectionValue(); - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction getDirection(); - } - /** - * Protobuf type {@code flyteidl.artifact.FindByWorkflowExecRequest} - */ - public static final class FindByWorkflowExecRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.FindByWorkflowExecRequest) - FindByWorkflowExecRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use FindByWorkflowExecRequest.newBuilder() to construct. - private FindByWorkflowExecRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private FindByWorkflowExecRequest() { - direction_ = 0; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private FindByWorkflowExecRequest( - 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.WorkflowExecutionIdentifier.Builder subBuilder = null; - if (execId_ != null) { - subBuilder = execId_.toBuilder(); - } - execId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(execId_); - execId_ = subBuilder.buildPartial(); - } - - break; - } - case 16: { - int rawValue = input.readEnum(); - - direction_ = rawValue; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_FindByWorkflowExecRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.class, flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Builder.class); - } - - /** - * Protobuf enum {@code flyteidl.artifact.FindByWorkflowExecRequest.Direction} - */ - public enum Direction - implements com.google.protobuf.ProtocolMessageEnum { - /** - * INPUTS = 0; - */ - INPUTS(0), - /** - * OUTPUTS = 1; - */ - OUTPUTS(1), - UNRECOGNIZED(-1), - ; - - /** - * INPUTS = 0; - */ - public static final int INPUTS_VALUE = 0; - /** - * OUTPUTS = 1; - */ - public static final int OUTPUTS_VALUE = 1; - - - public final int getNumber() { - if (this == UNRECOGNIZED) { - throw new java.lang.IllegalArgumentException( - "Can't get the number of an unknown enum value."); - } - return value; - } - - /** - * @deprecated Use {@link #forNumber(int)} instead. - */ - @java.lang.Deprecated - public static Direction valueOf(int value) { - return forNumber(value); - } - - public static Direction forNumber(int value) { - switch (value) { - case 0: return INPUTS; - case 1: return OUTPUTS; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static final com.google.protobuf.Internal.EnumLiteMap< - Direction> internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public Direction findValueByNumber(int number) { - return Direction.forNumber(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(ordinal()); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.getDescriptor().getEnumTypes().get(0); - } - - private static final Direction[] VALUES = values(); - - public static Direction valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - if (desc.getIndex() == -1) { - return UNRECOGNIZED; - } - return VALUES[desc.getIndex()]; - } - - private final int value; - - private Direction(int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:flyteidl.artifact.FindByWorkflowExecRequest.Direction) - } - - public static final int EXEC_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier execId_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public boolean hasExecId() { - return execId_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecId() { - return execId_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : execId_; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecIdOrBuilder() { - return getExecId(); - } - - public static final int DIRECTION_FIELD_NUMBER = 2; - private int direction_; - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public int getDirectionValue() { - return direction_; - } - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction getDirection() { - @SuppressWarnings("deprecation") - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction result = flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.valueOf(direction_); - return result == null ? flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.UNRECOGNIZED : result; - } - - 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 (execId_ != null) { - output.writeMessage(1, getExecId()); - } - if (direction_ != flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.INPUTS.getNumber()) { - output.writeEnum(2, direction_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (execId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getExecId()); - } - if (direction_ != flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.INPUTS.getNumber()) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(2, direction_); - } - 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.artifact.Artifacts.FindByWorkflowExecRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest other = (flyteidl.artifact.Artifacts.FindByWorkflowExecRequest) obj; - - if (hasExecId() != other.hasExecId()) return false; - if (hasExecId()) { - if (!getExecId() - .equals(other.getExecId())) return false; - } - if (direction_ != other.direction_) 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 (hasExecId()) { - hash = (37 * hash) + EXEC_ID_FIELD_NUMBER; - hash = (53 * hash) + getExecId().hashCode(); - } - hash = (37 * hash) + DIRECTION_FIELD_NUMBER; - hash = (53 * hash) + direction_; - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest 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.artifact.Artifacts.FindByWorkflowExecRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest 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.artifact.Artifacts.FindByWorkflowExecRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest 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.artifact.Artifacts.FindByWorkflowExecRequest 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.artifact.FindByWorkflowExecRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.FindByWorkflowExecRequest) - flyteidl.artifact.Artifacts.FindByWorkflowExecRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_FindByWorkflowExecRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.class, flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.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 (execIdBuilder_ == null) { - execId_ = null; - } else { - execId_ = null; - execIdBuilder_ = null; - } - direction_ = 0; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.FindByWorkflowExecRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.FindByWorkflowExecRequest build() { - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.FindByWorkflowExecRequest buildPartial() { - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest result = new flyteidl.artifact.Artifacts.FindByWorkflowExecRequest(this); - if (execIdBuilder_ == null) { - result.execId_ = execId_; - } else { - result.execId_ = execIdBuilder_.build(); - } - result.direction_ = direction_; - 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.artifact.Artifacts.FindByWorkflowExecRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.FindByWorkflowExecRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.FindByWorkflowExecRequest other) { - if (other == flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.getDefaultInstance()) return this; - if (other.hasExecId()) { - mergeExecId(other.getExecId()); - } - if (other.direction_ != 0) { - setDirectionValue(other.getDirectionValue()); - } - 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.artifact.Artifacts.FindByWorkflowExecRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.FindByWorkflowExecRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier execId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> execIdBuilder_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public boolean hasExecId() { - return execIdBuilder_ != null || execId_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecId() { - if (execIdBuilder_ == null) { - return execId_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : execId_; - } else { - return execIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public Builder setExecId(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (execIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - execId_ = value; - onChanged(); - } else { - execIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public Builder setExecId( - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (execIdBuilder_ == null) { - execId_ = builderForValue.build(); - onChanged(); - } else { - execIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public Builder mergeExecId(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (execIdBuilder_ == null) { - if (execId_ != null) { - execId_ = - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.newBuilder(execId_).mergeFrom(value).buildPartial(); - } else { - execId_ = value; - } - onChanged(); - } else { - execIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public Builder clearExecId() { - if (execIdBuilder_ == null) { - execId_ = null; - onChanged(); - } else { - execId_ = null; - execIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder getExecIdBuilder() { - - onChanged(); - return getExecIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecIdOrBuilder() { - if (execIdBuilder_ != null) { - return execIdBuilder_.getMessageOrBuilder(); - } else { - return execId_ == null ? - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : execId_; - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier exec_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> - getExecIdFieldBuilder() { - if (execIdBuilder_ == null) { - execIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder>( - getExecId(), - getParentForChildren(), - isClean()); - execId_ = null; - } - return execIdBuilder_; - } - - private int direction_ = 0; - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public int getDirectionValue() { - return direction_; - } - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public Builder setDirectionValue(int value) { - direction_ = value; - onChanged(); - return this; - } - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction getDirection() { - @SuppressWarnings("deprecation") - flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction result = flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.valueOf(direction_); - return result == null ? flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction.UNRECOGNIZED : result; - } - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public Builder setDirection(flyteidl.artifact.Artifacts.FindByWorkflowExecRequest.Direction value) { - if (value == null) { - throw new NullPointerException(); - } - - direction_ = value.getNumber(); - onChanged(); - return this; - } - /** - * .flyteidl.artifact.FindByWorkflowExecRequest.Direction direction = 2; - */ - public Builder clearDirection() { - - direction_ = 0; - onChanged(); - return this; - } - @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.artifact.FindByWorkflowExecRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.FindByWorkflowExecRequest) - private static final flyteidl.artifact.Artifacts.FindByWorkflowExecRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.FindByWorkflowExecRequest(); - } - - public static flyteidl.artifact.Artifacts.FindByWorkflowExecRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public FindByWorkflowExecRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new FindByWorkflowExecRequest(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.artifact.Artifacts.FindByWorkflowExecRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface AddTagRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.AddTagRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - boolean hasArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactID getArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder(); - - /** - * string value = 2; - */ - java.lang.String getValue(); - /** - * string value = 2; - */ - com.google.protobuf.ByteString - getValueBytes(); - - /** - *
-     * If true, and another version already has the specified kind/value, set this version instead
-     * 
- * - * bool overwrite = 3; - */ - boolean getOverwrite(); - } - /** - *
-   * Aliases identify a particular version of an artifact. They are different than tags in that they
-   * have to be unique for a given artifact project/domain/name. That is, for a given project/domain/name/kind,
-   * at most one version can have any given value at any point.
-   * 
- * - * Protobuf type {@code flyteidl.artifact.AddTagRequest} - */ - public static final class AddTagRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.AddTagRequest) - AddTagRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use AddTagRequest.newBuilder() to construct. - private AddTagRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private AddTagRequest() { - value_ = ""; - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private AddTagRequest( - 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.ArtifactId.ArtifactID.Builder subBuilder = null; - if (artifactId_ != null) { - subBuilder = artifactId_.toBuilder(); - } - artifactId_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactID.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifactId_); - artifactId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - java.lang.String s = input.readStringRequireUtf8(); - - value_ = s; - break; - } - case 24: { - - overwrite_ = input.readBool(); - 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.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.AddTagRequest.class, flyteidl.artifact.Artifacts.AddTagRequest.Builder.class); - } - - public static final int ARTIFACT_ID_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - return getArtifactId(); - } - - public static final int VALUE_FIELD_NUMBER = 2; - private volatile java.lang.Object value_; - /** - * string value = 2; - */ - public java.lang.String getValue() { - java.lang.Object ref = value_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - value_ = s; - return s; - } - } - /** - * string value = 2; - */ - public com.google.protobuf.ByteString - getValueBytes() { - java.lang.Object ref = value_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - value_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - public static final int OVERWRITE_FIELD_NUMBER = 3; - private boolean overwrite_; - /** - *
-     * If true, and another version already has the specified kind/value, set this version instead
-     * 
- * - * bool overwrite = 3; - */ - public boolean getOverwrite() { - return overwrite_; - } - - 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 (artifactId_ != null) { - output.writeMessage(1, getArtifactId()); - } - if (!getValueBytes().isEmpty()) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 2, value_); - } - if (overwrite_ != false) { - output.writeBool(3, overwrite_); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifactId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifactId()); - } - if (!getValueBytes().isEmpty()) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, value_); - } - if (overwrite_ != false) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, overwrite_); - } - 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.artifact.Artifacts.AddTagRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.AddTagRequest other = (flyteidl.artifact.Artifacts.AddTagRequest) obj; - - if (hasArtifactId() != other.hasArtifactId()) return false; - if (hasArtifactId()) { - if (!getArtifactId() - .equals(other.getArtifactId())) return false; - } - if (!getValue() - .equals(other.getValue())) return false; - if (getOverwrite() - != other.getOverwrite()) 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 (hasArtifactId()) { - hash = (37 * hash) + ARTIFACT_ID_FIELD_NUMBER; - hash = (53 * hash) + getArtifactId().hashCode(); - } - hash = (37 * hash) + VALUE_FIELD_NUMBER; - hash = (53 * hash) + getValue().hashCode(); - hash = (37 * hash) + OVERWRITE_FIELD_NUMBER; - hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( - getOverwrite()); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagRequest 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.artifact.Artifacts.AddTagRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagRequest 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.artifact.Artifacts.AddTagRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagRequest 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.artifact.Artifacts.AddTagRequest 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; - } - /** - *
-     * Aliases identify a particular version of an artifact. They are different than tags in that they
-     * have to be unique for a given artifact project/domain/name. That is, for a given project/domain/name/kind,
-     * at most one version can have any given value at any point.
-     * 
- * - * Protobuf type {@code flyteidl.artifact.AddTagRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.AddTagRequest) - flyteidl.artifact.Artifacts.AddTagRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.AddTagRequest.class, flyteidl.artifact.Artifacts.AddTagRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.AddTagRequest.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 (artifactIdBuilder_ == null) { - artifactId_ = null; - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - value_ = ""; - - overwrite_ = false; - - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.AddTagRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagRequest build() { - flyteidl.artifact.Artifacts.AddTagRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagRequest buildPartial() { - flyteidl.artifact.Artifacts.AddTagRequest result = new flyteidl.artifact.Artifacts.AddTagRequest(this); - if (artifactIdBuilder_ == null) { - result.artifactId_ = artifactId_; - } else { - result.artifactId_ = artifactIdBuilder_.build(); - } - result.value_ = value_; - result.overwrite_ = overwrite_; - 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.artifact.Artifacts.AddTagRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.AddTagRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.AddTagRequest other) { - if (other == flyteidl.artifact.Artifacts.AddTagRequest.getDefaultInstance()) return this; - if (other.hasArtifactId()) { - mergeArtifactId(other.getArtifactId()); - } - if (!other.getValue().isEmpty()) { - value_ = other.value_; - onChanged(); - } - if (other.getOverwrite() != false) { - setOverwrite(other.getOverwrite()); - } - 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.artifact.Artifacts.AddTagRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.AddTagRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> artifactIdBuilder_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactIdBuilder_ != null || artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - if (artifactIdBuilder_ == null) { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } else { - return artifactIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifactId_ = value; - onChanged(); - } else { - artifactIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId( - flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (artifactIdBuilder_ == null) { - artifactId_ = builderForValue.build(); - onChanged(); - } else { - artifactIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder mergeArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (artifactId_ != null) { - artifactId_ = - flyteidl.core.ArtifactId.ArtifactID.newBuilder(artifactId_).mergeFrom(value).buildPartial(); - } else { - artifactId_ = value; - } - onChanged(); - } else { - artifactIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder clearArtifactId() { - if (artifactIdBuilder_ == null) { - artifactId_ = null; - onChanged(); - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder getArtifactIdBuilder() { - - onChanged(); - return getArtifactIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - if (artifactIdBuilder_ != null) { - return artifactIdBuilder_.getMessageOrBuilder(); - } else { - return artifactId_ == null ? - flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> - getArtifactIdFieldBuilder() { - if (artifactIdBuilder_ == null) { - artifactIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder>( - getArtifactId(), - getParentForChildren(), - isClean()); - artifactId_ = null; - } - return artifactIdBuilder_; - } - - private java.lang.Object value_ = ""; - /** - * string value = 2; - */ - public java.lang.String getValue() { - java.lang.Object ref = value_; - if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - value_ = s; - return s; - } else { - return (java.lang.String) ref; - } - } - /** - * string value = 2; - */ - public com.google.protobuf.ByteString - getValueBytes() { - java.lang.Object ref = value_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - value_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * string value = 2; - */ - public Builder setValue( - java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - - value_ = value; - onChanged(); - return this; - } - /** - * string value = 2; - */ - public Builder clearValue() { - - value_ = getDefaultInstance().getValue(); - onChanged(); - return this; - } - /** - * string value = 2; - */ - public Builder setValueBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - checkByteStringIsUtf8(value); - - value_ = value; - onChanged(); - return this; - } - - private boolean overwrite_ ; - /** - *
-       * If true, and another version already has the specified kind/value, set this version instead
-       * 
- * - * bool overwrite = 3; - */ - public boolean getOverwrite() { - return overwrite_; - } - /** - *
-       * If true, and another version already has the specified kind/value, set this version instead
-       * 
- * - * bool overwrite = 3; - */ - public Builder setOverwrite(boolean value) { - - overwrite_ = value; - onChanged(); - return this; - } - /** - *
-       * If true, and another version already has the specified kind/value, set this version instead
-       * 
- * - * bool overwrite = 3; - */ - public Builder clearOverwrite() { - - overwrite_ = false; - onChanged(); - return this; - } - @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.artifact.AddTagRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.AddTagRequest) - private static final flyteidl.artifact.Artifacts.AddTagRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.AddTagRequest(); - } - - public static flyteidl.artifact.Artifacts.AddTagRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public AddTagRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new AddTagRequest(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.artifact.Artifacts.AddTagRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface AddTagResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.AddTagResponse) - com.google.protobuf.MessageOrBuilder { - } - /** - * Protobuf type {@code flyteidl.artifact.AddTagResponse} - */ - public static final class AddTagResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.AddTagResponse) - AddTagResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use AddTagResponse.newBuilder() to construct. - private AddTagResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private AddTagResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private AddTagResponse( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - 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; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.AddTagResponse.class, flyteidl.artifact.Artifacts.AddTagResponse.Builder.class); - } - - 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 { - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - 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.artifact.Artifacts.AddTagResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.AddTagResponse other = (flyteidl.artifact.Artifacts.AddTagResponse) obj; - - 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(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.AddTagResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagResponse 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.artifact.Artifacts.AddTagResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagResponse 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.artifact.Artifacts.AddTagResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.AddTagResponse 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.artifact.Artifacts.AddTagResponse 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.artifact.AddTagResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.AddTagResponse) - flyteidl.artifact.Artifacts.AddTagResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.AddTagResponse.class, flyteidl.artifact.Artifacts.AddTagResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.AddTagResponse.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(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_AddTagResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.AddTagResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagResponse build() { - flyteidl.artifact.Artifacts.AddTagResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.AddTagResponse buildPartial() { - flyteidl.artifact.Artifacts.AddTagResponse result = new flyteidl.artifact.Artifacts.AddTagResponse(this); - 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.artifact.Artifacts.AddTagResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.AddTagResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.AddTagResponse other) { - if (other == flyteidl.artifact.Artifacts.AddTagResponse.getDefaultInstance()) return this; - 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.artifact.Artifacts.AddTagResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.AddTagResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - @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.artifact.AddTagResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.AddTagResponse) - private static final flyteidl.artifact.Artifacts.AddTagResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.AddTagResponse(); - } - - public static flyteidl.artifact.Artifacts.AddTagResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public AddTagResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new AddTagResponse(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.artifact.Artifacts.AddTagResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface CreateTriggerRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.CreateTriggerRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - boolean hasTriggerLaunchPlan(); - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan getTriggerLaunchPlan(); - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder getTriggerLaunchPlanOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.CreateTriggerRequest} - */ - public static final class CreateTriggerRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.CreateTriggerRequest) - CreateTriggerRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use CreateTriggerRequest.newBuilder() to construct. - private CreateTriggerRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private CreateTriggerRequest() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private CreateTriggerRequest( - 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.admin.LaunchPlanOuterClass.LaunchPlan.Builder subBuilder = null; - if (triggerLaunchPlan_ != null) { - subBuilder = triggerLaunchPlan_.toBuilder(); - } - triggerLaunchPlan_ = input.readMessage(flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(triggerLaunchPlan_); - triggerLaunchPlan_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateTriggerRequest.class, flyteidl.artifact.Artifacts.CreateTriggerRequest.Builder.class); - } - - public static final int TRIGGER_LAUNCH_PLAN_FIELD_NUMBER = 1; - private flyteidl.admin.LaunchPlanOuterClass.LaunchPlan triggerLaunchPlan_; - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public boolean hasTriggerLaunchPlan() { - return triggerLaunchPlan_ != null; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public flyteidl.admin.LaunchPlanOuterClass.LaunchPlan getTriggerLaunchPlan() { - return triggerLaunchPlan_ == null ? flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.getDefaultInstance() : triggerLaunchPlan_; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder getTriggerLaunchPlanOrBuilder() { - return getTriggerLaunchPlan(); - } - - 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 (triggerLaunchPlan_ != null) { - output.writeMessage(1, getTriggerLaunchPlan()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (triggerLaunchPlan_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getTriggerLaunchPlan()); - } - 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.artifact.Artifacts.CreateTriggerRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.CreateTriggerRequest other = (flyteidl.artifact.Artifacts.CreateTriggerRequest) obj; - - if (hasTriggerLaunchPlan() != other.hasTriggerLaunchPlan()) return false; - if (hasTriggerLaunchPlan()) { - if (!getTriggerLaunchPlan() - .equals(other.getTriggerLaunchPlan())) 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 (hasTriggerLaunchPlan()) { - hash = (37 * hash) + TRIGGER_LAUNCH_PLAN_FIELD_NUMBER; - hash = (53 * hash) + getTriggerLaunchPlan().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest 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.artifact.Artifacts.CreateTriggerRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest 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.artifact.Artifacts.CreateTriggerRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerRequest 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.artifact.Artifacts.CreateTriggerRequest 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.artifact.CreateTriggerRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.CreateTriggerRequest) - flyteidl.artifact.Artifacts.CreateTriggerRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateTriggerRequest.class, flyteidl.artifact.Artifacts.CreateTriggerRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.CreateTriggerRequest.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 (triggerLaunchPlanBuilder_ == null) { - triggerLaunchPlan_ = null; - } else { - triggerLaunchPlan_ = null; - triggerLaunchPlanBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.CreateTriggerRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerRequest build() { - flyteidl.artifact.Artifacts.CreateTriggerRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerRequest buildPartial() { - flyteidl.artifact.Artifacts.CreateTriggerRequest result = new flyteidl.artifact.Artifacts.CreateTriggerRequest(this); - if (triggerLaunchPlanBuilder_ == null) { - result.triggerLaunchPlan_ = triggerLaunchPlan_; - } else { - result.triggerLaunchPlan_ = triggerLaunchPlanBuilder_.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.artifact.Artifacts.CreateTriggerRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.CreateTriggerRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.CreateTriggerRequest other) { - if (other == flyteidl.artifact.Artifacts.CreateTriggerRequest.getDefaultInstance()) return this; - if (other.hasTriggerLaunchPlan()) { - mergeTriggerLaunchPlan(other.getTriggerLaunchPlan()); - } - 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.artifact.Artifacts.CreateTriggerRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.CreateTriggerRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.admin.LaunchPlanOuterClass.LaunchPlan triggerLaunchPlan_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan, flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.Builder, flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder> triggerLaunchPlanBuilder_; - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public boolean hasTriggerLaunchPlan() { - return triggerLaunchPlanBuilder_ != null || triggerLaunchPlan_ != null; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public flyteidl.admin.LaunchPlanOuterClass.LaunchPlan getTriggerLaunchPlan() { - if (triggerLaunchPlanBuilder_ == null) { - return triggerLaunchPlan_ == null ? flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.getDefaultInstance() : triggerLaunchPlan_; - } else { - return triggerLaunchPlanBuilder_.getMessage(); - } - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public Builder setTriggerLaunchPlan(flyteidl.admin.LaunchPlanOuterClass.LaunchPlan value) { - if (triggerLaunchPlanBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - triggerLaunchPlan_ = value; - onChanged(); - } else { - triggerLaunchPlanBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public Builder setTriggerLaunchPlan( - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.Builder builderForValue) { - if (triggerLaunchPlanBuilder_ == null) { - triggerLaunchPlan_ = builderForValue.build(); - onChanged(); - } else { - triggerLaunchPlanBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public Builder mergeTriggerLaunchPlan(flyteidl.admin.LaunchPlanOuterClass.LaunchPlan value) { - if (triggerLaunchPlanBuilder_ == null) { - if (triggerLaunchPlan_ != null) { - triggerLaunchPlan_ = - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.newBuilder(triggerLaunchPlan_).mergeFrom(value).buildPartial(); - } else { - triggerLaunchPlan_ = value; - } - onChanged(); - } else { - triggerLaunchPlanBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public Builder clearTriggerLaunchPlan() { - if (triggerLaunchPlanBuilder_ == null) { - triggerLaunchPlan_ = null; - onChanged(); - } else { - triggerLaunchPlan_ = null; - triggerLaunchPlanBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.Builder getTriggerLaunchPlanBuilder() { - - onChanged(); - return getTriggerLaunchPlanFieldBuilder().getBuilder(); - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - public flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder getTriggerLaunchPlanOrBuilder() { - if (triggerLaunchPlanBuilder_ != null) { - return triggerLaunchPlanBuilder_.getMessageOrBuilder(); - } else { - return triggerLaunchPlan_ == null ? - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.getDefaultInstance() : triggerLaunchPlan_; - } - } - /** - * .flyteidl.admin.LaunchPlan trigger_launch_plan = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan, flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.Builder, flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder> - getTriggerLaunchPlanFieldBuilder() { - if (triggerLaunchPlanBuilder_ == null) { - triggerLaunchPlanBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.admin.LaunchPlanOuterClass.LaunchPlan, flyteidl.admin.LaunchPlanOuterClass.LaunchPlan.Builder, flyteidl.admin.LaunchPlanOuterClass.LaunchPlanOrBuilder>( - getTriggerLaunchPlan(), - getParentForChildren(), - isClean()); - triggerLaunchPlan_ = null; - } - return triggerLaunchPlanBuilder_; - } - @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.artifact.CreateTriggerRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateTriggerRequest) - private static final flyteidl.artifact.Artifacts.CreateTriggerRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.CreateTriggerRequest(); - } - - public static flyteidl.artifact.Artifacts.CreateTriggerRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public CreateTriggerRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new CreateTriggerRequest(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.artifact.Artifacts.CreateTriggerRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface CreateTriggerResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.CreateTriggerResponse) - com.google.protobuf.MessageOrBuilder { - } - /** - * Protobuf type {@code flyteidl.artifact.CreateTriggerResponse} - */ - public static final class CreateTriggerResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.CreateTriggerResponse) - CreateTriggerResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use CreateTriggerResponse.newBuilder() to construct. - private CreateTriggerResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private CreateTriggerResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private CreateTriggerResponse( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - 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; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateTriggerResponse.class, flyteidl.artifact.Artifacts.CreateTriggerResponse.Builder.class); - } - - 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 { - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - 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.artifact.Artifacts.CreateTriggerResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.CreateTriggerResponse other = (flyteidl.artifact.Artifacts.CreateTriggerResponse) obj; - - 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(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse 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.artifact.Artifacts.CreateTriggerResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse 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.artifact.Artifacts.CreateTriggerResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.CreateTriggerResponse 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.artifact.Artifacts.CreateTriggerResponse 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.artifact.CreateTriggerResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.CreateTriggerResponse) - flyteidl.artifact.Artifacts.CreateTriggerResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.CreateTriggerResponse.class, flyteidl.artifact.Artifacts.CreateTriggerResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.CreateTriggerResponse.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(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.CreateTriggerResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerResponse build() { - flyteidl.artifact.Artifacts.CreateTriggerResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.CreateTriggerResponse buildPartial() { - flyteidl.artifact.Artifacts.CreateTriggerResponse result = new flyteidl.artifact.Artifacts.CreateTriggerResponse(this); - 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.artifact.Artifacts.CreateTriggerResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.CreateTriggerResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.CreateTriggerResponse other) { - if (other == flyteidl.artifact.Artifacts.CreateTriggerResponse.getDefaultInstance()) return this; - 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.artifact.Artifacts.CreateTriggerResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.CreateTriggerResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - @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.artifact.CreateTriggerResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.CreateTriggerResponse) - private static final flyteidl.artifact.Artifacts.CreateTriggerResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.CreateTriggerResponse(); - } - - public static flyteidl.artifact.Artifacts.CreateTriggerResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public CreateTriggerResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new CreateTriggerResponse(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.artifact.Artifacts.CreateTriggerResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface DeactivateTriggerRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.DeactivateTriggerRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - boolean hasTriggerId(); - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - flyteidl.core.IdentifierOuterClass.Identifier getTriggerId(); - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.DeactivateTriggerRequest} - */ - public static final class DeactivateTriggerRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.DeactivateTriggerRequest) - DeactivateTriggerRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use DeactivateTriggerRequest.newBuilder() to construct. - private DeactivateTriggerRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private DeactivateTriggerRequest() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private DeactivateTriggerRequest( - 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.Identifier.Builder subBuilder = null; - if (triggerId_ != null) { - subBuilder = triggerId_.toBuilder(); - } - triggerId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.Identifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(triggerId_); - triggerId_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.DeactivateTriggerRequest.class, flyteidl.artifact.Artifacts.DeactivateTriggerRequest.Builder.class); - } - - public static final int TRIGGER_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.Identifier triggerId_; - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public boolean hasTriggerId() { - return triggerId_ != null; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTriggerId() { - return triggerId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder() { - return getTriggerId(); - } - - 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 (triggerId_ != null) { - output.writeMessage(1, getTriggerId()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (triggerId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getTriggerId()); - } - 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.artifact.Artifacts.DeactivateTriggerRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.DeactivateTriggerRequest other = (flyteidl.artifact.Artifacts.DeactivateTriggerRequest) obj; - - if (hasTriggerId() != other.hasTriggerId()) return false; - if (hasTriggerId()) { - if (!getTriggerId() - .equals(other.getTriggerId())) 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 (hasTriggerId()) { - hash = (37 * hash) + TRIGGER_ID_FIELD_NUMBER; - hash = (53 * hash) + getTriggerId().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest 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.artifact.Artifacts.DeactivateTriggerRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest 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.artifact.Artifacts.DeactivateTriggerRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest 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.artifact.Artifacts.DeactivateTriggerRequest 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.artifact.DeactivateTriggerRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.DeactivateTriggerRequest) - flyteidl.artifact.Artifacts.DeactivateTriggerRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.DeactivateTriggerRequest.class, flyteidl.artifact.Artifacts.DeactivateTriggerRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.DeactivateTriggerRequest.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 (triggerIdBuilder_ == null) { - triggerId_ = null; - } else { - triggerId_ = null; - triggerIdBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.DeactivateTriggerRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerRequest build() { - flyteidl.artifact.Artifacts.DeactivateTriggerRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerRequest buildPartial() { - flyteidl.artifact.Artifacts.DeactivateTriggerRequest result = new flyteidl.artifact.Artifacts.DeactivateTriggerRequest(this); - if (triggerIdBuilder_ == null) { - result.triggerId_ = triggerId_; - } else { - result.triggerId_ = triggerIdBuilder_.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.artifact.Artifacts.DeactivateTriggerRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.DeactivateTriggerRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.DeactivateTriggerRequest other) { - if (other == flyteidl.artifact.Artifacts.DeactivateTriggerRequest.getDefaultInstance()) return this; - if (other.hasTriggerId()) { - mergeTriggerId(other.getTriggerId()); - } - 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.artifact.Artifacts.DeactivateTriggerRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.DeactivateTriggerRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.IdentifierOuterClass.Identifier triggerId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> triggerIdBuilder_; - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public boolean hasTriggerId() { - return triggerIdBuilder_ != null || triggerId_ != null; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTriggerId() { - if (triggerIdBuilder_ == null) { - return triggerId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } else { - return triggerIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder setTriggerId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (triggerIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - triggerId_ = value; - onChanged(); - } else { - triggerIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder setTriggerId( - flyteidl.core.IdentifierOuterClass.Identifier.Builder builderForValue) { - if (triggerIdBuilder_ == null) { - triggerId_ = builderForValue.build(); - onChanged(); - } else { - triggerIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder mergeTriggerId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (triggerIdBuilder_ == null) { - if (triggerId_ != null) { - triggerId_ = - flyteidl.core.IdentifierOuterClass.Identifier.newBuilder(triggerId_).mergeFrom(value).buildPartial(); - } else { - triggerId_ = value; - } - onChanged(); - } else { - triggerIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder clearTriggerId() { - if (triggerIdBuilder_ == null) { - triggerId_ = null; - onChanged(); - } else { - triggerId_ = null; - triggerIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier.Builder getTriggerIdBuilder() { - - onChanged(); - return getTriggerIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder() { - if (triggerIdBuilder_ != null) { - return triggerIdBuilder_.getMessageOrBuilder(); - } else { - return triggerId_ == null ? - flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } - } - /** - * .flyteidl.core.Identifier trigger_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> - getTriggerIdFieldBuilder() { - if (triggerIdBuilder_ == null) { - triggerIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder>( - getTriggerId(), - getParentForChildren(), - isClean()); - triggerId_ = null; - } - return triggerIdBuilder_; - } - @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.artifact.DeactivateTriggerRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.DeactivateTriggerRequest) - private static final flyteidl.artifact.Artifacts.DeactivateTriggerRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.DeactivateTriggerRequest(); - } - - public static flyteidl.artifact.Artifacts.DeactivateTriggerRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public DeactivateTriggerRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new DeactivateTriggerRequest(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.artifact.Artifacts.DeactivateTriggerRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface DeactivateTriggerResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.DeactivateTriggerResponse) - com.google.protobuf.MessageOrBuilder { - } - /** - * Protobuf type {@code flyteidl.artifact.DeactivateTriggerResponse} - */ - public static final class DeactivateTriggerResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.DeactivateTriggerResponse) - DeactivateTriggerResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use DeactivateTriggerResponse.newBuilder() to construct. - private DeactivateTriggerResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private DeactivateTriggerResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private DeactivateTriggerResponse( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - 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; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.DeactivateTriggerResponse.class, flyteidl.artifact.Artifacts.DeactivateTriggerResponse.Builder.class); - } - - 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 { - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - 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.artifact.Artifacts.DeactivateTriggerResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.DeactivateTriggerResponse other = (flyteidl.artifact.Artifacts.DeactivateTriggerResponse) obj; - - 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(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse 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.artifact.Artifacts.DeactivateTriggerResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse 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.artifact.Artifacts.DeactivateTriggerResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse 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.artifact.Artifacts.DeactivateTriggerResponse 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.artifact.DeactivateTriggerResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.DeactivateTriggerResponse) - flyteidl.artifact.Artifacts.DeactivateTriggerResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.DeactivateTriggerResponse.class, flyteidl.artifact.Artifacts.DeactivateTriggerResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.DeactivateTriggerResponse.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(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.DeactivateTriggerResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerResponse build() { - flyteidl.artifact.Artifacts.DeactivateTriggerResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.DeactivateTriggerResponse buildPartial() { - flyteidl.artifact.Artifacts.DeactivateTriggerResponse result = new flyteidl.artifact.Artifacts.DeactivateTriggerResponse(this); - 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.artifact.Artifacts.DeactivateTriggerResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.DeactivateTriggerResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.DeactivateTriggerResponse other) { - if (other == flyteidl.artifact.Artifacts.DeactivateTriggerResponse.getDefaultInstance()) return this; - 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.artifact.Artifacts.DeactivateTriggerResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.DeactivateTriggerResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - @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.artifact.DeactivateTriggerResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.DeactivateTriggerResponse) - private static final flyteidl.artifact.Artifacts.DeactivateTriggerResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.DeactivateTriggerResponse(); - } - - public static flyteidl.artifact.Artifacts.DeactivateTriggerResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public DeactivateTriggerResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new DeactivateTriggerResponse(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.artifact.Artifacts.DeactivateTriggerResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ArtifactProducerOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ArtifactProducer) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - boolean hasEntityId(); - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - flyteidl.core.IdentifierOuterClass.Identifier getEntityId(); - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder(); - - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - boolean hasOutputs(); - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - flyteidl.core.Interface.VariableMap getOutputs(); - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - flyteidl.core.Interface.VariableMapOrBuilder getOutputsOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.ArtifactProducer} - */ - public static final class ArtifactProducer extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ArtifactProducer) - ArtifactProducerOrBuilder { - private static final long serialVersionUID = 0L; - // Use ArtifactProducer.newBuilder() to construct. - private ArtifactProducer(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ArtifactProducer() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ArtifactProducer( - 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.Identifier.Builder subBuilder = null; - if (entityId_ != null) { - subBuilder = entityId_.toBuilder(); - } - entityId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.Identifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(entityId_); - entityId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.core.Interface.VariableMap.Builder subBuilder = null; - if (outputs_ != null) { - subBuilder = outputs_.toBuilder(); - } - outputs_ = input.readMessage(flyteidl.core.Interface.VariableMap.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(outputs_); - outputs_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactProducer_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactProducer_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactProducer.class, flyteidl.artifact.Artifacts.ArtifactProducer.Builder.class); - } - - public static final int ENTITY_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.Identifier entityId_; - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public boolean hasEntityId() { - return entityId_ != null; - } - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getEntityId() { - return entityId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } - /** - *
-     * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-     * Admin's domain.
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder() { - return getEntityId(); - } - - public static final int OUTPUTS_FIELD_NUMBER = 2; - private flyteidl.core.Interface.VariableMap outputs_; - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public boolean hasOutputs() { - return outputs_ != null; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public flyteidl.core.Interface.VariableMap getOutputs() { - return outputs_ == null ? flyteidl.core.Interface.VariableMap.getDefaultInstance() : outputs_; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public flyteidl.core.Interface.VariableMapOrBuilder getOutputsOrBuilder() { - return getOutputs(); - } - - 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 (entityId_ != null) { - output.writeMessage(1, getEntityId()); - } - if (outputs_ != null) { - output.writeMessage(2, getOutputs()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (entityId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getEntityId()); - } - if (outputs_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getOutputs()); - } - 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.artifact.Artifacts.ArtifactProducer)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ArtifactProducer other = (flyteidl.artifact.Artifacts.ArtifactProducer) obj; - - if (hasEntityId() != other.hasEntityId()) return false; - if (hasEntityId()) { - if (!getEntityId() - .equals(other.getEntityId())) return false; - } - if (hasOutputs() != other.hasOutputs()) return false; - if (hasOutputs()) { - if (!getOutputs() - .equals(other.getOutputs())) 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 (hasEntityId()) { - hash = (37 * hash) + ENTITY_ID_FIELD_NUMBER; - hash = (53 * hash) + getEntityId().hashCode(); - } - if (hasOutputs()) { - hash = (37 * hash) + OUTPUTS_FIELD_NUMBER; - hash = (53 * hash) + getOutputs().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer 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.artifact.Artifacts.ArtifactProducer parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer 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.artifact.Artifacts.ArtifactProducer parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactProducer 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.artifact.Artifacts.ArtifactProducer 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.artifact.ArtifactProducer} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ArtifactProducer) - flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactProducer_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactProducer_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactProducer.class, flyteidl.artifact.Artifacts.ArtifactProducer.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ArtifactProducer.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 (entityIdBuilder_ == null) { - entityId_ = null; - } else { - entityId_ = null; - entityIdBuilder_ = null; - } - if (outputsBuilder_ == null) { - outputs_ = null; - } else { - outputs_ = null; - outputsBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactProducer_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactProducer getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ArtifactProducer.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactProducer build() { - flyteidl.artifact.Artifacts.ArtifactProducer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactProducer buildPartial() { - flyteidl.artifact.Artifacts.ArtifactProducer result = new flyteidl.artifact.Artifacts.ArtifactProducer(this); - if (entityIdBuilder_ == null) { - result.entityId_ = entityId_; - } else { - result.entityId_ = entityIdBuilder_.build(); - } - if (outputsBuilder_ == null) { - result.outputs_ = outputs_; - } else { - result.outputs_ = outputsBuilder_.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.artifact.Artifacts.ArtifactProducer) { - return mergeFrom((flyteidl.artifact.Artifacts.ArtifactProducer)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ArtifactProducer other) { - if (other == flyteidl.artifact.Artifacts.ArtifactProducer.getDefaultInstance()) return this; - if (other.hasEntityId()) { - mergeEntityId(other.getEntityId()); - } - if (other.hasOutputs()) { - mergeOutputs(other.getOutputs()); - } - 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.artifact.Artifacts.ArtifactProducer parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ArtifactProducer) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.IdentifierOuterClass.Identifier entityId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> entityIdBuilder_; - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public boolean hasEntityId() { - return entityIdBuilder_ != null || entityId_ != null; - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getEntityId() { - if (entityIdBuilder_ == null) { - return entityId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } else { - return entityIdBuilder_.getMessage(); - } - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder setEntityId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (entityIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - entityId_ = value; - onChanged(); - } else { - entityIdBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder setEntityId( - flyteidl.core.IdentifierOuterClass.Identifier.Builder builderForValue) { - if (entityIdBuilder_ == null) { - entityId_ = builderForValue.build(); - onChanged(); - } else { - entityIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder mergeEntityId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (entityIdBuilder_ == null) { - if (entityId_ != null) { - entityId_ = - flyteidl.core.IdentifierOuterClass.Identifier.newBuilder(entityId_).mergeFrom(value).buildPartial(); - } else { - entityId_ = value; - } - onChanged(); - } else { - entityIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder clearEntityId() { - if (entityIdBuilder_ == null) { - entityId_ = null; - onChanged(); - } else { - entityId_ = null; - entityIdBuilder_ = null; - } - - return this; - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier.Builder getEntityIdBuilder() { - - onChanged(); - return getEntityIdFieldBuilder().getBuilder(); - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder() { - if (entityIdBuilder_ != null) { - return entityIdBuilder_.getMessageOrBuilder(); - } else { - return entityId_ == null ? - flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } - } - /** - *
-       * These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in
-       * Admin's domain.
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> - getEntityIdFieldBuilder() { - if (entityIdBuilder_ == null) { - entityIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder>( - getEntityId(), - getParentForChildren(), - isClean()); - entityId_ = null; - } - return entityIdBuilder_; - } - - private flyteidl.core.Interface.VariableMap outputs_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.VariableMap, flyteidl.core.Interface.VariableMap.Builder, flyteidl.core.Interface.VariableMapOrBuilder> outputsBuilder_; - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public boolean hasOutputs() { - return outputsBuilder_ != null || outputs_ != null; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public flyteidl.core.Interface.VariableMap getOutputs() { - if (outputsBuilder_ == null) { - return outputs_ == null ? flyteidl.core.Interface.VariableMap.getDefaultInstance() : outputs_; - } else { - return outputsBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public Builder setOutputs(flyteidl.core.Interface.VariableMap value) { - if (outputsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - outputs_ = value; - onChanged(); - } else { - outputsBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public Builder setOutputs( - flyteidl.core.Interface.VariableMap.Builder builderForValue) { - if (outputsBuilder_ == null) { - outputs_ = builderForValue.build(); - onChanged(); - } else { - outputsBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public Builder mergeOutputs(flyteidl.core.Interface.VariableMap value) { - if (outputsBuilder_ == null) { - if (outputs_ != null) { - outputs_ = - flyteidl.core.Interface.VariableMap.newBuilder(outputs_).mergeFrom(value).buildPartial(); - } else { - outputs_ = value; - } - onChanged(); - } else { - outputsBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public Builder clearOutputs() { - if (outputsBuilder_ == null) { - outputs_ = null; - onChanged(); - } else { - outputs_ = null; - outputsBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public flyteidl.core.Interface.VariableMap.Builder getOutputsBuilder() { - - onChanged(); - return getOutputsFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - public flyteidl.core.Interface.VariableMapOrBuilder getOutputsOrBuilder() { - if (outputsBuilder_ != null) { - return outputsBuilder_.getMessageOrBuilder(); - } else { - return outputs_ == null ? - flyteidl.core.Interface.VariableMap.getDefaultInstance() : outputs_; - } - } - /** - * .flyteidl.core.VariableMap outputs = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.VariableMap, flyteidl.core.Interface.VariableMap.Builder, flyteidl.core.Interface.VariableMapOrBuilder> - getOutputsFieldBuilder() { - if (outputsBuilder_ == null) { - outputsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.VariableMap, flyteidl.core.Interface.VariableMap.Builder, flyteidl.core.Interface.VariableMapOrBuilder>( - getOutputs(), - getParentForChildren(), - isClean()); - outputs_ = null; - } - return outputsBuilder_; - } - @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.artifact.ArtifactProducer) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactProducer) - private static final flyteidl.artifact.Artifacts.ArtifactProducer DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ArtifactProducer(); - } - - public static flyteidl.artifact.Artifacts.ArtifactProducer getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ArtifactProducer parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ArtifactProducer(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.artifact.Artifacts.ArtifactProducer getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface RegisterProducerRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.RegisterProducerRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - java.util.List - getProducersList(); - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - flyteidl.artifact.Artifacts.ArtifactProducer getProducers(int index); - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - int getProducersCount(); - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - java.util.List - getProducersOrBuilderList(); - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder getProducersOrBuilder( - int index); - } - /** - * Protobuf type {@code flyteidl.artifact.RegisterProducerRequest} - */ - public static final class RegisterProducerRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.RegisterProducerRequest) - RegisterProducerRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use RegisterProducerRequest.newBuilder() to construct. - private RegisterProducerRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private RegisterProducerRequest() { - producers_ = java.util.Collections.emptyList(); - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private RegisterProducerRequest( - 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: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - producers_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - producers_.add( - input.readMessage(flyteidl.artifact.Artifacts.ArtifactProducer.parser(), extensionRegistry)); - 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 { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - producers_ = java.util.Collections.unmodifiableList(producers_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterProducerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterProducerRequest.class, flyteidl.artifact.Artifacts.RegisterProducerRequest.Builder.class); - } - - public static final int PRODUCERS_FIELD_NUMBER = 1; - private java.util.List producers_; - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public java.util.List getProducersList() { - return producers_; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public java.util.List - getProducersOrBuilderList() { - return producers_; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public int getProducersCount() { - return producers_.size(); - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducer getProducers(int index) { - return producers_.get(index); - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder getProducersOrBuilder( - int index) { - return producers_.get(index); - } - - 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 { - for (int i = 0; i < producers_.size(); i++) { - output.writeMessage(1, producers_.get(i)); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < producers_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, producers_.get(i)); - } - 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.artifact.Artifacts.RegisterProducerRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.RegisterProducerRequest other = (flyteidl.artifact.Artifacts.RegisterProducerRequest) obj; - - if (!getProducersList() - .equals(other.getProducersList())) 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 (getProducersCount() > 0) { - hash = (37 * hash) + PRODUCERS_FIELD_NUMBER; - hash = (53 * hash) + getProducersList().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest 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.artifact.Artifacts.RegisterProducerRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest 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.artifact.Artifacts.RegisterProducerRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterProducerRequest 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.artifact.Artifacts.RegisterProducerRequest 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.artifact.RegisterProducerRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.RegisterProducerRequest) - flyteidl.artifact.Artifacts.RegisterProducerRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterProducerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterProducerRequest.class, flyteidl.artifact.Artifacts.RegisterProducerRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.RegisterProducerRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getProducersFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (producersBuilder_ == null) { - producers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - } else { - producersBuilder_.clear(); - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterProducerRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.RegisterProducerRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterProducerRequest build() { - flyteidl.artifact.Artifacts.RegisterProducerRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterProducerRequest buildPartial() { - flyteidl.artifact.Artifacts.RegisterProducerRequest result = new flyteidl.artifact.Artifacts.RegisterProducerRequest(this); - int from_bitField0_ = bitField0_; - if (producersBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - producers_ = java.util.Collections.unmodifiableList(producers_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.producers_ = producers_; - } else { - result.producers_ = producersBuilder_.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.artifact.Artifacts.RegisterProducerRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.RegisterProducerRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.RegisterProducerRequest other) { - if (other == flyteidl.artifact.Artifacts.RegisterProducerRequest.getDefaultInstance()) return this; - if (producersBuilder_ == null) { - if (!other.producers_.isEmpty()) { - if (producers_.isEmpty()) { - producers_ = other.producers_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureProducersIsMutable(); - producers_.addAll(other.producers_); - } - onChanged(); - } - } else { - if (!other.producers_.isEmpty()) { - if (producersBuilder_.isEmpty()) { - producersBuilder_.dispose(); - producersBuilder_ = null; - producers_ = other.producers_; - bitField0_ = (bitField0_ & ~0x00000001); - producersBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getProducersFieldBuilder() : null; - } else { - producersBuilder_.addAllMessages(other.producers_); - } - } - } - 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.artifact.Artifacts.RegisterProducerRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.RegisterProducerRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private java.util.List producers_ = - java.util.Collections.emptyList(); - private void ensureProducersIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - producers_ = new java.util.ArrayList(producers_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactProducer, flyteidl.artifact.Artifacts.ArtifactProducer.Builder, flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder> producersBuilder_; - - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public java.util.List getProducersList() { - if (producersBuilder_ == null) { - return java.util.Collections.unmodifiableList(producers_); - } else { - return producersBuilder_.getMessageList(); - } - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public int getProducersCount() { - if (producersBuilder_ == null) { - return producers_.size(); - } else { - return producersBuilder_.getCount(); - } - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducer getProducers(int index) { - if (producersBuilder_ == null) { - return producers_.get(index); - } else { - return producersBuilder_.getMessage(index); - } - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder setProducers( - int index, flyteidl.artifact.Artifacts.ArtifactProducer value) { - if (producersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureProducersIsMutable(); - producers_.set(index, value); - onChanged(); - } else { - producersBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder setProducers( - int index, flyteidl.artifact.Artifacts.ArtifactProducer.Builder builderForValue) { - if (producersBuilder_ == null) { - ensureProducersIsMutable(); - producers_.set(index, builderForValue.build()); - onChanged(); - } else { - producersBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder addProducers(flyteidl.artifact.Artifacts.ArtifactProducer value) { - if (producersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureProducersIsMutable(); - producers_.add(value); - onChanged(); - } else { - producersBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder addProducers( - int index, flyteidl.artifact.Artifacts.ArtifactProducer value) { - if (producersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureProducersIsMutable(); - producers_.add(index, value); - onChanged(); - } else { - producersBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder addProducers( - flyteidl.artifact.Artifacts.ArtifactProducer.Builder builderForValue) { - if (producersBuilder_ == null) { - ensureProducersIsMutable(); - producers_.add(builderForValue.build()); - onChanged(); - } else { - producersBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder addProducers( - int index, flyteidl.artifact.Artifacts.ArtifactProducer.Builder builderForValue) { - if (producersBuilder_ == null) { - ensureProducersIsMutable(); - producers_.add(index, builderForValue.build()); - onChanged(); - } else { - producersBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder addAllProducers( - java.lang.Iterable values) { - if (producersBuilder_ == null) { - ensureProducersIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, producers_); - onChanged(); - } else { - producersBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder clearProducers() { - if (producersBuilder_ == null) { - producers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - producersBuilder_.clear(); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public Builder removeProducers(int index) { - if (producersBuilder_ == null) { - ensureProducersIsMutable(); - producers_.remove(index); - onChanged(); - } else { - producersBuilder_.remove(index); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducer.Builder getProducersBuilder( - int index) { - return getProducersFieldBuilder().getBuilder(index); - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder getProducersOrBuilder( - int index) { - if (producersBuilder_ == null) { - return producers_.get(index); } else { - return producersBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public java.util.List - getProducersOrBuilderList() { - if (producersBuilder_ != null) { - return producersBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(producers_); - } - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducer.Builder addProducersBuilder() { - return getProducersFieldBuilder().addBuilder( - flyteidl.artifact.Artifacts.ArtifactProducer.getDefaultInstance()); - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactProducer.Builder addProducersBuilder( - int index) { - return getProducersFieldBuilder().addBuilder( - index, flyteidl.artifact.Artifacts.ArtifactProducer.getDefaultInstance()); - } - /** - * repeated .flyteidl.artifact.ArtifactProducer producers = 1; - */ - public java.util.List - getProducersBuilderList() { - return getProducersFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactProducer, flyteidl.artifact.Artifacts.ArtifactProducer.Builder, flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder> - getProducersFieldBuilder() { - if (producersBuilder_ == null) { - producersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactProducer, flyteidl.artifact.Artifacts.ArtifactProducer.Builder, flyteidl.artifact.Artifacts.ArtifactProducerOrBuilder>( - producers_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - producers_ = null; - } - return producersBuilder_; - } - @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.artifact.RegisterProducerRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterProducerRequest) - private static final flyteidl.artifact.Artifacts.RegisterProducerRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.RegisterProducerRequest(); - } - - public static flyteidl.artifact.Artifacts.RegisterProducerRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RegisterProducerRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new RegisterProducerRequest(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.artifact.Artifacts.RegisterProducerRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ArtifactConsumerOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ArtifactConsumer) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - boolean hasEntityId(); - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - flyteidl.core.IdentifierOuterClass.Identifier getEntityId(); - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder(); - - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - boolean hasInputs(); - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - flyteidl.core.Interface.ParameterMap getInputs(); - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - flyteidl.core.Interface.ParameterMapOrBuilder getInputsOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.ArtifactConsumer} - */ - public static final class ArtifactConsumer extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ArtifactConsumer) - ArtifactConsumerOrBuilder { - private static final long serialVersionUID = 0L; - // Use ArtifactConsumer.newBuilder() to construct. - private ArtifactConsumer(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ArtifactConsumer() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ArtifactConsumer( - 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.Identifier.Builder subBuilder = null; - if (entityId_ != null) { - subBuilder = entityId_.toBuilder(); - } - entityId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.Identifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(entityId_); - entityId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - flyteidl.core.Interface.ParameterMap.Builder subBuilder = null; - if (inputs_ != null) { - subBuilder = inputs_.toBuilder(); - } - inputs_ = input.readMessage(flyteidl.core.Interface.ParameterMap.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(inputs_); - inputs_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactConsumer_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactConsumer_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactConsumer.class, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder.class); - } - - public static final int ENTITY_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.Identifier entityId_; - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public boolean hasEntityId() { - return entityId_ != null; - } - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getEntityId() { - return entityId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } - /** - *
-     * These should all be launch plan IDs
-     * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder() { - return getEntityId(); - } - - public static final int INPUTS_FIELD_NUMBER = 2; - private flyteidl.core.Interface.ParameterMap inputs_; - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public boolean hasInputs() { - return inputs_ != null; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public flyteidl.core.Interface.ParameterMap getInputs() { - return inputs_ == null ? flyteidl.core.Interface.ParameterMap.getDefaultInstance() : inputs_; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public flyteidl.core.Interface.ParameterMapOrBuilder getInputsOrBuilder() { - return getInputs(); - } - - 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 (entityId_ != null) { - output.writeMessage(1, getEntityId()); - } - if (inputs_ != null) { - output.writeMessage(2, getInputs()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (entityId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getEntityId()); - } - if (inputs_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getInputs()); - } - 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.artifact.Artifacts.ArtifactConsumer)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ArtifactConsumer other = (flyteidl.artifact.Artifacts.ArtifactConsumer) obj; - - if (hasEntityId() != other.hasEntityId()) return false; - if (hasEntityId()) { - if (!getEntityId() - .equals(other.getEntityId())) return false; - } - if (hasInputs() != other.hasInputs()) return false; - if (hasInputs()) { - if (!getInputs() - .equals(other.getInputs())) 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 (hasEntityId()) { - hash = (37 * hash) + ENTITY_ID_FIELD_NUMBER; - hash = (53 * hash) + getEntityId().hashCode(); - } - if (hasInputs()) { - hash = (37 * hash) + INPUTS_FIELD_NUMBER; - hash = (53 * hash) + getInputs().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer 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.artifact.Artifacts.ArtifactConsumer parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer 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.artifact.Artifacts.ArtifactConsumer parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ArtifactConsumer 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.artifact.Artifacts.ArtifactConsumer 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.artifact.ArtifactConsumer} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ArtifactConsumer) - flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactConsumer_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactConsumer_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ArtifactConsumer.class, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ArtifactConsumer.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 (entityIdBuilder_ == null) { - entityId_ = null; - } else { - entityId_ = null; - entityIdBuilder_ = null; - } - if (inputsBuilder_ == null) { - inputs_ = null; - } else { - inputs_ = null; - inputsBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ArtifactConsumer_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactConsumer getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ArtifactConsumer.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactConsumer build() { - flyteidl.artifact.Artifacts.ArtifactConsumer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ArtifactConsumer buildPartial() { - flyteidl.artifact.Artifacts.ArtifactConsumer result = new flyteidl.artifact.Artifacts.ArtifactConsumer(this); - if (entityIdBuilder_ == null) { - result.entityId_ = entityId_; - } else { - result.entityId_ = entityIdBuilder_.build(); - } - if (inputsBuilder_ == null) { - result.inputs_ = inputs_; - } else { - result.inputs_ = inputsBuilder_.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.artifact.Artifacts.ArtifactConsumer) { - return mergeFrom((flyteidl.artifact.Artifacts.ArtifactConsumer)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ArtifactConsumer other) { - if (other == flyteidl.artifact.Artifacts.ArtifactConsumer.getDefaultInstance()) return this; - if (other.hasEntityId()) { - mergeEntityId(other.getEntityId()); - } - if (other.hasInputs()) { - mergeInputs(other.getInputs()); - } - 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.artifact.Artifacts.ArtifactConsumer parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ArtifactConsumer) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.IdentifierOuterClass.Identifier entityId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> entityIdBuilder_; - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public boolean hasEntityId() { - return entityIdBuilder_ != null || entityId_ != null; - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getEntityId() { - if (entityIdBuilder_ == null) { - return entityId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } else { - return entityIdBuilder_.getMessage(); - } - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder setEntityId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (entityIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - entityId_ = value; - onChanged(); - } else { - entityIdBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder setEntityId( - flyteidl.core.IdentifierOuterClass.Identifier.Builder builderForValue) { - if (entityIdBuilder_ == null) { - entityId_ = builderForValue.build(); - onChanged(); - } else { - entityIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder mergeEntityId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (entityIdBuilder_ == null) { - if (entityId_ != null) { - entityId_ = - flyteidl.core.IdentifierOuterClass.Identifier.newBuilder(entityId_).mergeFrom(value).buildPartial(); - } else { - entityId_ = value; - } - onChanged(); - } else { - entityIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public Builder clearEntityId() { - if (entityIdBuilder_ == null) { - entityId_ = null; - onChanged(); - } else { - entityId_ = null; - entityIdBuilder_ = null; - } - - return this; - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier.Builder getEntityIdBuilder() { - - onChanged(); - return getEntityIdFieldBuilder().getBuilder(); - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getEntityIdOrBuilder() { - if (entityIdBuilder_ != null) { - return entityIdBuilder_.getMessageOrBuilder(); - } else { - return entityId_ == null ? - flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : entityId_; - } - } - /** - *
-       * These should all be launch plan IDs
-       * 
- * - * .flyteidl.core.Identifier entity_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> - getEntityIdFieldBuilder() { - if (entityIdBuilder_ == null) { - entityIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder>( - getEntityId(), - getParentForChildren(), - isClean()); - entityId_ = null; - } - return entityIdBuilder_; - } - - private flyteidl.core.Interface.ParameterMap inputs_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.ParameterMap, flyteidl.core.Interface.ParameterMap.Builder, flyteidl.core.Interface.ParameterMapOrBuilder> inputsBuilder_; - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public boolean hasInputs() { - return inputsBuilder_ != null || inputs_ != null; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public flyteidl.core.Interface.ParameterMap getInputs() { - if (inputsBuilder_ == null) { - return inputs_ == null ? flyteidl.core.Interface.ParameterMap.getDefaultInstance() : inputs_; - } else { - return inputsBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public Builder setInputs(flyteidl.core.Interface.ParameterMap value) { - if (inputsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - inputs_ = value; - onChanged(); - } else { - inputsBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public Builder setInputs( - flyteidl.core.Interface.ParameterMap.Builder builderForValue) { - if (inputsBuilder_ == null) { - inputs_ = builderForValue.build(); - onChanged(); - } else { - inputsBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public Builder mergeInputs(flyteidl.core.Interface.ParameterMap value) { - if (inputsBuilder_ == null) { - if (inputs_ != null) { - inputs_ = - flyteidl.core.Interface.ParameterMap.newBuilder(inputs_).mergeFrom(value).buildPartial(); - } else { - inputs_ = value; - } - onChanged(); - } else { - inputsBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public Builder clearInputs() { - if (inputsBuilder_ == null) { - inputs_ = null; - onChanged(); - } else { - inputs_ = null; - inputsBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public flyteidl.core.Interface.ParameterMap.Builder getInputsBuilder() { - - onChanged(); - return getInputsFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - public flyteidl.core.Interface.ParameterMapOrBuilder getInputsOrBuilder() { - if (inputsBuilder_ != null) { - return inputsBuilder_.getMessageOrBuilder(); - } else { - return inputs_ == null ? - flyteidl.core.Interface.ParameterMap.getDefaultInstance() : inputs_; - } - } - /** - * .flyteidl.core.ParameterMap inputs = 2; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.ParameterMap, flyteidl.core.Interface.ParameterMap.Builder, flyteidl.core.Interface.ParameterMapOrBuilder> - getInputsFieldBuilder() { - if (inputsBuilder_ == null) { - inputsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.Interface.ParameterMap, flyteidl.core.Interface.ParameterMap.Builder, flyteidl.core.Interface.ParameterMapOrBuilder>( - getInputs(), - getParentForChildren(), - isClean()); - inputs_ = null; - } - return inputsBuilder_; - } - @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.artifact.ArtifactConsumer) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ArtifactConsumer) - private static final flyteidl.artifact.Artifacts.ArtifactConsumer DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ArtifactConsumer(); - } - - public static flyteidl.artifact.Artifacts.ArtifactConsumer getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ArtifactConsumer parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ArtifactConsumer(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.artifact.Artifacts.ArtifactConsumer getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface RegisterConsumerRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.RegisterConsumerRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - java.util.List - getConsumersList(); - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - flyteidl.artifact.Artifacts.ArtifactConsumer getConsumers(int index); - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - int getConsumersCount(); - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - java.util.List - getConsumersOrBuilderList(); - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder getConsumersOrBuilder( - int index); - } - /** - * Protobuf type {@code flyteidl.artifact.RegisterConsumerRequest} - */ - public static final class RegisterConsumerRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.RegisterConsumerRequest) - RegisterConsumerRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use RegisterConsumerRequest.newBuilder() to construct. - private RegisterConsumerRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private RegisterConsumerRequest() { - consumers_ = java.util.Collections.emptyList(); - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private RegisterConsumerRequest( - 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: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - consumers_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - consumers_.add( - input.readMessage(flyteidl.artifact.Artifacts.ArtifactConsumer.parser(), extensionRegistry)); - 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 { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - consumers_ = java.util.Collections.unmodifiableList(consumers_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterConsumerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterConsumerRequest.class, flyteidl.artifact.Artifacts.RegisterConsumerRequest.Builder.class); - } - - public static final int CONSUMERS_FIELD_NUMBER = 1; - private java.util.List consumers_; - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public java.util.List getConsumersList() { - return consumers_; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public java.util.List - getConsumersOrBuilderList() { - return consumers_; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public int getConsumersCount() { - return consumers_.size(); - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumer getConsumers(int index) { - return consumers_.get(index); - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder getConsumersOrBuilder( - int index) { - return consumers_.get(index); - } - - 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 { - for (int i = 0; i < consumers_.size(); i++) { - output.writeMessage(1, consumers_.get(i)); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < consumers_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, consumers_.get(i)); - } - 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.artifact.Artifacts.RegisterConsumerRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.RegisterConsumerRequest other = (flyteidl.artifact.Artifacts.RegisterConsumerRequest) obj; - - if (!getConsumersList() - .equals(other.getConsumersList())) 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 (getConsumersCount() > 0) { - hash = (37 * hash) + CONSUMERS_FIELD_NUMBER; - hash = (53 * hash) + getConsumersList().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest 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.artifact.Artifacts.RegisterConsumerRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest 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.artifact.Artifacts.RegisterConsumerRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest 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.artifact.Artifacts.RegisterConsumerRequest 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.artifact.RegisterConsumerRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.RegisterConsumerRequest) - flyteidl.artifact.Artifacts.RegisterConsumerRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterConsumerRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterConsumerRequest.class, flyteidl.artifact.Artifacts.RegisterConsumerRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.RegisterConsumerRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getConsumersFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (consumersBuilder_ == null) { - consumers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - } else { - consumersBuilder_.clear(); - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterConsumerRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.RegisterConsumerRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterConsumerRequest build() { - flyteidl.artifact.Artifacts.RegisterConsumerRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterConsumerRequest buildPartial() { - flyteidl.artifact.Artifacts.RegisterConsumerRequest result = new flyteidl.artifact.Artifacts.RegisterConsumerRequest(this); - int from_bitField0_ = bitField0_; - if (consumersBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - consumers_ = java.util.Collections.unmodifiableList(consumers_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.consumers_ = consumers_; - } else { - result.consumers_ = consumersBuilder_.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.artifact.Artifacts.RegisterConsumerRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.RegisterConsumerRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.RegisterConsumerRequest other) { - if (other == flyteidl.artifact.Artifacts.RegisterConsumerRequest.getDefaultInstance()) return this; - if (consumersBuilder_ == null) { - if (!other.consumers_.isEmpty()) { - if (consumers_.isEmpty()) { - consumers_ = other.consumers_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureConsumersIsMutable(); - consumers_.addAll(other.consumers_); - } - onChanged(); - } - } else { - if (!other.consumers_.isEmpty()) { - if (consumersBuilder_.isEmpty()) { - consumersBuilder_.dispose(); - consumersBuilder_ = null; - consumers_ = other.consumers_; - bitField0_ = (bitField0_ & ~0x00000001); - consumersBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getConsumersFieldBuilder() : null; - } else { - consumersBuilder_.addAllMessages(other.consumers_); - } - } - } - 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.artifact.Artifacts.RegisterConsumerRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.RegisterConsumerRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private java.util.List consumers_ = - java.util.Collections.emptyList(); - private void ensureConsumersIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - consumers_ = new java.util.ArrayList(consumers_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactConsumer, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder, flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder> consumersBuilder_; - - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public java.util.List getConsumersList() { - if (consumersBuilder_ == null) { - return java.util.Collections.unmodifiableList(consumers_); - } else { - return consumersBuilder_.getMessageList(); - } - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public int getConsumersCount() { - if (consumersBuilder_ == null) { - return consumers_.size(); - } else { - return consumersBuilder_.getCount(); - } - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumer getConsumers(int index) { - if (consumersBuilder_ == null) { - return consumers_.get(index); - } else { - return consumersBuilder_.getMessage(index); - } - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder setConsumers( - int index, flyteidl.artifact.Artifacts.ArtifactConsumer value) { - if (consumersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureConsumersIsMutable(); - consumers_.set(index, value); - onChanged(); - } else { - consumersBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder setConsumers( - int index, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder builderForValue) { - if (consumersBuilder_ == null) { - ensureConsumersIsMutable(); - consumers_.set(index, builderForValue.build()); - onChanged(); - } else { - consumersBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder addConsumers(flyteidl.artifact.Artifacts.ArtifactConsumer value) { - if (consumersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureConsumersIsMutable(); - consumers_.add(value); - onChanged(); - } else { - consumersBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder addConsumers( - int index, flyteidl.artifact.Artifacts.ArtifactConsumer value) { - if (consumersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureConsumersIsMutable(); - consumers_.add(index, value); - onChanged(); - } else { - consumersBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder addConsumers( - flyteidl.artifact.Artifacts.ArtifactConsumer.Builder builderForValue) { - if (consumersBuilder_ == null) { - ensureConsumersIsMutable(); - consumers_.add(builderForValue.build()); - onChanged(); - } else { - consumersBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder addConsumers( - int index, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder builderForValue) { - if (consumersBuilder_ == null) { - ensureConsumersIsMutable(); - consumers_.add(index, builderForValue.build()); - onChanged(); - } else { - consumersBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder addAllConsumers( - java.lang.Iterable values) { - if (consumersBuilder_ == null) { - ensureConsumersIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, consumers_); - onChanged(); - } else { - consumersBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder clearConsumers() { - if (consumersBuilder_ == null) { - consumers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - consumersBuilder_.clear(); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public Builder removeConsumers(int index) { - if (consumersBuilder_ == null) { - ensureConsumersIsMutable(); - consumers_.remove(index); - onChanged(); - } else { - consumersBuilder_.remove(index); - } - return this; - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumer.Builder getConsumersBuilder( - int index) { - return getConsumersFieldBuilder().getBuilder(index); - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder getConsumersOrBuilder( - int index) { - if (consumersBuilder_ == null) { - return consumers_.get(index); } else { - return consumersBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public java.util.List - getConsumersOrBuilderList() { - if (consumersBuilder_ != null) { - return consumersBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(consumers_); - } - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumer.Builder addConsumersBuilder() { - return getConsumersFieldBuilder().addBuilder( - flyteidl.artifact.Artifacts.ArtifactConsumer.getDefaultInstance()); - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public flyteidl.artifact.Artifacts.ArtifactConsumer.Builder addConsumersBuilder( - int index) { - return getConsumersFieldBuilder().addBuilder( - index, flyteidl.artifact.Artifacts.ArtifactConsumer.getDefaultInstance()); - } - /** - * repeated .flyteidl.artifact.ArtifactConsumer consumers = 1; - */ - public java.util.List - getConsumersBuilderList() { - return getConsumersFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactConsumer, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder, flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder> - getConsumersFieldBuilder() { - if (consumersBuilder_ == null) { - consumersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.artifact.Artifacts.ArtifactConsumer, flyteidl.artifact.Artifacts.ArtifactConsumer.Builder, flyteidl.artifact.Artifacts.ArtifactConsumerOrBuilder>( - consumers_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - consumers_ = null; - } - return consumersBuilder_; - } - @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.artifact.RegisterConsumerRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterConsumerRequest) - private static final flyteidl.artifact.Artifacts.RegisterConsumerRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.RegisterConsumerRequest(); - } - - public static flyteidl.artifact.Artifacts.RegisterConsumerRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RegisterConsumerRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new RegisterConsumerRequest(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.artifact.Artifacts.RegisterConsumerRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface RegisterResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.RegisterResponse) - com.google.protobuf.MessageOrBuilder { - } - /** - * Protobuf type {@code flyteidl.artifact.RegisterResponse} - */ - public static final class RegisterResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.RegisterResponse) - RegisterResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use RegisterResponse.newBuilder() to construct. - private RegisterResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private RegisterResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private RegisterResponse( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - 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; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterResponse.class, flyteidl.artifact.Artifacts.RegisterResponse.Builder.class); - } - - 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 { - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - 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.artifact.Artifacts.RegisterResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.RegisterResponse other = (flyteidl.artifact.Artifacts.RegisterResponse) obj; - - 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(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.RegisterResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterResponse 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.artifact.Artifacts.RegisterResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterResponse 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.artifact.Artifacts.RegisterResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.RegisterResponse 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.artifact.Artifacts.RegisterResponse 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.artifact.RegisterResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.RegisterResponse) - flyteidl.artifact.Artifacts.RegisterResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.RegisterResponse.class, flyteidl.artifact.Artifacts.RegisterResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.RegisterResponse.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(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_RegisterResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.RegisterResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterResponse build() { - flyteidl.artifact.Artifacts.RegisterResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.RegisterResponse buildPartial() { - flyteidl.artifact.Artifacts.RegisterResponse result = new flyteidl.artifact.Artifacts.RegisterResponse(this); - 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.artifact.Artifacts.RegisterResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.RegisterResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.RegisterResponse other) { - if (other == flyteidl.artifact.Artifacts.RegisterResponse.getDefaultInstance()) return this; - 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.artifact.Artifacts.RegisterResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.RegisterResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - @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.artifact.RegisterResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.RegisterResponse) - private static final flyteidl.artifact.Artifacts.RegisterResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.RegisterResponse(); - } - - public static flyteidl.artifact.Artifacts.RegisterResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public RegisterResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new RegisterResponse(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.artifact.Artifacts.RegisterResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ExecutionInputsRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ExecutionInputsRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - boolean hasExecutionId(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutionId(); - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionIdOrBuilder(); - - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - java.util.List - getInputsList(); - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - flyteidl.core.ArtifactId.ArtifactID getInputs(int index); - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - int getInputsCount(); - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - java.util.List - getInputsOrBuilderList(); - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - flyteidl.core.ArtifactId.ArtifactIDOrBuilder getInputsOrBuilder( - int index); - } - /** - * Protobuf type {@code flyteidl.artifact.ExecutionInputsRequest} - */ - public static final class ExecutionInputsRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ExecutionInputsRequest) - ExecutionInputsRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use ExecutionInputsRequest.newBuilder() to construct. - private ExecutionInputsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ExecutionInputsRequest() { - inputs_ = java.util.Collections.emptyList(); - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ExecutionInputsRequest( - 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.WorkflowExecutionIdentifier.Builder subBuilder = null; - if (executionId_ != null) { - subBuilder = executionId_.toBuilder(); - } - executionId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(executionId_); - executionId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - inputs_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - inputs_.add( - input.readMessage(flyteidl.core.ArtifactId.ArtifactID.parser(), extensionRegistry)); - 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 { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - inputs_ = java.util.Collections.unmodifiableList(inputs_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ExecutionInputsRequest.class, flyteidl.artifact.Artifacts.ExecutionInputsRequest.Builder.class); - } - - private int bitField0_; - public static final int EXECUTION_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier executionId_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public boolean hasExecutionId() { - return executionId_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutionId() { - return executionId_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : executionId_; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionIdOrBuilder() { - return getExecutionId(); - } - - public static final int INPUTS_FIELD_NUMBER = 2; - private java.util.List inputs_; - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public java.util.List getInputsList() { - return inputs_; - } - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public java.util.List - getInputsOrBuilderList() { - return inputs_; - } - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public int getInputsCount() { - return inputs_.size(); - } - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID getInputs(int index) { - return inputs_.get(index); - } - /** - *
-     * can make this a map in the future, currently no need.
-     * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getInputsOrBuilder( - int index) { - return inputs_.get(index); - } - - 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 (executionId_ != null) { - output.writeMessage(1, getExecutionId()); - } - for (int i = 0; i < inputs_.size(); i++) { - output.writeMessage(2, inputs_.get(i)); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (executionId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getExecutionId()); - } - for (int i = 0; i < inputs_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, inputs_.get(i)); - } - 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.artifact.Artifacts.ExecutionInputsRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ExecutionInputsRequest other = (flyteidl.artifact.Artifacts.ExecutionInputsRequest) obj; - - if (hasExecutionId() != other.hasExecutionId()) return false; - if (hasExecutionId()) { - if (!getExecutionId() - .equals(other.getExecutionId())) return false; - } - if (!getInputsList() - .equals(other.getInputsList())) 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 (hasExecutionId()) { - hash = (37 * hash) + EXECUTION_ID_FIELD_NUMBER; - hash = (53 * hash) + getExecutionId().hashCode(); - } - if (getInputsCount() > 0) { - hash = (37 * hash) + INPUTS_FIELD_NUMBER; - hash = (53 * hash) + getInputsList().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest 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.artifact.Artifacts.ExecutionInputsRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest 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.artifact.Artifacts.ExecutionInputsRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest 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.artifact.Artifacts.ExecutionInputsRequest 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.artifact.ExecutionInputsRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ExecutionInputsRequest) - flyteidl.artifact.Artifacts.ExecutionInputsRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ExecutionInputsRequest.class, flyteidl.artifact.Artifacts.ExecutionInputsRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ExecutionInputsRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getInputsFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (executionIdBuilder_ == null) { - executionId_ = null; - } else { - executionId_ = null; - executionIdBuilder_ = null; - } - if (inputsBuilder_ == null) { - inputs_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - } else { - inputsBuilder_.clear(); - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ExecutionInputsRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsRequest build() { - flyteidl.artifact.Artifacts.ExecutionInputsRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsRequest buildPartial() { - flyteidl.artifact.Artifacts.ExecutionInputsRequest result = new flyteidl.artifact.Artifacts.ExecutionInputsRequest(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (executionIdBuilder_ == null) { - result.executionId_ = executionId_; - } else { - result.executionId_ = executionIdBuilder_.build(); - } - if (inputsBuilder_ == null) { - if (((bitField0_ & 0x00000002) != 0)) { - inputs_ = java.util.Collections.unmodifiableList(inputs_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.inputs_ = inputs_; - } else { - result.inputs_ = inputsBuilder_.build(); - } - result.bitField0_ = to_bitField0_; - 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.artifact.Artifacts.ExecutionInputsRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.ExecutionInputsRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ExecutionInputsRequest other) { - if (other == flyteidl.artifact.Artifacts.ExecutionInputsRequest.getDefaultInstance()) return this; - if (other.hasExecutionId()) { - mergeExecutionId(other.getExecutionId()); - } - if (inputsBuilder_ == null) { - if (!other.inputs_.isEmpty()) { - if (inputs_.isEmpty()) { - inputs_ = other.inputs_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureInputsIsMutable(); - inputs_.addAll(other.inputs_); - } - onChanged(); - } - } else { - if (!other.inputs_.isEmpty()) { - if (inputsBuilder_.isEmpty()) { - inputsBuilder_.dispose(); - inputsBuilder_ = null; - inputs_ = other.inputs_; - bitField0_ = (bitField0_ & ~0x00000002); - inputsBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getInputsFieldBuilder() : null; - } else { - inputsBuilder_.addAllMessages(other.inputs_); - } - } - } - 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.artifact.Artifacts.ExecutionInputsRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ExecutionInputsRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier executionId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> executionIdBuilder_; - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public boolean hasExecutionId() { - return executionIdBuilder_ != null || executionId_ != null; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutionId() { - if (executionIdBuilder_ == null) { - return executionId_ == null ? flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : executionId_; - } else { - return executionIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public Builder setExecutionId(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (executionIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - executionId_ = value; - onChanged(); - } else { - executionIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public Builder setExecutionId( - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (executionIdBuilder_ == null) { - executionId_ = builderForValue.build(); - onChanged(); - } else { - executionIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public Builder mergeExecutionId(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (executionIdBuilder_ == null) { - if (executionId_ != null) { - executionId_ = - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.newBuilder(executionId_).mergeFrom(value).buildPartial(); - } else { - executionId_ = value; - } - onChanged(); - } else { - executionIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public Builder clearExecutionId() { - if (executionIdBuilder_ == null) { - executionId_ = null; - onChanged(); - } else { - executionId_ = null; - executionIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder getExecutionIdBuilder() { - - onChanged(); - return getExecutionIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionIdOrBuilder() { - if (executionIdBuilder_ != null) { - return executionIdBuilder_.getMessageOrBuilder(); - } else { - return executionId_ == null ? - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance() : executionId_; - } - } - /** - * .flyteidl.core.WorkflowExecutionIdentifier execution_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> - getExecutionIdFieldBuilder() { - if (executionIdBuilder_ == null) { - executionIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder>( - getExecutionId(), - getParentForChildren(), - isClean()); - executionId_ = null; - } - return executionIdBuilder_; - } - - private java.util.List inputs_ = - java.util.Collections.emptyList(); - private void ensureInputsIsMutable() { - if (!((bitField0_ & 0x00000002) != 0)) { - inputs_ = new java.util.ArrayList(inputs_); - bitField0_ |= 0x00000002; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> inputsBuilder_; - - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public java.util.List getInputsList() { - if (inputsBuilder_ == null) { - return java.util.Collections.unmodifiableList(inputs_); - } else { - return inputsBuilder_.getMessageList(); - } - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public int getInputsCount() { - if (inputsBuilder_ == null) { - return inputs_.size(); - } else { - return inputsBuilder_.getCount(); - } - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID getInputs(int index) { - if (inputsBuilder_ == null) { - return inputs_.get(index); - } else { - return inputsBuilder_.getMessage(index); - } - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder setInputs( - int index, flyteidl.core.ArtifactId.ArtifactID value) { - if (inputsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureInputsIsMutable(); - inputs_.set(index, value); - onChanged(); - } else { - inputsBuilder_.setMessage(index, value); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder setInputs( - int index, flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (inputsBuilder_ == null) { - ensureInputsIsMutable(); - inputs_.set(index, builderForValue.build()); - onChanged(); - } else { - inputsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder addInputs(flyteidl.core.ArtifactId.ArtifactID value) { - if (inputsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureInputsIsMutable(); - inputs_.add(value); - onChanged(); - } else { - inputsBuilder_.addMessage(value); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder addInputs( - int index, flyteidl.core.ArtifactId.ArtifactID value) { - if (inputsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureInputsIsMutable(); - inputs_.add(index, value); - onChanged(); - } else { - inputsBuilder_.addMessage(index, value); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder addInputs( - flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (inputsBuilder_ == null) { - ensureInputsIsMutable(); - inputs_.add(builderForValue.build()); - onChanged(); - } else { - inputsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder addInputs( - int index, flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (inputsBuilder_ == null) { - ensureInputsIsMutable(); - inputs_.add(index, builderForValue.build()); - onChanged(); - } else { - inputsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder addAllInputs( - java.lang.Iterable values) { - if (inputsBuilder_ == null) { - ensureInputsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, inputs_); - onChanged(); - } else { - inputsBuilder_.addAllMessages(values); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder clearInputs() { - if (inputsBuilder_ == null) { - inputs_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - } else { - inputsBuilder_.clear(); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public Builder removeInputs(int index) { - if (inputsBuilder_ == null) { - ensureInputsIsMutable(); - inputs_.remove(index); - onChanged(); - } else { - inputsBuilder_.remove(index); - } - return this; - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder getInputsBuilder( - int index) { - return getInputsFieldBuilder().getBuilder(index); - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getInputsOrBuilder( - int index) { - if (inputsBuilder_ == null) { - return inputs_.get(index); } else { - return inputsBuilder_.getMessageOrBuilder(index); - } - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public java.util.List - getInputsOrBuilderList() { - if (inputsBuilder_ != null) { - return inputsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(inputs_); - } - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder addInputsBuilder() { - return getInputsFieldBuilder().addBuilder( - flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance()); - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder addInputsBuilder( - int index) { - return getInputsFieldBuilder().addBuilder( - index, flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance()); - } - /** - *
-       * can make this a map in the future, currently no need.
-       * 
- * - * repeated .flyteidl.core.ArtifactID inputs = 2; - */ - public java.util.List - getInputsBuilderList() { - return getInputsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> - getInputsFieldBuilder() { - if (inputsBuilder_ == null) { - inputsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder>( - inputs_, - ((bitField0_ & 0x00000002) != 0), - getParentForChildren(), - isClean()); - inputs_ = null; - } - return inputsBuilder_; - } - @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.artifact.ExecutionInputsRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ExecutionInputsRequest) - private static final flyteidl.artifact.Artifacts.ExecutionInputsRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ExecutionInputsRequest(); - } - - public static flyteidl.artifact.Artifacts.ExecutionInputsRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ExecutionInputsRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ExecutionInputsRequest(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.artifact.Artifacts.ExecutionInputsRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ExecutionInputsResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ExecutionInputsResponse) - com.google.protobuf.MessageOrBuilder { - } - /** - * Protobuf type {@code flyteidl.artifact.ExecutionInputsResponse} - */ - public static final class ExecutionInputsResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ExecutionInputsResponse) - ExecutionInputsResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use ExecutionInputsResponse.newBuilder() to construct. - private ExecutionInputsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ExecutionInputsResponse() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ExecutionInputsResponse( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - 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; - 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.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ExecutionInputsResponse.class, flyteidl.artifact.Artifacts.ExecutionInputsResponse.Builder.class); - } - - 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 { - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - 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.artifact.Artifacts.ExecutionInputsResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ExecutionInputsResponse other = (flyteidl.artifact.Artifacts.ExecutionInputsResponse) obj; - - 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(); - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse 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.artifact.Artifacts.ExecutionInputsResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse 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.artifact.Artifacts.ExecutionInputsResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse 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.artifact.Artifacts.ExecutionInputsResponse 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.artifact.ExecutionInputsResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ExecutionInputsResponse) - flyteidl.artifact.Artifacts.ExecutionInputsResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ExecutionInputsResponse.class, flyteidl.artifact.Artifacts.ExecutionInputsResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ExecutionInputsResponse.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(); - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ExecutionInputsResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsResponse build() { - flyteidl.artifact.Artifacts.ExecutionInputsResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ExecutionInputsResponse buildPartial() { - flyteidl.artifact.Artifacts.ExecutionInputsResponse result = new flyteidl.artifact.Artifacts.ExecutionInputsResponse(this); - 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.artifact.Artifacts.ExecutionInputsResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.ExecutionInputsResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ExecutionInputsResponse other) { - if (other == flyteidl.artifact.Artifacts.ExecutionInputsResponse.getDefaultInstance()) return this; - 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.artifact.Artifacts.ExecutionInputsResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ExecutionInputsResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - @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.artifact.ExecutionInputsResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ExecutionInputsResponse) - private static final flyteidl.artifact.Artifacts.ExecutionInputsResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ExecutionInputsResponse(); - } - - public static flyteidl.artifact.Artifacts.ExecutionInputsResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ExecutionInputsResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ExecutionInputsResponse(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.artifact.Artifacts.ExecutionInputsResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ListUsageRequestOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ListUsageRequest) - com.google.protobuf.MessageOrBuilder { - - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - boolean hasArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactID getArtifactId(); - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder(); - } - /** - * Protobuf type {@code flyteidl.artifact.ListUsageRequest} - */ - public static final class ListUsageRequest extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ListUsageRequest) - ListUsageRequestOrBuilder { - private static final long serialVersionUID = 0L; - // Use ListUsageRequest.newBuilder() to construct. - private ListUsageRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ListUsageRequest() { - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ListUsageRequest( - 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.ArtifactId.ArtifactID.Builder subBuilder = null; - if (artifactId_ != null) { - subBuilder = artifactId_.toBuilder(); - } - artifactId_ = input.readMessage(flyteidl.core.ArtifactId.ArtifactID.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(artifactId_); - artifactId_ = 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.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ListUsageRequest.class, flyteidl.artifact.Artifacts.ListUsageRequest.Builder.class); - } - - public static final int ARTIFACT_ID_FIELD_NUMBER = 1; - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - return getArtifactId(); - } - - 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 (artifactId_ != null) { - output.writeMessage(1, getArtifactId()); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (artifactId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getArtifactId()); - } - 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.artifact.Artifacts.ListUsageRequest)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ListUsageRequest other = (flyteidl.artifact.Artifacts.ListUsageRequest) obj; - - if (hasArtifactId() != other.hasArtifactId()) return false; - if (hasArtifactId()) { - if (!getArtifactId() - .equals(other.getArtifactId())) 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 (hasArtifactId()) { - hash = (37 * hash) + ARTIFACT_ID_FIELD_NUMBER; - hash = (53 * hash) + getArtifactId().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest 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.artifact.Artifacts.ListUsageRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest 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.artifact.Artifacts.ListUsageRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageRequest 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.artifact.Artifacts.ListUsageRequest 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.artifact.ListUsageRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ListUsageRequest) - flyteidl.artifact.Artifacts.ListUsageRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageRequest_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ListUsageRequest.class, flyteidl.artifact.Artifacts.ListUsageRequest.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ListUsageRequest.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 (artifactIdBuilder_ == null) { - artifactId_ = null; - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageRequest_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageRequest getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ListUsageRequest.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageRequest build() { - flyteidl.artifact.Artifacts.ListUsageRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageRequest buildPartial() { - flyteidl.artifact.Artifacts.ListUsageRequest result = new flyteidl.artifact.Artifacts.ListUsageRequest(this); - if (artifactIdBuilder_ == null) { - result.artifactId_ = artifactId_; - } else { - result.artifactId_ = artifactIdBuilder_.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.artifact.Artifacts.ListUsageRequest) { - return mergeFrom((flyteidl.artifact.Artifacts.ListUsageRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ListUsageRequest other) { - if (other == flyteidl.artifact.Artifacts.ListUsageRequest.getDefaultInstance()) return this; - if (other.hasArtifactId()) { - mergeArtifactId(other.getArtifactId()); - } - 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.artifact.Artifacts.ListUsageRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ListUsageRequest) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - - private flyteidl.core.ArtifactId.ArtifactID artifactId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> artifactIdBuilder_; - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public boolean hasArtifactId() { - return artifactIdBuilder_ != null || artifactId_ != null; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID getArtifactId() { - if (artifactIdBuilder_ == null) { - return artifactId_ == null ? flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } else { - return artifactIdBuilder_.getMessage(); - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - artifactId_ = value; - onChanged(); - } else { - artifactIdBuilder_.setMessage(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder setArtifactId( - flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (artifactIdBuilder_ == null) { - artifactId_ = builderForValue.build(); - onChanged(); - } else { - artifactIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder mergeArtifactId(flyteidl.core.ArtifactId.ArtifactID value) { - if (artifactIdBuilder_ == null) { - if (artifactId_ != null) { - artifactId_ = - flyteidl.core.ArtifactId.ArtifactID.newBuilder(artifactId_).mergeFrom(value).buildPartial(); - } else { - artifactId_ = value; - } - onChanged(); - } else { - artifactIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public Builder clearArtifactId() { - if (artifactIdBuilder_ == null) { - artifactId_ = null; - onChanged(); - } else { - artifactId_ = null; - artifactIdBuilder_ = null; - } - - return this; - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder getArtifactIdBuilder() { - - onChanged(); - return getArtifactIdFieldBuilder().getBuilder(); - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getArtifactIdOrBuilder() { - if (artifactIdBuilder_ != null) { - return artifactIdBuilder_.getMessageOrBuilder(); - } else { - return artifactId_ == null ? - flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance() : artifactId_; - } - } - /** - * .flyteidl.core.ArtifactID artifact_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> - getArtifactIdFieldBuilder() { - if (artifactIdBuilder_ == null) { - artifactIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder>( - getArtifactId(), - getParentForChildren(), - isClean()); - artifactId_ = null; - } - return artifactIdBuilder_; - } - @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.artifact.ListUsageRequest) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ListUsageRequest) - private static final flyteidl.artifact.Artifacts.ListUsageRequest DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ListUsageRequest(); - } - - public static flyteidl.artifact.Artifacts.ListUsageRequest getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ListUsageRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ListUsageRequest(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.artifact.Artifacts.ListUsageRequest getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - public interface ListUsageResponseOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.artifact.ListUsageResponse) - com.google.protobuf.MessageOrBuilder { - - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - java.util.List - getExecutionsList(); - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutions(int index); - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - int getExecutionsCount(); - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - java.util.List - getExecutionsOrBuilderList(); - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionsOrBuilder( - int index); - } - /** - * Protobuf type {@code flyteidl.artifact.ListUsageResponse} - */ - public static final class ListUsageResponse extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.artifact.ListUsageResponse) - ListUsageResponseOrBuilder { - private static final long serialVersionUID = 0L; - // Use ListUsageResponse.newBuilder() to construct. - private ListUsageResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private ListUsageResponse() { - executions_ = java.util.Collections.emptyList(); - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private ListUsageResponse( - 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: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - executions_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - executions_.add( - input.readMessage(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.parser(), extensionRegistry)); - 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 { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - executions_ = java.util.Collections.unmodifiableList(executions_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ListUsageResponse.class, flyteidl.artifact.Artifacts.ListUsageResponse.Builder.class); - } - - public static final int EXECUTIONS_FIELD_NUMBER = 1; - private java.util.List executions_; - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public java.util.List getExecutionsList() { - return executions_; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public java.util.List - getExecutionsOrBuilderList() { - return executions_; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public int getExecutionsCount() { - return executions_.size(); - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutions(int index) { - return executions_.get(index); - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionsOrBuilder( - int index) { - return executions_.get(index); - } - - 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 { - for (int i = 0; i < executions_.size(); i++) { - output.writeMessage(1, executions_.get(i)); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < executions_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, executions_.get(i)); - } - 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.artifact.Artifacts.ListUsageResponse)) { - return super.equals(obj); - } - flyteidl.artifact.Artifacts.ListUsageResponse other = (flyteidl.artifact.Artifacts.ListUsageResponse) obj; - - if (!getExecutionsList() - .equals(other.getExecutionsList())) 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 (getExecutionsCount() > 0) { - hash = (37 * hash) + EXECUTIONS_FIELD_NUMBER; - hash = (53 * hash) + getExecutionsList().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse 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.artifact.Artifacts.ListUsageResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse 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.artifact.Artifacts.ListUsageResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.artifact.Artifacts.ListUsageResponse 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.artifact.Artifacts.ListUsageResponse 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.artifact.ListUsageResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.artifact.ListUsageResponse) - flyteidl.artifact.Artifacts.ListUsageResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageResponse_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.artifact.Artifacts.ListUsageResponse.class, flyteidl.artifact.Artifacts.ListUsageResponse.Builder.class); - } - - // Construct using flyteidl.artifact.Artifacts.ListUsageResponse.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getExecutionsFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (executionsBuilder_ == null) { - executions_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - } else { - executionsBuilder_.clear(); - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.artifact.Artifacts.internal_static_flyteidl_artifact_ListUsageResponse_descriptor; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageResponse getDefaultInstanceForType() { - return flyteidl.artifact.Artifacts.ListUsageResponse.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageResponse build() { - flyteidl.artifact.Artifacts.ListUsageResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.artifact.Artifacts.ListUsageResponse buildPartial() { - flyteidl.artifact.Artifacts.ListUsageResponse result = new flyteidl.artifact.Artifacts.ListUsageResponse(this); - int from_bitField0_ = bitField0_; - if (executionsBuilder_ == null) { - if (((bitField0_ & 0x00000001) != 0)) { - executions_ = java.util.Collections.unmodifiableList(executions_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.executions_ = executions_; - } else { - result.executions_ = executionsBuilder_.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.artifact.Artifacts.ListUsageResponse) { - return mergeFrom((flyteidl.artifact.Artifacts.ListUsageResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.artifact.Artifacts.ListUsageResponse other) { - if (other == flyteidl.artifact.Artifacts.ListUsageResponse.getDefaultInstance()) return this; - if (executionsBuilder_ == null) { - if (!other.executions_.isEmpty()) { - if (executions_.isEmpty()) { - executions_ = other.executions_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureExecutionsIsMutable(); - executions_.addAll(other.executions_); - } - onChanged(); - } - } else { - if (!other.executions_.isEmpty()) { - if (executionsBuilder_.isEmpty()) { - executionsBuilder_.dispose(); - executionsBuilder_ = null; - executions_ = other.executions_; - bitField0_ = (bitField0_ & ~0x00000001); - executionsBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getExecutionsFieldBuilder() : null; - } else { - executionsBuilder_.addAllMessages(other.executions_); - } - } - } - 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.artifact.Artifacts.ListUsageResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.artifact.Artifacts.ListUsageResponse) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private java.util.List executions_ = - java.util.Collections.emptyList(); - private void ensureExecutionsIsMutable() { - if (!((bitField0_ & 0x00000001) != 0)) { - executions_ = new java.util.ArrayList(executions_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> executionsBuilder_; - - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public java.util.List getExecutionsList() { - if (executionsBuilder_ == null) { - return java.util.Collections.unmodifiableList(executions_); - } else { - return executionsBuilder_.getMessageList(); - } - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public int getExecutionsCount() { - if (executionsBuilder_ == null) { - return executions_.size(); - } else { - return executionsBuilder_.getCount(); - } - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier getExecutions(int index) { - if (executionsBuilder_ == null) { - return executions_.get(index); - } else { - return executionsBuilder_.getMessage(index); - } - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder setExecutions( - int index, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (executionsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureExecutionsIsMutable(); - executions_.set(index, value); - onChanged(); - } else { - executionsBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder setExecutions( - int index, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (executionsBuilder_ == null) { - ensureExecutionsIsMutable(); - executions_.set(index, builderForValue.build()); - onChanged(); - } else { - executionsBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder addExecutions(flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (executionsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureExecutionsIsMutable(); - executions_.add(value); - onChanged(); - } else { - executionsBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder addExecutions( - int index, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier value) { - if (executionsBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureExecutionsIsMutable(); - executions_.add(index, value); - onChanged(); - } else { - executionsBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder addExecutions( - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (executionsBuilder_ == null) { - ensureExecutionsIsMutable(); - executions_.add(builderForValue.build()); - onChanged(); - } else { - executionsBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder addExecutions( - int index, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder builderForValue) { - if (executionsBuilder_ == null) { - ensureExecutionsIsMutable(); - executions_.add(index, builderForValue.build()); - onChanged(); - } else { - executionsBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder addAllExecutions( - java.lang.Iterable values) { - if (executionsBuilder_ == null) { - ensureExecutionsIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, executions_); - onChanged(); - } else { - executionsBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder clearExecutions() { - if (executionsBuilder_ == null) { - executions_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - executionsBuilder_.clear(); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public Builder removeExecutions(int index) { - if (executionsBuilder_ == null) { - ensureExecutionsIsMutable(); - executions_.remove(index); - onChanged(); - } else { - executionsBuilder_.remove(index); - } - return this; - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder getExecutionsBuilder( - int index) { - return getExecutionsFieldBuilder().getBuilder(index); - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder getExecutionsOrBuilder( - int index) { - if (executionsBuilder_ == null) { - return executions_.get(index); } else { - return executionsBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public java.util.List - getExecutionsOrBuilderList() { - if (executionsBuilder_ != null) { - return executionsBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(executions_); - } - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder addExecutionsBuilder() { - return getExecutionsFieldBuilder().addBuilder( - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance()); - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder addExecutionsBuilder( - int index) { - return getExecutionsFieldBuilder().addBuilder( - index, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.getDefaultInstance()); - } - /** - * repeated .flyteidl.core.WorkflowExecutionIdentifier executions = 1; - */ - public java.util.List - getExecutionsBuilderList() { - return getExecutionsFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder> - getExecutionsFieldBuilder() { - if (executionsBuilder_ == null) { - executionsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifier.Builder, flyteidl.core.IdentifierOuterClass.WorkflowExecutionIdentifierOrBuilder>( - executions_, - ((bitField0_ & 0x00000001) != 0), - getParentForChildren(), - isClean()); - executions_ = null; - } - return executionsBuilder_; - } - @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.artifact.ListUsageResponse) - } - - // @@protoc_insertion_point(class_scope:flyteidl.artifact.ListUsageResponse) - private static final flyteidl.artifact.Artifacts.ListUsageResponse DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.artifact.Artifacts.ListUsageResponse(); - } - - public static flyteidl.artifact.Artifacts.ListUsageResponse getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public ListUsageResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ListUsageResponse(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.artifact.Artifacts.ListUsageResponse getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_Artifact_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_Artifact_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_CreateArtifactRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ArtifactSource_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ArtifactSource_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ArtifactSpec_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ArtifactSpec_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_CreateArtifactResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_GetArtifactRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_GetArtifactRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_GetArtifactResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_GetArtifactResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_SearchOptions_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_SearchOptions_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_SearchArtifactsRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_SearchArtifactsResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_FindByWorkflowExecRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_AddTagRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_AddTagRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_AddTagResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_AddTagResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_CreateTriggerRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_CreateTriggerResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_DeactivateTriggerRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_DeactivateTriggerResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ArtifactProducer_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ArtifactProducer_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_RegisterProducerRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ArtifactConsumer_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ArtifactConsumer_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_RegisterConsumerRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_RegisterResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_RegisterResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ExecutionInputsRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ExecutionInputsResponse_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ListUsageRequest_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ListUsageRequest_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_artifact_ListUsageResponse_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_artifact_ListUsageResponse_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n!flyteidl/artifact/artifacts.proto\022\021fly" + - "teidl.artifact\032\031google/protobuf/any.prot" + - "o\032\034google/api/annotations.proto\032\037google/" + - "protobuf/timestamp.proto\032 flyteidl/admin" + - "/launch_plan.proto\032\034flyteidl/core/litera" + - "ls.proto\032\031flyteidl/core/types.proto\032\036fly" + - "teidl/core/identifier.proto\032\037flyteidl/co" + - "re/artifact_id.proto\032\035flyteidl/core/inte" + - "rface.proto\032 flyteidl/event/cloudevents." + - "proto\"\252\001\n\010Artifact\022.\n\013artifact_id\030\001 \001(\0132" + - "\031.flyteidl.core.ArtifactID\022-\n\004spec\030\002 \001(\013" + - "2\037.flyteidl.artifact.ArtifactSpec\022\014\n\004tag" + - "s\030\003 \003(\t\0221\n\006source\030\004 \001(\0132!.flyteidl.artif" + - "act.ArtifactSource\"\367\002\n\025CreateArtifactReq" + - "uest\0220\n\014artifact_key\030\001 \001(\0132\032.flyteidl.co" + - "re.ArtifactKey\022\017\n\007version\030\003 \001(\t\022-\n\004spec\030" + - "\002 \001(\0132\037.flyteidl.artifact.ArtifactSpec\022L" + - "\n\npartitions\030\004 \003(\01328.flyteidl.artifact.C" + - "reateArtifactRequest.PartitionsEntry\0228\n\024" + - "time_partition_value\030\005 \001(\0132\032.google.prot" + - "obuf.Timestamp\0221\n\006source\030\006 \001(\0132!.flyteid" + - "l.artifact.ArtifactSource\0321\n\017PartitionsE" + - "ntry\022\013\n\003key\030\001 \001(\t\022\r\n\005value\030\002 \001(\t:\0028\001\"\277\001\n" + - "\016ArtifactSource\022F\n\022workflow_execution\030\001 " + - "\001(\0132*.flyteidl.core.WorkflowExecutionIde" + - "ntifier\022\017\n\007node_id\030\002 \001(\t\022*\n\007task_id\030\003 \001(" + - "\0132\031.flyteidl.core.Identifier\022\025\n\rretry_at" + - "tempt\030\004 \001(\r\022\021\n\tprincipal\030\005 \001(\t\"\203\002\n\014Artif" + - "actSpec\022%\n\005value\030\001 \001(\0132\026.flyteidl.core.L" + - "iteral\022(\n\004type\030\002 \001(\0132\032.flyteidl.core.Lit" + - "eralType\022\031\n\021short_description\030\003 \001(\t\022+\n\ru" + - "ser_metadata\030\004 \001(\0132\024.google.protobuf.Any" + - "\022\025\n\rmetadata_type\030\005 \001(\t\022.\n\ncreated_at\030\006 " + - "\001(\0132\032.google.protobuf.Timestamp\022\023\n\013file_" + - "format\030\007 \001(\t\"G\n\026CreateArtifactResponse\022-" + - "\n\010artifact\030\001 \001(\0132\033.flyteidl.artifact.Art" + - "ifact\"R\n\022GetArtifactRequest\022+\n\005query\030\001 \001" + - "(\0132\034.flyteidl.core.ArtifactQuery\022\017\n\007deta" + - "ils\030\002 \001(\010\"D\n\023GetArtifactResponse\022-\n\010arti" + - "fact\030\001 \001(\0132\033.flyteidl.artifact.Artifact\"" + - "A\n\rSearchOptions\022\031\n\021strict_partitions\030\001 " + - "\001(\010\022\025\n\rlatest_by_key\030\002 \001(\010\"\250\002\n\026SearchArt" + - "ifactsRequest\0220\n\014artifact_key\030\001 \001(\0132\032.fl" + - "yteidl.core.ArtifactKey\022-\n\npartitions\030\002 " + - "\001(\0132\031.flyteidl.core.Partitions\0228\n\024time_p" + - "artition_value\030\003 \001(\0132\032.google.protobuf.T" + - "imestamp\022\021\n\tprincipal\030\004 \001(\t\022\017\n\007version\030\005" + - " \001(\t\0221\n\007options\030\006 \001(\0132 .flyteidl.artifac" + - "t.SearchOptions\022\r\n\005token\030\007 \001(\t\022\r\n\005limit\030" + - "\010 \001(\005\"X\n\027SearchArtifactsResponse\022.\n\tarti" + - "facts\030\001 \003(\0132\033.flyteidl.artifact.Artifact" + - "\022\r\n\005token\030\002 \001(\t\"\311\001\n\031FindByWorkflowExecRe" + - "quest\022;\n\007exec_id\030\001 \001(\0132*.flyteidl.core.W" + - "orkflowExecutionIdentifier\022I\n\tdirection\030" + - "\002 \001(\01626.flyteidl.artifact.FindByWorkflow" + - "ExecRequest.Direction\"$\n\tDirection\022\n\n\006IN" + - "PUTS\020\000\022\013\n\007OUTPUTS\020\001\"a\n\rAddTagRequest\022.\n\013" + - "artifact_id\030\001 \001(\0132\031.flyteidl.core.Artifa" + - "ctID\022\r\n\005value\030\002 \001(\t\022\021\n\toverwrite\030\003 \001(\010\"\020" + - "\n\016AddTagResponse\"O\n\024CreateTriggerRequest" + - "\0227\n\023trigger_launch_plan\030\001 \001(\0132\032.flyteidl" + - ".admin.LaunchPlan\"\027\n\025CreateTriggerRespon" + - "se\"I\n\030DeactivateTriggerRequest\022-\n\ntrigge" + - "r_id\030\001 \001(\0132\031.flyteidl.core.Identifier\"\033\n" + - "\031DeactivateTriggerResponse\"m\n\020ArtifactPr" + - "oducer\022,\n\tentity_id\030\001 \001(\0132\031.flyteidl.cor" + - "e.Identifier\022+\n\007outputs\030\002 \001(\0132\032.flyteidl" + - ".core.VariableMap\"Q\n\027RegisterProducerReq" + - "uest\0226\n\tproducers\030\001 \003(\0132#.flyteidl.artif" + - "act.ArtifactProducer\"m\n\020ArtifactConsumer" + - "\022,\n\tentity_id\030\001 \001(\0132\031.flyteidl.core.Iden" + - "tifier\022+\n\006inputs\030\002 \001(\0132\033.flyteidl.core.P" + - "arameterMap\"Q\n\027RegisterConsumerRequest\0226" + - "\n\tconsumers\030\001 \003(\0132#.flyteidl.artifact.Ar" + - "tifactConsumer\"\022\n\020RegisterResponse\"\205\001\n\026E" + - "xecutionInputsRequest\022@\n\014execution_id\030\001 " + - "\001(\0132*.flyteidl.core.WorkflowExecutionIde" + - "ntifier\022)\n\006inputs\030\002 \003(\0132\031.flyteidl.core." + - "ArtifactID\"\031\n\027ExecutionInputsResponse\"B\n" + - "\020ListUsageRequest\022.\n\013artifact_id\030\001 \001(\0132\031" + - ".flyteidl.core.ArtifactID\"S\n\021ListUsageRe" + - "sponse\022>\n\nexecutions\030\001 \003(\0132*.flyteidl.co" + - "re.WorkflowExecutionIdentifier2\373\013\n\020Artif" + - "actRegistry\022g\n\016CreateArtifact\022(.flyteidl" + - ".artifact.CreateArtifactRequest\032).flytei" + - "dl.artifact.CreateArtifactResponse\"\000\022\204\001\n" + - "\013GetArtifact\022%.flyteidl.artifact.GetArti" + - "factRequest\032&.flyteidl.artifact.GetArtif" + - "actResponse\"&\202\323\344\223\002 \"\033/artifacts/api/v1/a" + - "rtifacts:\001*\022\215\001\n\017SearchArtifacts\022).flytei" + - "dl.artifact.SearchArtifactsRequest\032*.fly" + - "teidl.artifact.SearchArtifactsResponse\"#" + - "\202\323\344\223\002\035\"\030/artifacts/api/v1/search:\001*\022d\n\rC" + - "reateTrigger\022\'.flyteidl.artifact.CreateT" + - "riggerRequest\032(.flyteidl.artifact.Create" + - "TriggerResponse\"\000\022\237\001\n\021DeactivateTrigger\022" + - "+.flyteidl.artifact.DeactivateTriggerReq" + - "uest\032,.flyteidl.artifact.DeactivateTrigg" + - "erResponse\"/\202\323\344\223\002)2$/artifacts/api/v1/tr" + - "igger/deactivate:\001*\022O\n\006AddTag\022 .flyteidl" + - ".artifact.AddTagRequest\032!.flyteidl.artif" + - "act.AddTagResponse\"\000\022e\n\020RegisterProducer" + - "\022*.flyteidl.artifact.RegisterProducerReq" + - "uest\032#.flyteidl.artifact.RegisterRespons" + - "e\"\000\022e\n\020RegisterConsumer\022*.flyteidl.artif" + - "act.RegisterConsumerRequest\032#.flyteidl.a" + - "rtifact.RegisterResponse\"\000\022m\n\022SetExecuti" + - "onInputs\022).flyteidl.artifact.ExecutionIn" + - "putsRequest\032*.flyteidl.artifact.Executio" + - "nInputsResponse\"\000\022\330\001\n\022FindByWorkflowExec" + - "\022,.flyteidl.artifact.FindByWorkflowExecR" + - "equest\032*.flyteidl.artifact.SearchArtifac" + - "tsResponse\"h\202\323\344\223\002b\022`/artifacts/api/v1/se" + - "arch/execution/{exec_id.project}/{exec_i" + - "d.domain}/{exec_id.name}/{direction}\022\365\001\n" + - "\tListUsage\022#.flyteidl.artifact.ListUsage" + - "Request\032$.flyteidl.artifact.ListUsageRes" + - "ponse\"\234\001\202\323\344\223\002\225\001\022\222\001/artifacts/api/v1/usag" + - "e/{artifact_id.artifact_key.project}/{ar" + - "tifact_id.artifact_key.domain}/{artifact" + - "_id.artifact_key.name}/{artifact_id.vers" + - "ion}B@Z>github.com/flyteorg/flyte/flytei" + - "dl/gen/pb-go/flyteidl/artifactb\006proto3" - }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { - descriptor = root; - return null; - } - }; - com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - com.google.protobuf.AnyProto.getDescriptor(), - com.google.api.AnnotationsProto.getDescriptor(), - com.google.protobuf.TimestampProto.getDescriptor(), - flyteidl.admin.LaunchPlanOuterClass.getDescriptor(), - flyteidl.core.Literals.getDescriptor(), - flyteidl.core.Types.getDescriptor(), - flyteidl.core.IdentifierOuterClass.getDescriptor(), - flyteidl.core.ArtifactId.getDescriptor(), - flyteidl.core.Interface.getDescriptor(), - flyteidl.event.Cloudevents.getDescriptor(), - }, assigner); - internal_static_flyteidl_artifact_Artifact_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_flyteidl_artifact_Artifact_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_Artifact_descriptor, - new java.lang.String[] { "ArtifactId", "Spec", "Tags", "Source", }); - internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor = - getDescriptor().getMessageTypes().get(1); - internal_static_flyteidl_artifact_CreateArtifactRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor, - new java.lang.String[] { "ArtifactKey", "Version", "Spec", "Partitions", "TimePartitionValue", "Source", }); - internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_descriptor = - internal_static_flyteidl_artifact_CreateArtifactRequest_descriptor.getNestedTypes().get(0); - internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_CreateArtifactRequest_PartitionsEntry_descriptor, - new java.lang.String[] { "Key", "Value", }); - internal_static_flyteidl_artifact_ArtifactSource_descriptor = - getDescriptor().getMessageTypes().get(2); - internal_static_flyteidl_artifact_ArtifactSource_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ArtifactSource_descriptor, - new java.lang.String[] { "WorkflowExecution", "NodeId", "TaskId", "RetryAttempt", "Principal", }); - internal_static_flyteidl_artifact_ArtifactSpec_descriptor = - getDescriptor().getMessageTypes().get(3); - internal_static_flyteidl_artifact_ArtifactSpec_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ArtifactSpec_descriptor, - new java.lang.String[] { "Value", "Type", "ShortDescription", "UserMetadata", "MetadataType", "CreatedAt", "FileFormat", }); - internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor = - getDescriptor().getMessageTypes().get(4); - internal_static_flyteidl_artifact_CreateArtifactResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_CreateArtifactResponse_descriptor, - new java.lang.String[] { "Artifact", }); - internal_static_flyteidl_artifact_GetArtifactRequest_descriptor = - getDescriptor().getMessageTypes().get(5); - internal_static_flyteidl_artifact_GetArtifactRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_GetArtifactRequest_descriptor, - new java.lang.String[] { "Query", "Details", }); - internal_static_flyteidl_artifact_GetArtifactResponse_descriptor = - getDescriptor().getMessageTypes().get(6); - internal_static_flyteidl_artifact_GetArtifactResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_GetArtifactResponse_descriptor, - new java.lang.String[] { "Artifact", }); - internal_static_flyteidl_artifact_SearchOptions_descriptor = - getDescriptor().getMessageTypes().get(7); - internal_static_flyteidl_artifact_SearchOptions_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_SearchOptions_descriptor, - new java.lang.String[] { "StrictPartitions", "LatestByKey", }); - internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor = - getDescriptor().getMessageTypes().get(8); - internal_static_flyteidl_artifact_SearchArtifactsRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_SearchArtifactsRequest_descriptor, - new java.lang.String[] { "ArtifactKey", "Partitions", "TimePartitionValue", "Principal", "Version", "Options", "Token", "Limit", }); - internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor = - getDescriptor().getMessageTypes().get(9); - internal_static_flyteidl_artifact_SearchArtifactsResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_SearchArtifactsResponse_descriptor, - new java.lang.String[] { "Artifacts", "Token", }); - internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor = - getDescriptor().getMessageTypes().get(10); - internal_static_flyteidl_artifact_FindByWorkflowExecRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_FindByWorkflowExecRequest_descriptor, - new java.lang.String[] { "ExecId", "Direction", }); - internal_static_flyteidl_artifact_AddTagRequest_descriptor = - getDescriptor().getMessageTypes().get(11); - internal_static_flyteidl_artifact_AddTagRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_AddTagRequest_descriptor, - new java.lang.String[] { "ArtifactId", "Value", "Overwrite", }); - internal_static_flyteidl_artifact_AddTagResponse_descriptor = - getDescriptor().getMessageTypes().get(12); - internal_static_flyteidl_artifact_AddTagResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_AddTagResponse_descriptor, - new java.lang.String[] { }); - internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor = - getDescriptor().getMessageTypes().get(13); - internal_static_flyteidl_artifact_CreateTriggerRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_CreateTriggerRequest_descriptor, - new java.lang.String[] { "TriggerLaunchPlan", }); - internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor = - getDescriptor().getMessageTypes().get(14); - internal_static_flyteidl_artifact_CreateTriggerResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_CreateTriggerResponse_descriptor, - new java.lang.String[] { }); - internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor = - getDescriptor().getMessageTypes().get(15); - internal_static_flyteidl_artifact_DeactivateTriggerRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_DeactivateTriggerRequest_descriptor, - new java.lang.String[] { "TriggerId", }); - internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor = - getDescriptor().getMessageTypes().get(16); - internal_static_flyteidl_artifact_DeactivateTriggerResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_DeactivateTriggerResponse_descriptor, - new java.lang.String[] { }); - internal_static_flyteidl_artifact_ArtifactProducer_descriptor = - getDescriptor().getMessageTypes().get(17); - internal_static_flyteidl_artifact_ArtifactProducer_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ArtifactProducer_descriptor, - new java.lang.String[] { "EntityId", "Outputs", }); - internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor = - getDescriptor().getMessageTypes().get(18); - internal_static_flyteidl_artifact_RegisterProducerRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_RegisterProducerRequest_descriptor, - new java.lang.String[] { "Producers", }); - internal_static_flyteidl_artifact_ArtifactConsumer_descriptor = - getDescriptor().getMessageTypes().get(19); - internal_static_flyteidl_artifact_ArtifactConsumer_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ArtifactConsumer_descriptor, - new java.lang.String[] { "EntityId", "Inputs", }); - internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor = - getDescriptor().getMessageTypes().get(20); - internal_static_flyteidl_artifact_RegisterConsumerRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_RegisterConsumerRequest_descriptor, - new java.lang.String[] { "Consumers", }); - internal_static_flyteidl_artifact_RegisterResponse_descriptor = - getDescriptor().getMessageTypes().get(21); - internal_static_flyteidl_artifact_RegisterResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_RegisterResponse_descriptor, - new java.lang.String[] { }); - internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor = - getDescriptor().getMessageTypes().get(22); - internal_static_flyteidl_artifact_ExecutionInputsRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ExecutionInputsRequest_descriptor, - new java.lang.String[] { "ExecutionId", "Inputs", }); - internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor = - getDescriptor().getMessageTypes().get(23); - internal_static_flyteidl_artifact_ExecutionInputsResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ExecutionInputsResponse_descriptor, - new java.lang.String[] { }); - internal_static_flyteidl_artifact_ListUsageRequest_descriptor = - getDescriptor().getMessageTypes().get(24); - internal_static_flyteidl_artifact_ListUsageRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ListUsageRequest_descriptor, - new java.lang.String[] { "ArtifactId", }); - internal_static_flyteidl_artifact_ListUsageResponse_descriptor = - getDescriptor().getMessageTypes().get(25); - internal_static_flyteidl_artifact_ListUsageResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( - internal_static_flyteidl_artifact_ListUsageResponse_descriptor, - new java.lang.String[] { "Executions", }); - com.google.protobuf.ExtensionRegistry registry = - com.google.protobuf.ExtensionRegistry.newInstance(); - registry.add(com.google.api.AnnotationsProto.http); - com.google.protobuf.Descriptors.FileDescriptor - .internalUpdateFileDescriptor(descriptor, registry); - com.google.protobuf.AnyProto.getDescriptor(); - com.google.api.AnnotationsProto.getDescriptor(); - com.google.protobuf.TimestampProto.getDescriptor(); - flyteidl.admin.LaunchPlanOuterClass.getDescriptor(); - flyteidl.core.Literals.getDescriptor(); - flyteidl.core.Types.getDescriptor(); - flyteidl.core.IdentifierOuterClass.getDescriptor(); - flyteidl.core.ArtifactId.getDescriptor(); - flyteidl.core.Interface.getDescriptor(); - flyteidl.event.Cloudevents.getDescriptor(); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/flyteidl/gen/pb-java/flyteidl/core/ArtifactId.java b/flyteidl/gen/pb-java/flyteidl/core/ArtifactId.java index 3b4a1132e7..86fd69c612 100644 --- a/flyteidl/gen/pb-java/flyteidl/core/ArtifactId.java +++ b/flyteidl/gen/pb-java/flyteidl/core/ArtifactId.java @@ -8557,1184 +8557,6 @@ public flyteidl.core.ArtifactId.ArtifactQuery getDefaultInstanceForType() { } - public interface TriggerOrBuilder extends - // @@protoc_insertion_point(interface_extends:flyteidl.core.Trigger) - com.google.protobuf.MessageOrBuilder { - - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - boolean hasTriggerId(); - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - flyteidl.core.IdentifierOuterClass.Identifier getTriggerId(); - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder(); - - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - java.util.List - getTriggersList(); - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - flyteidl.core.ArtifactId.ArtifactID getTriggers(int index); - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - int getTriggersCount(); - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - java.util.List - getTriggersOrBuilderList(); - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - flyteidl.core.ArtifactId.ArtifactIDOrBuilder getTriggersOrBuilder( - int index); - } - /** - * Protobuf type {@code flyteidl.core.Trigger} - */ - public static final class Trigger extends - com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:flyteidl.core.Trigger) - TriggerOrBuilder { - private static final long serialVersionUID = 0L; - // Use Trigger.newBuilder() to construct. - private Trigger(com.google.protobuf.GeneratedMessageV3.Builder builder) { - super(builder); - } - private Trigger() { - triggers_ = java.util.Collections.emptyList(); - } - - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private Trigger( - 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.Identifier.Builder subBuilder = null; - if (triggerId_ != null) { - subBuilder = triggerId_.toBuilder(); - } - triggerId_ = input.readMessage(flyteidl.core.IdentifierOuterClass.Identifier.parser(), extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(triggerId_); - triggerId_ = subBuilder.buildPartial(); - } - - break; - } - case 18: { - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - triggers_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - triggers_.add( - input.readMessage(flyteidl.core.ArtifactId.ArtifactID.parser(), extensionRegistry)); - 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 { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - triggers_ = java.util.Collections.unmodifiableList(triggers_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.core.ArtifactId.internal_static_flyteidl_core_Trigger_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.core.ArtifactId.internal_static_flyteidl_core_Trigger_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.core.ArtifactId.Trigger.class, flyteidl.core.ArtifactId.Trigger.Builder.class); - } - - private int bitField0_; - public static final int TRIGGER_ID_FIELD_NUMBER = 1; - private flyteidl.core.IdentifierOuterClass.Identifier triggerId_; - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public boolean hasTriggerId() { - return triggerId_ != null; - } - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTriggerId() { - return triggerId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } - /** - *
-     * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-     * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder() { - return getTriggerId(); - } - - public static final int TRIGGERS_FIELD_NUMBER = 2; - private java.util.List triggers_; - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public java.util.List getTriggersList() { - return triggers_; - } - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public java.util.List - getTriggersOrBuilderList() { - return triggers_; - } - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public int getTriggersCount() { - return triggers_.size(); - } - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID getTriggers(int index) { - return triggers_.get(index); - } - /** - *
-     * These are partial artifact IDs that will be triggered on
-     * Consider making these ArtifactQuery instead.
-     * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getTriggersOrBuilder( - int index) { - return triggers_.get(index); - } - - 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 (triggerId_ != null) { - output.writeMessage(1, getTriggerId()); - } - for (int i = 0; i < triggers_.size(); i++) { - output.writeMessage(2, triggers_.get(i)); - } - unknownFields.writeTo(output); - } - - @java.lang.Override - public int getSerializedSize() { - int size = memoizedSize; - if (size != -1) return size; - - size = 0; - if (triggerId_ != null) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getTriggerId()); - } - for (int i = 0; i < triggers_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, triggers_.get(i)); - } - 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.core.ArtifactId.Trigger)) { - return super.equals(obj); - } - flyteidl.core.ArtifactId.Trigger other = (flyteidl.core.ArtifactId.Trigger) obj; - - if (hasTriggerId() != other.hasTriggerId()) return false; - if (hasTriggerId()) { - if (!getTriggerId() - .equals(other.getTriggerId())) return false; - } - if (!getTriggersList() - .equals(other.getTriggersList())) 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 (hasTriggerId()) { - hash = (37 * hash) + TRIGGER_ID_FIELD_NUMBER; - hash = (53 * hash) + getTriggerId().hashCode(); - } - if (getTriggersCount() > 0) { - hash = (37 * hash) + TRIGGERS_FIELD_NUMBER; - hash = (53 * hash) + getTriggersList().hashCode(); - } - hash = (29 * hash) + unknownFields.hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static flyteidl.core.ArtifactId.Trigger parseFrom( - java.nio.ByteBuffer data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom( - java.nio.ByteBuffer data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static flyteidl.core.ArtifactId.Trigger parseFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.core.ArtifactId.Trigger 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.core.ArtifactId.Trigger parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseDelimitedWithIOException(PARSER, input); - } - public static flyteidl.core.ArtifactId.Trigger 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.core.ArtifactId.Trigger parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 - .parseWithIOException(PARSER, input); - } - public static flyteidl.core.ArtifactId.Trigger 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.core.ArtifactId.Trigger 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.core.Trigger} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements - // @@protoc_insertion_point(builder_implements:flyteidl.core.Trigger) - flyteidl.core.ArtifactId.TriggerOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return flyteidl.core.ArtifactId.internal_static_flyteidl_core_Trigger_descriptor; - } - - @java.lang.Override - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internalGetFieldAccessorTable() { - return flyteidl.core.ArtifactId.internal_static_flyteidl_core_Trigger_fieldAccessorTable - .ensureFieldAccessorsInitialized( - flyteidl.core.ArtifactId.Trigger.class, flyteidl.core.ArtifactId.Trigger.Builder.class); - } - - // Construct using flyteidl.core.ArtifactId.Trigger.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getTriggersFieldBuilder(); - } - } - @java.lang.Override - public Builder clear() { - super.clear(); - if (triggerIdBuilder_ == null) { - triggerId_ = null; - } else { - triggerId_ = null; - triggerIdBuilder_ = null; - } - if (triggersBuilder_ == null) { - triggers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - } else { - triggersBuilder_.clear(); - } - return this; - } - - @java.lang.Override - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return flyteidl.core.ArtifactId.internal_static_flyteidl_core_Trigger_descriptor; - } - - @java.lang.Override - public flyteidl.core.ArtifactId.Trigger getDefaultInstanceForType() { - return flyteidl.core.ArtifactId.Trigger.getDefaultInstance(); - } - - @java.lang.Override - public flyteidl.core.ArtifactId.Trigger build() { - flyteidl.core.ArtifactId.Trigger result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - @java.lang.Override - public flyteidl.core.ArtifactId.Trigger buildPartial() { - flyteidl.core.ArtifactId.Trigger result = new flyteidl.core.ArtifactId.Trigger(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (triggerIdBuilder_ == null) { - result.triggerId_ = triggerId_; - } else { - result.triggerId_ = triggerIdBuilder_.build(); - } - if (triggersBuilder_ == null) { - if (((bitField0_ & 0x00000002) != 0)) { - triggers_ = java.util.Collections.unmodifiableList(triggers_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.triggers_ = triggers_; - } else { - result.triggers_ = triggersBuilder_.build(); - } - result.bitField0_ = to_bitField0_; - 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.core.ArtifactId.Trigger) { - return mergeFrom((flyteidl.core.ArtifactId.Trigger)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(flyteidl.core.ArtifactId.Trigger other) { - if (other == flyteidl.core.ArtifactId.Trigger.getDefaultInstance()) return this; - if (other.hasTriggerId()) { - mergeTriggerId(other.getTriggerId()); - } - if (triggersBuilder_ == null) { - if (!other.triggers_.isEmpty()) { - if (triggers_.isEmpty()) { - triggers_ = other.triggers_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureTriggersIsMutable(); - triggers_.addAll(other.triggers_); - } - onChanged(); - } - } else { - if (!other.triggers_.isEmpty()) { - if (triggersBuilder_.isEmpty()) { - triggersBuilder_.dispose(); - triggersBuilder_ = null; - triggers_ = other.triggers_; - bitField0_ = (bitField0_ & ~0x00000002); - triggersBuilder_ = - com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? - getTriggersFieldBuilder() : null; - } else { - triggersBuilder_.addAllMessages(other.triggers_); - } - } - } - 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.core.ArtifactId.Trigger parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (flyteidl.core.ArtifactId.Trigger) e.getUnfinishedMessage(); - throw e.unwrapIOException(); - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - private flyteidl.core.IdentifierOuterClass.Identifier triggerId_; - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> triggerIdBuilder_; - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public boolean hasTriggerId() { - return triggerIdBuilder_ != null || triggerId_ != null; - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier getTriggerId() { - if (triggerIdBuilder_ == null) { - return triggerId_ == null ? flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } else { - return triggerIdBuilder_.getMessage(); - } - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder setTriggerId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (triggerIdBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - triggerId_ = value; - onChanged(); - } else { - triggerIdBuilder_.setMessage(value); - } - - return this; - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder setTriggerId( - flyteidl.core.IdentifierOuterClass.Identifier.Builder builderForValue) { - if (triggerIdBuilder_ == null) { - triggerId_ = builderForValue.build(); - onChanged(); - } else { - triggerIdBuilder_.setMessage(builderForValue.build()); - } - - return this; - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder mergeTriggerId(flyteidl.core.IdentifierOuterClass.Identifier value) { - if (triggerIdBuilder_ == null) { - if (triggerId_ != null) { - triggerId_ = - flyteidl.core.IdentifierOuterClass.Identifier.newBuilder(triggerId_).mergeFrom(value).buildPartial(); - } else { - triggerId_ = value; - } - onChanged(); - } else { - triggerIdBuilder_.mergeFrom(value); - } - - return this; - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public Builder clearTriggerId() { - if (triggerIdBuilder_ == null) { - triggerId_ = null; - onChanged(); - } else { - triggerId_ = null; - triggerIdBuilder_ = null; - } - - return this; - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.Identifier.Builder getTriggerIdBuilder() { - - onChanged(); - return getTriggerIdFieldBuilder().getBuilder(); - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - public flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder getTriggerIdOrBuilder() { - if (triggerIdBuilder_ != null) { - return triggerIdBuilder_.getMessageOrBuilder(); - } else { - return triggerId_ == null ? - flyteidl.core.IdentifierOuterClass.Identifier.getDefaultInstance() : triggerId_; - } - } - /** - *
-       * This will be set to a launch plan type, but note that this is different than the actual launch plan type.
-       * 
- * - * .flyteidl.core.Identifier trigger_id = 1; - */ - private com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder> - getTriggerIdFieldBuilder() { - if (triggerIdBuilder_ == null) { - triggerIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< - flyteidl.core.IdentifierOuterClass.Identifier, flyteidl.core.IdentifierOuterClass.Identifier.Builder, flyteidl.core.IdentifierOuterClass.IdentifierOrBuilder>( - getTriggerId(), - getParentForChildren(), - isClean()); - triggerId_ = null; - } - return triggerIdBuilder_; - } - - private java.util.List triggers_ = - java.util.Collections.emptyList(); - private void ensureTriggersIsMutable() { - if (!((bitField0_ & 0x00000002) != 0)) { - triggers_ = new java.util.ArrayList(triggers_); - bitField0_ |= 0x00000002; - } - } - - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> triggersBuilder_; - - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public java.util.List getTriggersList() { - if (triggersBuilder_ == null) { - return java.util.Collections.unmodifiableList(triggers_); - } else { - return triggersBuilder_.getMessageList(); - } - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public int getTriggersCount() { - if (triggersBuilder_ == null) { - return triggers_.size(); - } else { - return triggersBuilder_.getCount(); - } - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID getTriggers(int index) { - if (triggersBuilder_ == null) { - return triggers_.get(index); - } else { - return triggersBuilder_.getMessage(index); - } - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder setTriggers( - int index, flyteidl.core.ArtifactId.ArtifactID value) { - if (triggersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureTriggersIsMutable(); - triggers_.set(index, value); - onChanged(); - } else { - triggersBuilder_.setMessage(index, value); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder setTriggers( - int index, flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (triggersBuilder_ == null) { - ensureTriggersIsMutable(); - triggers_.set(index, builderForValue.build()); - onChanged(); - } else { - triggersBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder addTriggers(flyteidl.core.ArtifactId.ArtifactID value) { - if (triggersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureTriggersIsMutable(); - triggers_.add(value); - onChanged(); - } else { - triggersBuilder_.addMessage(value); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder addTriggers( - int index, flyteidl.core.ArtifactId.ArtifactID value) { - if (triggersBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureTriggersIsMutable(); - triggers_.add(index, value); - onChanged(); - } else { - triggersBuilder_.addMessage(index, value); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder addTriggers( - flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (triggersBuilder_ == null) { - ensureTriggersIsMutable(); - triggers_.add(builderForValue.build()); - onChanged(); - } else { - triggersBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder addTriggers( - int index, flyteidl.core.ArtifactId.ArtifactID.Builder builderForValue) { - if (triggersBuilder_ == null) { - ensureTriggersIsMutable(); - triggers_.add(index, builderForValue.build()); - onChanged(); - } else { - triggersBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder addAllTriggers( - java.lang.Iterable values) { - if (triggersBuilder_ == null) { - ensureTriggersIsMutable(); - com.google.protobuf.AbstractMessageLite.Builder.addAll( - values, triggers_); - onChanged(); - } else { - triggersBuilder_.addAllMessages(values); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder clearTriggers() { - if (triggersBuilder_ == null) { - triggers_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - } else { - triggersBuilder_.clear(); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public Builder removeTriggers(int index) { - if (triggersBuilder_ == null) { - ensureTriggersIsMutable(); - triggers_.remove(index); - onChanged(); - } else { - triggersBuilder_.remove(index); - } - return this; - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder getTriggersBuilder( - int index) { - return getTriggersFieldBuilder().getBuilder(index); - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactIDOrBuilder getTriggersOrBuilder( - int index) { - if (triggersBuilder_ == null) { - return triggers_.get(index); } else { - return triggersBuilder_.getMessageOrBuilder(index); - } - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public java.util.List - getTriggersOrBuilderList() { - if (triggersBuilder_ != null) { - return triggersBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(triggers_); - } - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder addTriggersBuilder() { - return getTriggersFieldBuilder().addBuilder( - flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance()); - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public flyteidl.core.ArtifactId.ArtifactID.Builder addTriggersBuilder( - int index) { - return getTriggersFieldBuilder().addBuilder( - index, flyteidl.core.ArtifactId.ArtifactID.getDefaultInstance()); - } - /** - *
-       * These are partial artifact IDs that will be triggered on
-       * Consider making these ArtifactQuery instead.
-       * 
- * - * repeated .flyteidl.core.ArtifactID triggers = 2; - */ - public java.util.List - getTriggersBuilderList() { - return getTriggersFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder> - getTriggersFieldBuilder() { - if (triggersBuilder_ == null) { - triggersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< - flyteidl.core.ArtifactId.ArtifactID, flyteidl.core.ArtifactId.ArtifactID.Builder, flyteidl.core.ArtifactId.ArtifactIDOrBuilder>( - triggers_, - ((bitField0_ & 0x00000002) != 0), - getParentForChildren(), - isClean()); - triggers_ = null; - } - return triggersBuilder_; - } - @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.core.Trigger) - } - - // @@protoc_insertion_point(class_scope:flyteidl.core.Trigger) - private static final flyteidl.core.ArtifactId.Trigger DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new flyteidl.core.ArtifactId.Trigger(); - } - - public static flyteidl.core.ArtifactId.Trigger getDefaultInstance() { - return DEFAULT_INSTANCE; - } - - private static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { - @java.lang.Override - public Trigger parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new Trigger(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.core.ArtifactId.Trigger getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } - - } - private static final com.google.protobuf.Descriptors.Descriptor internal_static_flyteidl_core_ArtifactKey_descriptor; private static final @@ -9785,11 +8607,6 @@ public flyteidl.core.ArtifactId.Trigger getDefaultInstanceForType() { private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_flyteidl_core_ArtifactQuery_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor - internal_static_flyteidl_core_Trigger_descriptor; - private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable - internal_static_flyteidl_core_Trigger_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -9830,11 +8647,9 @@ public flyteidl.core.ArtifactId.Trigger getDefaultInstanceForType() { "H\000\0222\n\014artifact_tag\030\002 \001(\0132\032.flyteidl.core" + ".ArtifactTagH\000\022\r\n\003uri\030\003 \001(\tH\000\0225\n\007binding" + "\030\004 \001(\0132\".flyteidl.core.ArtifactBindingDa" + - "taH\000B\014\n\nidentifier\"e\n\007Trigger\022-\n\ntrigger" + - "_id\030\001 \001(\0132\031.flyteidl.core.Identifier\022+\n\010" + - "triggers\030\002 \003(\0132\031.flyteidl.core.ArtifactI" + - "DB|null} [triggers] Trigger triggers - */ - - /** - * Constructs a new Trigger. - * @memberof flyteidl.core - * @classdesc Represents a Trigger. - * @implements ITrigger - * @constructor - * @param {flyteidl.core.ITrigger=} [properties] Properties to set - */ - function Trigger(properties) { - this.triggers = []; - 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]]; - } - - /** - * Trigger triggerId. - * @member {flyteidl.core.IIdentifier|null|undefined} triggerId - * @memberof flyteidl.core.Trigger - * @instance - */ - Trigger.prototype.triggerId = null; - - /** - * Trigger triggers. - * @member {Array.} triggers - * @memberof flyteidl.core.Trigger - * @instance - */ - Trigger.prototype.triggers = $util.emptyArray; - - /** - * Creates a new Trigger instance using the specified properties. - * @function create - * @memberof flyteidl.core.Trigger - * @static - * @param {flyteidl.core.ITrigger=} [properties] Properties to set - * @returns {flyteidl.core.Trigger} Trigger instance - */ - Trigger.create = function create(properties) { - return new Trigger(properties); - }; - - /** - * Encodes the specified Trigger message. Does not implicitly {@link flyteidl.core.Trigger.verify|verify} messages. - * @function encode - * @memberof flyteidl.core.Trigger - * @static - * @param {flyteidl.core.ITrigger} message Trigger message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Trigger.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.triggerId != null && message.hasOwnProperty("triggerId")) - $root.flyteidl.core.Identifier.encode(message.triggerId, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.triggers != null && message.triggers.length) - for (var i = 0; i < message.triggers.length; ++i) - $root.flyteidl.core.ArtifactID.encode(message.triggers[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - return writer; - }; - - /** - * Decodes a Trigger message from the specified reader or buffer. - * @function decode - * @memberof flyteidl.core.Trigger - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {flyteidl.core.Trigger} Trigger - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Trigger.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.core.Trigger(); - while (reader.pos < end) { - var tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - message.triggerId = $root.flyteidl.core.Identifier.decode(reader, reader.uint32()); - break; - case 2: - if (!(message.triggers && message.triggers.length)) - message.triggers = []; - message.triggers.push($root.flyteidl.core.ArtifactID.decode(reader, reader.uint32())); - break; - default: - reader.skipType(tag & 7); - break; - } - } - return message; - }; - - /** - * Verifies a Trigger message. - * @function verify - * @memberof flyteidl.core.Trigger - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Trigger.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.triggerId != null && message.hasOwnProperty("triggerId")) { - var error = $root.flyteidl.core.Identifier.verify(message.triggerId); - if (error) - return "triggerId." + error; - } - if (message.triggers != null && message.hasOwnProperty("triggers")) { - if (!Array.isArray(message.triggers)) - return "triggers: array expected"; - for (var i = 0; i < message.triggers.length; ++i) { - var error = $root.flyteidl.core.ArtifactID.verify(message.triggers[i]); - if (error) - return "triggers." + error; - } - } - return null; - }; - - return Trigger; - })(); - /** * ResourceType enum. * @name flyteidl.core.ResourceType diff --git a/flyteidl/gen/pb_python/flyteidl/artifact/__init__.py b/flyteidl/gen/pb_python/flyteidl/artifact/__init__.py deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.py b/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.py deleted file mode 100644 index c253948ddb..0000000000 --- a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.py +++ /dev/null @@ -1,105 +0,0 @@ -# -*- coding: utf-8 -*- -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: flyteidl/artifact/artifacts.proto -"""Generated protocol buffer code.""" -from google.protobuf.internal import builder as _builder -from google.protobuf import descriptor as _descriptor -from google.protobuf import descriptor_pool as _descriptor_pool -from google.protobuf import symbol_database as _symbol_database -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -from google.protobuf import any_pb2 as google_dot_protobuf_dot_any__pb2 -from google.api import annotations_pb2 as google_dot_api_dot_annotations__pb2 -from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -from flyteidl.admin import launch_plan_pb2 as flyteidl_dot_admin_dot_launch__plan__pb2 -from flyteidl.core import literals_pb2 as flyteidl_dot_core_dot_literals__pb2 -from flyteidl.core import types_pb2 as flyteidl_dot_core_dot_types__pb2 -from flyteidl.core import identifier_pb2 as flyteidl_dot_core_dot_identifier__pb2 -from flyteidl.core import artifact_id_pb2 as flyteidl_dot_core_dot_artifact__id__pb2 -from flyteidl.core import interface_pb2 as flyteidl_dot_core_dot_interface__pb2 -from flyteidl.event import cloudevents_pb2 as flyteidl_dot_event_dot_cloudevents__pb2 - - -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n!flyteidl/artifact/artifacts.proto\x12\x11\x66lyteidl.artifact\x1a\x19google/protobuf/any.proto\x1a\x1cgoogle/api/annotations.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a flyteidl/admin/launch_plan.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x19\x66lyteidl/core/types.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1f\x66lyteidl/core/artifact_id.proto\x1a\x1d\x66lyteidl/core/interface.proto\x1a flyteidl/event/cloudevents.proto\"\xca\x01\n\x08\x41rtifact\x12:\n\x0b\x61rtifact_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.ArtifactIDR\nartifactId\x12\x33\n\x04spec\x18\x02 \x01(\x0b\x32\x1f.flyteidl.artifact.ArtifactSpecR\x04spec\x12\x12\n\x04tags\x18\x03 \x03(\tR\x04tags\x12\x39\n\x06source\x18\x04 \x01(\x0b\x32!.flyteidl.artifact.ArtifactSourceR\x06source\"\xc7\x03\n\x15\x43reateArtifactRequest\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12\x18\n\x07version\x18\x03 \x01(\tR\x07version\x12\x33\n\x04spec\x18\x02 \x01(\x0b\x32\x1f.flyteidl.artifact.ArtifactSpecR\x04spec\x12X\n\npartitions\x18\x04 \x03(\x0b\x32\x38.flyteidl.artifact.CreateArtifactRequest.PartitionsEntryR\npartitions\x12L\n\x14time_partition_value\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x12timePartitionValue\x12\x39\n\x06source\x18\x06 \x01(\x0b\x32!.flyteidl.artifact.ArtifactSourceR\x06source\x1a=\n\x0fPartitionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\"\xfb\x01\n\x0e\x41rtifactSource\x12Y\n\x12workflow_execution\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x11workflowExecution\x12\x17\n\x07node_id\x18\x02 \x01(\tR\x06nodeId\x12\x32\n\x07task_id\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x06taskId\x12#\n\rretry_attempt\x18\x04 \x01(\rR\x0cretryAttempt\x12\x1c\n\tprincipal\x18\x05 \x01(\tR\tprincipal\"\xd5\x02\n\x0c\x41rtifactSpec\x12,\n\x05value\x18\x01 \x01(\x0b\x32\x16.flyteidl.core.LiteralR\x05value\x12.\n\x04type\x18\x02 \x01(\x0b\x32\x1a.flyteidl.core.LiteralTypeR\x04type\x12+\n\x11short_description\x18\x03 \x01(\tR\x10shortDescription\x12\x39\n\ruser_metadata\x18\x04 \x01(\x0b\x32\x14.google.protobuf.AnyR\x0cuserMetadata\x12#\n\rmetadata_type\x18\x05 \x01(\tR\x0cmetadataType\x12\x39\n\ncreated_at\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x1f\n\x0b\x66ile_format\x18\x07 \x01(\tR\nfileFormat\"Q\n\x16\x43reateArtifactResponse\x12\x37\n\x08\x61rtifact\x18\x01 \x01(\x0b\x32\x1b.flyteidl.artifact.ArtifactR\x08\x61rtifact\"b\n\x12GetArtifactRequest\x12\x32\n\x05query\x18\x01 \x01(\x0b\x32\x1c.flyteidl.core.ArtifactQueryR\x05query\x12\x18\n\x07\x64\x65tails\x18\x02 \x01(\x08R\x07\x64\x65tails\"N\n\x13GetArtifactResponse\x12\x37\n\x08\x61rtifact\x18\x01 \x01(\x0b\x32\x1b.flyteidl.artifact.ArtifactR\x08\x61rtifact\"`\n\rSearchOptions\x12+\n\x11strict_partitions\x18\x01 \x01(\x08R\x10strictPartitions\x12\"\n\rlatest_by_key\x18\x02 \x01(\x08R\x0blatestByKey\"\x80\x03\n\x16SearchArtifactsRequest\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12\x39\n\npartitions\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.PartitionsR\npartitions\x12L\n\x14time_partition_value\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x12timePartitionValue\x12\x1c\n\tprincipal\x18\x04 \x01(\tR\tprincipal\x12\x18\n\x07version\x18\x05 \x01(\tR\x07version\x12:\n\x07options\x18\x06 \x01(\x0b\x32 .flyteidl.artifact.SearchOptionsR\x07options\x12\x14\n\x05token\x18\x07 \x01(\tR\x05token\x12\x14\n\x05limit\x18\x08 \x01(\x05R\x05limit\"j\n\x17SearchArtifactsResponse\x12\x39\n\tartifacts\x18\x01 \x03(\x0b\x32\x1b.flyteidl.artifact.ArtifactR\tartifacts\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"\xdc\x01\n\x19\x46indByWorkflowExecRequest\x12\x43\n\x07\x65xec_id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x06\x65xecId\x12T\n\tdirection\x18\x02 \x01(\x0e\x32\x36.flyteidl.artifact.FindByWorkflowExecRequest.DirectionR\tdirection\"$\n\tDirection\x12\n\n\x06INPUTS\x10\x00\x12\x0b\n\x07OUTPUTS\x10\x01\"\x7f\n\rAddTagRequest\x12:\n\x0b\x61rtifact_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.ArtifactIDR\nartifactId\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value\x12\x1c\n\toverwrite\x18\x03 \x01(\x08R\toverwrite\"\x10\n\x0e\x41\x64\x64TagResponse\"b\n\x14\x43reateTriggerRequest\x12J\n\x13trigger_launch_plan\x18\x01 \x01(\x0b\x32\x1a.flyteidl.admin.LaunchPlanR\x11triggerLaunchPlan\"\x17\n\x15\x43reateTriggerResponse\"T\n\x18\x44\x65\x61\x63tivateTriggerRequest\x12\x38\n\ntrigger_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\ttriggerId\"\x1b\n\x19\x44\x65\x61\x63tivateTriggerResponse\"\x80\x01\n\x10\x41rtifactProducer\x12\x36\n\tentity_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x08\x65ntityId\x12\x34\n\x07outputs\x18\x02 \x01(\x0b\x32\x1a.flyteidl.core.VariableMapR\x07outputs\"\\\n\x17RegisterProducerRequest\x12\x41\n\tproducers\x18\x01 \x03(\x0b\x32#.flyteidl.artifact.ArtifactProducerR\tproducers\"\x7f\n\x10\x41rtifactConsumer\x12\x36\n\tentity_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x08\x65ntityId\x12\x33\n\x06inputs\x18\x02 \x01(\x0b\x32\x1b.flyteidl.core.ParameterMapR\x06inputs\"\\\n\x17RegisterConsumerRequest\x12\x41\n\tconsumers\x18\x01 \x03(\x0b\x32#.flyteidl.artifact.ArtifactConsumerR\tconsumers\"\x12\n\x10RegisterResponse\"\x9a\x01\n\x16\x45xecutionInputsRequest\x12M\n\x0c\x65xecution_id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x0b\x65xecutionId\x12\x31\n\x06inputs\x18\x02 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x06inputs\"\x19\n\x17\x45xecutionInputsResponse\"N\n\x10ListUsageRequest\x12:\n\x0b\x61rtifact_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.ArtifactIDR\nartifactId\"_\n\x11ListUsageResponse\x12J\n\nexecutions\x18\x01 \x03(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\nexecutions2\xfb\x0b\n\x10\x41rtifactRegistry\x12g\n\x0e\x43reateArtifact\x12(.flyteidl.artifact.CreateArtifactRequest\x1a).flyteidl.artifact.CreateArtifactResponse\"\x00\x12\x84\x01\n\x0bGetArtifact\x12%.flyteidl.artifact.GetArtifactRequest\x1a&.flyteidl.artifact.GetArtifactResponse\"&\x82\xd3\xe4\x93\x02 :\x01*\"\x1b/artifacts/api/v1/artifacts\x12\x8d\x01\n\x0fSearchArtifacts\x12).flyteidl.artifact.SearchArtifactsRequest\x1a*.flyteidl.artifact.SearchArtifactsResponse\"#\x82\xd3\xe4\x93\x02\x1d:\x01*\"\x18/artifacts/api/v1/search\x12\x64\n\rCreateTrigger\x12\'.flyteidl.artifact.CreateTriggerRequest\x1a(.flyteidl.artifact.CreateTriggerResponse\"\x00\x12\x9f\x01\n\x11\x44\x65\x61\x63tivateTrigger\x12+.flyteidl.artifact.DeactivateTriggerRequest\x1a,.flyteidl.artifact.DeactivateTriggerResponse\"/\x82\xd3\xe4\x93\x02):\x01*2$/artifacts/api/v1/trigger/deactivate\x12O\n\x06\x41\x64\x64Tag\x12 .flyteidl.artifact.AddTagRequest\x1a!.flyteidl.artifact.AddTagResponse\"\x00\x12\x65\n\x10RegisterProducer\x12*.flyteidl.artifact.RegisterProducerRequest\x1a#.flyteidl.artifact.RegisterResponse\"\x00\x12\x65\n\x10RegisterConsumer\x12*.flyteidl.artifact.RegisterConsumerRequest\x1a#.flyteidl.artifact.RegisterResponse\"\x00\x12m\n\x12SetExecutionInputs\x12).flyteidl.artifact.ExecutionInputsRequest\x1a*.flyteidl.artifact.ExecutionInputsResponse\"\x00\x12\xd8\x01\n\x12\x46indByWorkflowExec\x12,.flyteidl.artifact.FindByWorkflowExecRequest\x1a*.flyteidl.artifact.SearchArtifactsResponse\"h\x82\xd3\xe4\x93\x02\x62\x12`/artifacts/api/v1/search/execution/{exec_id.project}/{exec_id.domain}/{exec_id.name}/{direction}\x12\xf5\x01\n\tListUsage\x12#.flyteidl.artifact.ListUsageRequest\x1a$.flyteidl.artifact.ListUsageResponse\"\x9c\x01\x82\xd3\xe4\x93\x02\x95\x01\x12\x92\x01/artifacts/api/v1/usage/{artifact_id.artifact_key.project}/{artifact_id.artifact_key.domain}/{artifact_id.artifact_key.name}/{artifact_id.version}B\xcc\x01\n\x15\x63om.flyteidl.artifactB\x0e\x41rtifactsProtoP\x01Z>github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact\xa2\x02\x03\x46\x41X\xaa\x02\x11\x46lyteidl.Artifact\xca\x02\x11\x46lyteidl\\Artifact\xe2\x02\x1d\x46lyteidl\\Artifact\\GPBMetadata\xea\x02\x12\x46lyteidl::Artifactb\x06proto3') - -_globals = globals() -_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) -_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'flyteidl.artifact.artifacts_pb2', _globals) -if _descriptor._USE_C_DESCRIPTORS == False: - - DESCRIPTOR._options = None - DESCRIPTOR._serialized_options = b'\n\025com.flyteidl.artifactB\016ArtifactsProtoP\001Z>github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact\242\002\003FAX\252\002\021Flyteidl.Artifact\312\002\021Flyteidl\\Artifact\342\002\035Flyteidl\\Artifact\\GPBMetadata\352\002\022Flyteidl::Artifact' - _CREATEARTIFACTREQUEST_PARTITIONSENTRY._options = None - _CREATEARTIFACTREQUEST_PARTITIONSENTRY._serialized_options = b'8\001' - _ARTIFACTREGISTRY.methods_by_name['GetArtifact']._options = None - _ARTIFACTREGISTRY.methods_by_name['GetArtifact']._serialized_options = b'\202\323\344\223\002 :\001*\"\033/artifacts/api/v1/artifacts' - _ARTIFACTREGISTRY.methods_by_name['SearchArtifacts']._options = None - _ARTIFACTREGISTRY.methods_by_name['SearchArtifacts']._serialized_options = b'\202\323\344\223\002\035:\001*\"\030/artifacts/api/v1/search' - _ARTIFACTREGISTRY.methods_by_name['DeactivateTrigger']._options = None - _ARTIFACTREGISTRY.methods_by_name['DeactivateTrigger']._serialized_options = b'\202\323\344\223\002):\001*2$/artifacts/api/v1/trigger/deactivate' - _ARTIFACTREGISTRY.methods_by_name['FindByWorkflowExec']._options = None - _ARTIFACTREGISTRY.methods_by_name['FindByWorkflowExec']._serialized_options = b'\202\323\344\223\002b\022`/artifacts/api/v1/search/execution/{exec_id.project}/{exec_id.domain}/{exec_id.name}/{direction}' - _ARTIFACTREGISTRY.methods_by_name['ListUsage']._options = None - _ARTIFACTREGISTRY.methods_by_name['ListUsage']._serialized_options = b'\202\323\344\223\002\225\001\022\222\001/artifacts/api/v1/usage/{artifact_id.artifact_key.project}/{artifact_id.artifact_key.domain}/{artifact_id.artifact_key.name}/{artifact_id.version}' - _globals['_ARTIFACT']._serialized_start=368 - _globals['_ARTIFACT']._serialized_end=570 - _globals['_CREATEARTIFACTREQUEST']._serialized_start=573 - _globals['_CREATEARTIFACTREQUEST']._serialized_end=1028 - _globals['_CREATEARTIFACTREQUEST_PARTITIONSENTRY']._serialized_start=967 - _globals['_CREATEARTIFACTREQUEST_PARTITIONSENTRY']._serialized_end=1028 - _globals['_ARTIFACTSOURCE']._serialized_start=1031 - _globals['_ARTIFACTSOURCE']._serialized_end=1282 - _globals['_ARTIFACTSPEC']._serialized_start=1285 - _globals['_ARTIFACTSPEC']._serialized_end=1626 - _globals['_CREATEARTIFACTRESPONSE']._serialized_start=1628 - _globals['_CREATEARTIFACTRESPONSE']._serialized_end=1709 - _globals['_GETARTIFACTREQUEST']._serialized_start=1711 - _globals['_GETARTIFACTREQUEST']._serialized_end=1809 - _globals['_GETARTIFACTRESPONSE']._serialized_start=1811 - _globals['_GETARTIFACTRESPONSE']._serialized_end=1889 - _globals['_SEARCHOPTIONS']._serialized_start=1891 - _globals['_SEARCHOPTIONS']._serialized_end=1987 - _globals['_SEARCHARTIFACTSREQUEST']._serialized_start=1990 - _globals['_SEARCHARTIFACTSREQUEST']._serialized_end=2374 - _globals['_SEARCHARTIFACTSRESPONSE']._serialized_start=2376 - _globals['_SEARCHARTIFACTSRESPONSE']._serialized_end=2482 - _globals['_FINDBYWORKFLOWEXECREQUEST']._serialized_start=2485 - _globals['_FINDBYWORKFLOWEXECREQUEST']._serialized_end=2705 - _globals['_FINDBYWORKFLOWEXECREQUEST_DIRECTION']._serialized_start=2669 - _globals['_FINDBYWORKFLOWEXECREQUEST_DIRECTION']._serialized_end=2705 - _globals['_ADDTAGREQUEST']._serialized_start=2707 - _globals['_ADDTAGREQUEST']._serialized_end=2834 - _globals['_ADDTAGRESPONSE']._serialized_start=2836 - _globals['_ADDTAGRESPONSE']._serialized_end=2852 - _globals['_CREATETRIGGERREQUEST']._serialized_start=2854 - _globals['_CREATETRIGGERREQUEST']._serialized_end=2952 - _globals['_CREATETRIGGERRESPONSE']._serialized_start=2954 - _globals['_CREATETRIGGERRESPONSE']._serialized_end=2977 - _globals['_DEACTIVATETRIGGERREQUEST']._serialized_start=2979 - _globals['_DEACTIVATETRIGGERREQUEST']._serialized_end=3063 - _globals['_DEACTIVATETRIGGERRESPONSE']._serialized_start=3065 - _globals['_DEACTIVATETRIGGERRESPONSE']._serialized_end=3092 - _globals['_ARTIFACTPRODUCER']._serialized_start=3095 - _globals['_ARTIFACTPRODUCER']._serialized_end=3223 - _globals['_REGISTERPRODUCERREQUEST']._serialized_start=3225 - _globals['_REGISTERPRODUCERREQUEST']._serialized_end=3317 - _globals['_ARTIFACTCONSUMER']._serialized_start=3319 - _globals['_ARTIFACTCONSUMER']._serialized_end=3446 - _globals['_REGISTERCONSUMERREQUEST']._serialized_start=3448 - _globals['_REGISTERCONSUMERREQUEST']._serialized_end=3540 - _globals['_REGISTERRESPONSE']._serialized_start=3542 - _globals['_REGISTERRESPONSE']._serialized_end=3560 - _globals['_EXECUTIONINPUTSREQUEST']._serialized_start=3563 - _globals['_EXECUTIONINPUTSREQUEST']._serialized_end=3717 - _globals['_EXECUTIONINPUTSRESPONSE']._serialized_start=3719 - _globals['_EXECUTIONINPUTSRESPONSE']._serialized_end=3744 - _globals['_LISTUSAGEREQUEST']._serialized_start=3746 - _globals['_LISTUSAGEREQUEST']._serialized_end=3824 - _globals['_LISTUSAGERESPONSE']._serialized_start=3826 - _globals['_LISTUSAGERESPONSE']._serialized_end=3921 - _globals['_ARTIFACTREGISTRY']._serialized_start=3924 - _globals['_ARTIFACTREGISTRY']._serialized_end=5455 -# @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.pyi deleted file mode 100644 index fe6c5fecfb..0000000000 --- a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2.pyi +++ /dev/null @@ -1,244 +0,0 @@ -from google.protobuf import any_pb2 as _any_pb2 -from google.api import annotations_pb2 as _annotations_pb2 -from google.protobuf import timestamp_pb2 as _timestamp_pb2 -from flyteidl.admin import launch_plan_pb2 as _launch_plan_pb2 -from flyteidl.core import literals_pb2 as _literals_pb2 -from flyteidl.core import types_pb2 as _types_pb2 -from flyteidl.core import identifier_pb2 as _identifier_pb2 -from flyteidl.core import artifact_id_pb2 as _artifact_id_pb2 -from flyteidl.core import interface_pb2 as _interface_pb2 -from flyteidl.event import cloudevents_pb2 as _cloudevents_pb2 -from google.protobuf.internal import containers as _containers -from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper -from google.protobuf import descriptor as _descriptor -from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union - -DESCRIPTOR: _descriptor.FileDescriptor - -class Artifact(_message.Message): - __slots__ = ["artifact_id", "spec", "tags", "source"] - ARTIFACT_ID_FIELD_NUMBER: _ClassVar[int] - SPEC_FIELD_NUMBER: _ClassVar[int] - TAGS_FIELD_NUMBER: _ClassVar[int] - SOURCE_FIELD_NUMBER: _ClassVar[int] - artifact_id: _artifact_id_pb2.ArtifactID - spec: ArtifactSpec - tags: _containers.RepeatedScalarFieldContainer[str] - source: ArtifactSource - def __init__(self, artifact_id: _Optional[_Union[_artifact_id_pb2.ArtifactID, _Mapping]] = ..., spec: _Optional[_Union[ArtifactSpec, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., source: _Optional[_Union[ArtifactSource, _Mapping]] = ...) -> None: ... - -class CreateArtifactRequest(_message.Message): - __slots__ = ["artifact_key", "version", "spec", "partitions", "time_partition_value", "source"] - class PartitionsEntry(_message.Message): - __slots__ = ["key", "value"] - KEY_FIELD_NUMBER: _ClassVar[int] - VALUE_FIELD_NUMBER: _ClassVar[int] - key: str - value: str - def __init__(self, key: _Optional[str] = ..., value: _Optional[str] = ...) -> None: ... - ARTIFACT_KEY_FIELD_NUMBER: _ClassVar[int] - VERSION_FIELD_NUMBER: _ClassVar[int] - SPEC_FIELD_NUMBER: _ClassVar[int] - PARTITIONS_FIELD_NUMBER: _ClassVar[int] - TIME_PARTITION_VALUE_FIELD_NUMBER: _ClassVar[int] - SOURCE_FIELD_NUMBER: _ClassVar[int] - artifact_key: _artifact_id_pb2.ArtifactKey - version: str - spec: ArtifactSpec - partitions: _containers.ScalarMap[str, str] - time_partition_value: _timestamp_pb2.Timestamp - source: ArtifactSource - def __init__(self, artifact_key: _Optional[_Union[_artifact_id_pb2.ArtifactKey, _Mapping]] = ..., version: _Optional[str] = ..., spec: _Optional[_Union[ArtifactSpec, _Mapping]] = ..., partitions: _Optional[_Mapping[str, str]] = ..., time_partition_value: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., source: _Optional[_Union[ArtifactSource, _Mapping]] = ...) -> None: ... - -class ArtifactSource(_message.Message): - __slots__ = ["workflow_execution", "node_id", "task_id", "retry_attempt", "principal"] - WORKFLOW_EXECUTION_FIELD_NUMBER: _ClassVar[int] - NODE_ID_FIELD_NUMBER: _ClassVar[int] - TASK_ID_FIELD_NUMBER: _ClassVar[int] - RETRY_ATTEMPT_FIELD_NUMBER: _ClassVar[int] - PRINCIPAL_FIELD_NUMBER: _ClassVar[int] - workflow_execution: _identifier_pb2.WorkflowExecutionIdentifier - node_id: str - task_id: _identifier_pb2.Identifier - retry_attempt: int - principal: str - def __init__(self, workflow_execution: _Optional[_Union[_identifier_pb2.WorkflowExecutionIdentifier, _Mapping]] = ..., node_id: _Optional[str] = ..., task_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., retry_attempt: _Optional[int] = ..., principal: _Optional[str] = ...) -> None: ... - -class ArtifactSpec(_message.Message): - __slots__ = ["value", "type", "short_description", "user_metadata", "metadata_type", "created_at", "file_format"] - VALUE_FIELD_NUMBER: _ClassVar[int] - TYPE_FIELD_NUMBER: _ClassVar[int] - SHORT_DESCRIPTION_FIELD_NUMBER: _ClassVar[int] - USER_METADATA_FIELD_NUMBER: _ClassVar[int] - METADATA_TYPE_FIELD_NUMBER: _ClassVar[int] - CREATED_AT_FIELD_NUMBER: _ClassVar[int] - FILE_FORMAT_FIELD_NUMBER: _ClassVar[int] - value: _literals_pb2.Literal - type: _types_pb2.LiteralType - short_description: str - user_metadata: _any_pb2.Any - metadata_type: str - created_at: _timestamp_pb2.Timestamp - file_format: str - def __init__(self, value: _Optional[_Union[_literals_pb2.Literal, _Mapping]] = ..., type: _Optional[_Union[_types_pb2.LiteralType, _Mapping]] = ..., short_description: _Optional[str] = ..., user_metadata: _Optional[_Union[_any_pb2.Any, _Mapping]] = ..., metadata_type: _Optional[str] = ..., created_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., file_format: _Optional[str] = ...) -> None: ... - -class CreateArtifactResponse(_message.Message): - __slots__ = ["artifact"] - ARTIFACT_FIELD_NUMBER: _ClassVar[int] - artifact: Artifact - def __init__(self, artifact: _Optional[_Union[Artifact, _Mapping]] = ...) -> None: ... - -class GetArtifactRequest(_message.Message): - __slots__ = ["query", "details"] - QUERY_FIELD_NUMBER: _ClassVar[int] - DETAILS_FIELD_NUMBER: _ClassVar[int] - query: _artifact_id_pb2.ArtifactQuery - details: bool - def __init__(self, query: _Optional[_Union[_artifact_id_pb2.ArtifactQuery, _Mapping]] = ..., details: bool = ...) -> None: ... - -class GetArtifactResponse(_message.Message): - __slots__ = ["artifact"] - ARTIFACT_FIELD_NUMBER: _ClassVar[int] - artifact: Artifact - def __init__(self, artifact: _Optional[_Union[Artifact, _Mapping]] = ...) -> None: ... - -class SearchOptions(_message.Message): - __slots__ = ["strict_partitions", "latest_by_key"] - STRICT_PARTITIONS_FIELD_NUMBER: _ClassVar[int] - LATEST_BY_KEY_FIELD_NUMBER: _ClassVar[int] - strict_partitions: bool - latest_by_key: bool - def __init__(self, strict_partitions: bool = ..., latest_by_key: bool = ...) -> None: ... - -class SearchArtifactsRequest(_message.Message): - __slots__ = ["artifact_key", "partitions", "time_partition_value", "principal", "version", "options", "token", "limit"] - ARTIFACT_KEY_FIELD_NUMBER: _ClassVar[int] - PARTITIONS_FIELD_NUMBER: _ClassVar[int] - TIME_PARTITION_VALUE_FIELD_NUMBER: _ClassVar[int] - PRINCIPAL_FIELD_NUMBER: _ClassVar[int] - VERSION_FIELD_NUMBER: _ClassVar[int] - OPTIONS_FIELD_NUMBER: _ClassVar[int] - TOKEN_FIELD_NUMBER: _ClassVar[int] - LIMIT_FIELD_NUMBER: _ClassVar[int] - artifact_key: _artifact_id_pb2.ArtifactKey - partitions: _artifact_id_pb2.Partitions - time_partition_value: _timestamp_pb2.Timestamp - principal: str - version: str - options: SearchOptions - token: str - limit: int - def __init__(self, artifact_key: _Optional[_Union[_artifact_id_pb2.ArtifactKey, _Mapping]] = ..., partitions: _Optional[_Union[_artifact_id_pb2.Partitions, _Mapping]] = ..., time_partition_value: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., principal: _Optional[str] = ..., version: _Optional[str] = ..., options: _Optional[_Union[SearchOptions, _Mapping]] = ..., token: _Optional[str] = ..., limit: _Optional[int] = ...) -> None: ... - -class SearchArtifactsResponse(_message.Message): - __slots__ = ["artifacts", "token"] - ARTIFACTS_FIELD_NUMBER: _ClassVar[int] - TOKEN_FIELD_NUMBER: _ClassVar[int] - artifacts: _containers.RepeatedCompositeFieldContainer[Artifact] - token: str - def __init__(self, artifacts: _Optional[_Iterable[_Union[Artifact, _Mapping]]] = ..., token: _Optional[str] = ...) -> None: ... - -class FindByWorkflowExecRequest(_message.Message): - __slots__ = ["exec_id", "direction"] - class Direction(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): - __slots__ = [] - INPUTS: _ClassVar[FindByWorkflowExecRequest.Direction] - OUTPUTS: _ClassVar[FindByWorkflowExecRequest.Direction] - INPUTS: FindByWorkflowExecRequest.Direction - OUTPUTS: FindByWorkflowExecRequest.Direction - EXEC_ID_FIELD_NUMBER: _ClassVar[int] - DIRECTION_FIELD_NUMBER: _ClassVar[int] - exec_id: _identifier_pb2.WorkflowExecutionIdentifier - direction: FindByWorkflowExecRequest.Direction - def __init__(self, exec_id: _Optional[_Union[_identifier_pb2.WorkflowExecutionIdentifier, _Mapping]] = ..., direction: _Optional[_Union[FindByWorkflowExecRequest.Direction, str]] = ...) -> None: ... - -class AddTagRequest(_message.Message): - __slots__ = ["artifact_id", "value", "overwrite"] - ARTIFACT_ID_FIELD_NUMBER: _ClassVar[int] - VALUE_FIELD_NUMBER: _ClassVar[int] - OVERWRITE_FIELD_NUMBER: _ClassVar[int] - artifact_id: _artifact_id_pb2.ArtifactID - value: str - overwrite: bool - def __init__(self, artifact_id: _Optional[_Union[_artifact_id_pb2.ArtifactID, _Mapping]] = ..., value: _Optional[str] = ..., overwrite: bool = ...) -> None: ... - -class AddTagResponse(_message.Message): - __slots__ = [] - def __init__(self) -> None: ... - -class CreateTriggerRequest(_message.Message): - __slots__ = ["trigger_launch_plan"] - TRIGGER_LAUNCH_PLAN_FIELD_NUMBER: _ClassVar[int] - trigger_launch_plan: _launch_plan_pb2.LaunchPlan - def __init__(self, trigger_launch_plan: _Optional[_Union[_launch_plan_pb2.LaunchPlan, _Mapping]] = ...) -> None: ... - -class CreateTriggerResponse(_message.Message): - __slots__ = [] - def __init__(self) -> None: ... - -class DeactivateTriggerRequest(_message.Message): - __slots__ = ["trigger_id"] - TRIGGER_ID_FIELD_NUMBER: _ClassVar[int] - trigger_id: _identifier_pb2.Identifier - def __init__(self, trigger_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ...) -> None: ... - -class DeactivateTriggerResponse(_message.Message): - __slots__ = [] - def __init__(self) -> None: ... - -class ArtifactProducer(_message.Message): - __slots__ = ["entity_id", "outputs"] - ENTITY_ID_FIELD_NUMBER: _ClassVar[int] - OUTPUTS_FIELD_NUMBER: _ClassVar[int] - entity_id: _identifier_pb2.Identifier - outputs: _interface_pb2.VariableMap - def __init__(self, entity_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., outputs: _Optional[_Union[_interface_pb2.VariableMap, _Mapping]] = ...) -> None: ... - -class RegisterProducerRequest(_message.Message): - __slots__ = ["producers"] - PRODUCERS_FIELD_NUMBER: _ClassVar[int] - producers: _containers.RepeatedCompositeFieldContainer[ArtifactProducer] - def __init__(self, producers: _Optional[_Iterable[_Union[ArtifactProducer, _Mapping]]] = ...) -> None: ... - -class ArtifactConsumer(_message.Message): - __slots__ = ["entity_id", "inputs"] - ENTITY_ID_FIELD_NUMBER: _ClassVar[int] - INPUTS_FIELD_NUMBER: _ClassVar[int] - entity_id: _identifier_pb2.Identifier - inputs: _interface_pb2.ParameterMap - def __init__(self, entity_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., inputs: _Optional[_Union[_interface_pb2.ParameterMap, _Mapping]] = ...) -> None: ... - -class RegisterConsumerRequest(_message.Message): - __slots__ = ["consumers"] - CONSUMERS_FIELD_NUMBER: _ClassVar[int] - consumers: _containers.RepeatedCompositeFieldContainer[ArtifactConsumer] - def __init__(self, consumers: _Optional[_Iterable[_Union[ArtifactConsumer, _Mapping]]] = ...) -> None: ... - -class RegisterResponse(_message.Message): - __slots__ = [] - def __init__(self) -> None: ... - -class ExecutionInputsRequest(_message.Message): - __slots__ = ["execution_id", "inputs"] - EXECUTION_ID_FIELD_NUMBER: _ClassVar[int] - INPUTS_FIELD_NUMBER: _ClassVar[int] - execution_id: _identifier_pb2.WorkflowExecutionIdentifier - inputs: _containers.RepeatedCompositeFieldContainer[_artifact_id_pb2.ArtifactID] - def __init__(self, execution_id: _Optional[_Union[_identifier_pb2.WorkflowExecutionIdentifier, _Mapping]] = ..., inputs: _Optional[_Iterable[_Union[_artifact_id_pb2.ArtifactID, _Mapping]]] = ...) -> None: ... - -class ExecutionInputsResponse(_message.Message): - __slots__ = [] - def __init__(self) -> None: ... - -class ListUsageRequest(_message.Message): - __slots__ = ["artifact_id"] - ARTIFACT_ID_FIELD_NUMBER: _ClassVar[int] - artifact_id: _artifact_id_pb2.ArtifactID - def __init__(self, artifact_id: _Optional[_Union[_artifact_id_pb2.ArtifactID, _Mapping]] = ...) -> None: ... - -class ListUsageResponse(_message.Message): - __slots__ = ["executions"] - EXECUTIONS_FIELD_NUMBER: _ClassVar[int] - executions: _containers.RepeatedCompositeFieldContainer[_identifier_pb2.WorkflowExecutionIdentifier] - def __init__(self, executions: _Optional[_Iterable[_Union[_identifier_pb2.WorkflowExecutionIdentifier, _Mapping]]] = ...) -> None: ... diff --git a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2_grpc.py b/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2_grpc.py deleted file mode 100644 index 661103389b..0000000000 --- a/flyteidl/gen/pb_python/flyteidl/artifact/artifacts_pb2_grpc.py +++ /dev/null @@ -1,396 +0,0 @@ -# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! -"""Client and server classes corresponding to protobuf-defined services.""" -import grpc - -from flyteidl.artifact import artifacts_pb2 as flyteidl_dot_artifact_dot_artifacts__pb2 - - -class ArtifactRegistryStub(object): - """Missing associated documentation comment in .proto file.""" - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.CreateArtifact = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/CreateArtifact', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactResponse.FromString, - ) - self.GetArtifact = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/GetArtifact', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactResponse.FromString, - ) - self.SearchArtifacts = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/SearchArtifacts', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.FromString, - ) - self.CreateTrigger = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/CreateTrigger', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerResponse.FromString, - ) - self.DeactivateTrigger = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/DeactivateTrigger', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerResponse.FromString, - ) - self.AddTag = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/AddTag', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.AddTagRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.AddTagResponse.FromString, - ) - self.RegisterProducer = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/RegisterProducer', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterProducerRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.FromString, - ) - self.RegisterConsumer = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/RegisterConsumer', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterConsumerRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.FromString, - ) - self.SetExecutionInputs = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/SetExecutionInputs', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsResponse.FromString, - ) - self.FindByWorkflowExec = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/FindByWorkflowExec', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.FindByWorkflowExecRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.FromString, - ) - self.ListUsage = channel.unary_unary( - '/flyteidl.artifact.ArtifactRegistry/ListUsage', - request_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageRequest.SerializeToString, - response_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageResponse.FromString, - ) - - -class ArtifactRegistryServicer(object): - """Missing associated documentation comment in .proto file.""" - - def CreateArtifact(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def GetArtifact(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def SearchArtifacts(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def CreateTrigger(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def DeactivateTrigger(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def AddTag(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def RegisterProducer(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def RegisterConsumer(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def SetExecutionInputs(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def FindByWorkflowExec(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - def ListUsage(self, request, context): - """Missing associated documentation comment in .proto file.""" - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - -def add_ArtifactRegistryServicer_to_server(servicer, server): - rpc_method_handlers = { - 'CreateArtifact': grpc.unary_unary_rpc_method_handler( - servicer.CreateArtifact, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactResponse.SerializeToString, - ), - 'GetArtifact': grpc.unary_unary_rpc_method_handler( - servicer.GetArtifact, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactResponse.SerializeToString, - ), - 'SearchArtifacts': grpc.unary_unary_rpc_method_handler( - servicer.SearchArtifacts, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.SerializeToString, - ), - 'CreateTrigger': grpc.unary_unary_rpc_method_handler( - servicer.CreateTrigger, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerResponse.SerializeToString, - ), - 'DeactivateTrigger': grpc.unary_unary_rpc_method_handler( - servicer.DeactivateTrigger, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerResponse.SerializeToString, - ), - 'AddTag': grpc.unary_unary_rpc_method_handler( - servicer.AddTag, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.AddTagRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.AddTagResponse.SerializeToString, - ), - 'RegisterProducer': grpc.unary_unary_rpc_method_handler( - servicer.RegisterProducer, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterProducerRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.SerializeToString, - ), - 'RegisterConsumer': grpc.unary_unary_rpc_method_handler( - servicer.RegisterConsumer, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterConsumerRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.SerializeToString, - ), - 'SetExecutionInputs': grpc.unary_unary_rpc_method_handler( - servicer.SetExecutionInputs, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsResponse.SerializeToString, - ), - 'FindByWorkflowExec': grpc.unary_unary_rpc_method_handler( - servicer.FindByWorkflowExec, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.FindByWorkflowExecRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.SerializeToString, - ), - 'ListUsage': grpc.unary_unary_rpc_method_handler( - servicer.ListUsage, - request_deserializer=flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageRequest.FromString, - response_serializer=flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageResponse.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'flyteidl.artifact.ArtifactRegistry', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - - # This class is part of an EXPERIMENTAL API. -class ArtifactRegistry(object): - """Missing associated documentation comment in .proto file.""" - - @staticmethod - def CreateArtifact(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.artifact.ArtifactRegistry/CreateArtifact', - flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.CreateArtifactResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def GetArtifact(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.artifact.ArtifactRegistry/GetArtifact', - flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.GetArtifactResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def SearchArtifacts(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.artifact.ArtifactRegistry/SearchArtifacts', - flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def CreateTrigger(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.artifact.ArtifactRegistry/CreateTrigger', - flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.CreateTriggerResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def DeactivateTrigger(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.artifact.ArtifactRegistry/DeactivateTrigger', - flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.DeactivateTriggerResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def AddTag(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.artifact.ArtifactRegistry/AddTag', - flyteidl_dot_artifact_dot_artifacts__pb2.AddTagRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.AddTagResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def RegisterProducer(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.artifact.ArtifactRegistry/RegisterProducer', - flyteidl_dot_artifact_dot_artifacts__pb2.RegisterProducerRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def RegisterConsumer(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.artifact.ArtifactRegistry/RegisterConsumer', - flyteidl_dot_artifact_dot_artifacts__pb2.RegisterConsumerRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.RegisterResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def SetExecutionInputs(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.artifact.ArtifactRegistry/SetExecutionInputs', - flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.ExecutionInputsResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def FindByWorkflowExec(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.artifact.ArtifactRegistry/FindByWorkflowExec', - flyteidl_dot_artifact_dot_artifacts__pb2.FindByWorkflowExecRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.SearchArtifactsResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) - - @staticmethod - def ListUsage(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.artifact.ArtifactRegistry/ListUsage', - flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageRequest.SerializeToString, - flyteidl_dot_artifact_dot_artifacts__pb2.ListUsageResponse.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.py b/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.py index a84b5629ed..ff603987f3 100644 --- a/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.py @@ -15,7 +15,7 @@ from flyteidl.core import identifier_pb2 as flyteidl_dot_core_dot_identifier__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1f\x66lyteidl/core/artifact_id.proto\x12\rflyteidl.core\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1e\x66lyteidl/core/identifier.proto\"S\n\x0b\x41rtifactKey\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\"\xb9\x01\n\x13\x41rtifactBindingData\x12\x14\n\x05index\x18\x01 \x01(\rR\x05index\x12%\n\rpartition_key\x18\x02 \x01(\tH\x00R\x0cpartitionKey\x12\x35\n\x16\x62ind_to_time_partition\x18\x03 \x01(\x08H\x00R\x13\x62indToTimePartition\x12\x1c\n\ttransform\x18\x04 \x01(\tR\ttransformB\x10\n\x0epartition_data\"$\n\x10InputBindingData\x12\x10\n\x03var\x18\x01 \x01(\tR\x03var\"\x92\x02\n\nLabelValue\x12#\n\x0cstatic_value\x18\x01 \x01(\tH\x00R\x0bstaticValue\x12;\n\ntime_value\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampH\x00R\ttimeValue\x12Q\n\x11triggered_binding\x18\x03 \x01(\x0b\x32\".flyteidl.core.ArtifactBindingDataH\x00R\x10triggeredBinding\x12\x46\n\rinput_binding\x18\x04 \x01(\x0b\x32\x1f.flyteidl.core.InputBindingDataH\x00R\x0cinputBindingB\x07\n\x05value\"\x9d\x01\n\nPartitions\x12:\n\x05value\x18\x01 \x03(\x0b\x32$.flyteidl.core.Partitions.ValueEntryR\x05value\x1aS\n\nValueEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value:\x02\x38\x01\"@\n\rTimePartition\x12/\n\x05value\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value\"\xe5\x01\n\nArtifactID\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12\x18\n\x07version\x18\x02 \x01(\tR\x07version\x12\x39\n\npartitions\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.PartitionsR\npartitions\x12\x43\n\x0etime_partition\x18\x04 \x01(\x0b\x32\x1c.flyteidl.core.TimePartitionR\rtimePartition\"}\n\x0b\x41rtifactTag\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value\"\xf0\x01\n\rArtifactQuery\x12<\n\x0b\x61rtifact_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.ArtifactIDH\x00R\nartifactId\x12?\n\x0c\x61rtifact_tag\x18\x02 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactTagH\x00R\x0b\x61rtifactTag\x12\x12\n\x03uri\x18\x03 \x01(\tH\x00R\x03uri\x12>\n\x07\x62inding\x18\x04 \x01(\x0b\x32\".flyteidl.core.ArtifactBindingDataH\x00R\x07\x62indingB\x0c\n\nidentifier\"z\n\x07Trigger\x12\x38\n\ntrigger_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\ttriggerId\x12\x35\n\x08triggers\x18\x02 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x08triggersB\xb5\x01\n\x11\x63om.flyteidl.coreB\x0f\x41rtifactIdProtoP\x01Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\xa2\x02\x03\x46\x43X\xaa\x02\rFlyteidl.Core\xca\x02\rFlyteidl\\Core\xe2\x02\x19\x46lyteidl\\Core\\GPBMetadata\xea\x02\x0e\x46lyteidl::Coreb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1f\x66lyteidl/core/artifact_id.proto\x12\rflyteidl.core\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1e\x66lyteidl/core/identifier.proto\"S\n\x0b\x41rtifactKey\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\"\xb9\x01\n\x13\x41rtifactBindingData\x12\x14\n\x05index\x18\x01 \x01(\rR\x05index\x12%\n\rpartition_key\x18\x02 \x01(\tH\x00R\x0cpartitionKey\x12\x35\n\x16\x62ind_to_time_partition\x18\x03 \x01(\x08H\x00R\x13\x62indToTimePartition\x12\x1c\n\ttransform\x18\x04 \x01(\tR\ttransformB\x10\n\x0epartition_data\"$\n\x10InputBindingData\x12\x10\n\x03var\x18\x01 \x01(\tR\x03var\"\x92\x02\n\nLabelValue\x12#\n\x0cstatic_value\x18\x01 \x01(\tH\x00R\x0bstaticValue\x12;\n\ntime_value\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampH\x00R\ttimeValue\x12Q\n\x11triggered_binding\x18\x03 \x01(\x0b\x32\".flyteidl.core.ArtifactBindingDataH\x00R\x10triggeredBinding\x12\x46\n\rinput_binding\x18\x04 \x01(\x0b\x32\x1f.flyteidl.core.InputBindingDataH\x00R\x0cinputBindingB\x07\n\x05value\"\x9d\x01\n\nPartitions\x12:\n\x05value\x18\x01 \x03(\x0b\x32$.flyteidl.core.Partitions.ValueEntryR\x05value\x1aS\n\nValueEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value:\x02\x38\x01\"@\n\rTimePartition\x12/\n\x05value\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value\"\xe5\x01\n\nArtifactID\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12\x18\n\x07version\x18\x02 \x01(\tR\x07version\x12\x39\n\npartitions\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.PartitionsR\npartitions\x12\x43\n\x0etime_partition\x18\x04 \x01(\x0b\x32\x1c.flyteidl.core.TimePartitionR\rtimePartition\"}\n\x0b\x41rtifactTag\x12=\n\x0c\x61rtifact_key\x18\x01 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactKeyR\x0b\x61rtifactKey\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LabelValueR\x05value\"\xf0\x01\n\rArtifactQuery\x12<\n\x0b\x61rtifact_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.ArtifactIDH\x00R\nartifactId\x12?\n\x0c\x61rtifact_tag\x18\x02 \x01(\x0b\x32\x1a.flyteidl.core.ArtifactTagH\x00R\x0b\x61rtifactTag\x12\x12\n\x03uri\x18\x03 \x01(\tH\x00R\x03uri\x12>\n\x07\x62inding\x18\x04 \x01(\x0b\x32\".flyteidl.core.ArtifactBindingDataH\x00R\x07\x62indingB\x0c\n\nidentifierB\xb5\x01\n\x11\x63om.flyteidl.coreB\x0f\x41rtifactIdProtoP\x01Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\xa2\x02\x03\x46\x43X\xaa\x02\rFlyteidl.Core\xca\x02\rFlyteidl\\Core\xe2\x02\x19\x46lyteidl\\Core\\GPBMetadata\xea\x02\x0e\x46lyteidl::Coreb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -46,6 +46,4 @@ _globals['_ARTIFACTTAG']._serialized_end=1286 _globals['_ARTIFACTQUERY']._serialized_start=1289 _globals['_ARTIFACTQUERY']._serialized_end=1529 - _globals['_TRIGGER']._serialized_start=1531 - _globals['_TRIGGER']._serialized_end=1653 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.pyi index 1e0aedd6d4..53a9420bc1 100644 --- a/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/core/artifact_id_pb2.pyi @@ -3,7 +3,7 @@ from flyteidl.core import identifier_pb2 as _identifier_pb2 from google.protobuf.internal import containers as _containers from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message -from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union +from typing import ClassVar as _ClassVar, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor @@ -97,11 +97,3 @@ class ArtifactQuery(_message.Message): uri: str binding: ArtifactBindingData def __init__(self, artifact_id: _Optional[_Union[ArtifactID, _Mapping]] = ..., artifact_tag: _Optional[_Union[ArtifactTag, _Mapping]] = ..., uri: _Optional[str] = ..., binding: _Optional[_Union[ArtifactBindingData, _Mapping]] = ...) -> None: ... - -class Trigger(_message.Message): - __slots__ = ["trigger_id", "triggers"] - TRIGGER_ID_FIELD_NUMBER: _ClassVar[int] - TRIGGERS_FIELD_NUMBER: _ClassVar[int] - trigger_id: _identifier_pb2.Identifier - triggers: _containers.RepeatedCompositeFieldContainer[ArtifactID] - def __init__(self, trigger_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., triggers: _Optional[_Iterable[_Union[ArtifactID, _Mapping]]] = ...) -> None: ... diff --git a/flyteidl/gen/pb_rust/flyteidl.artifact.rs b/flyteidl/gen/pb_rust/flyteidl.artifact.rs deleted file mode 100644 index d15c98c74e..0000000000 --- a/flyteidl/gen/pb_rust/flyteidl.artifact.rs +++ /dev/null @@ -1,266 +0,0 @@ -// @generated -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct Artifact { - #[prost(message, optional, tag="1")] - pub artifact_id: ::core::option::Option, - #[prost(message, optional, tag="2")] - pub spec: ::core::option::Option, - /// references the tag field in ArtifactTag - #[prost(string, repeated, tag="3")] - pub tags: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, - #[prost(message, optional, tag="4")] - pub source: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreateArtifactRequest { - /// Specify just project/domain on creation - #[prost(message, optional, tag="1")] - pub artifact_key: ::core::option::Option, - #[prost(string, tag="3")] - pub version: ::prost::alloc::string::String, - #[prost(message, optional, tag="2")] - pub spec: ::core::option::Option, - #[prost(map="string, string", tag="4")] - pub partitions: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, - #[prost(message, optional, tag="5")] - pub time_partition_value: ::core::option::Option<::prost_types::Timestamp>, - #[prost(message, optional, tag="6")] - pub source: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ArtifactSource { - #[prost(message, optional, tag="1")] - pub workflow_execution: ::core::option::Option, - #[prost(string, tag="2")] - pub node_id: ::prost::alloc::string::String, - #[prost(message, optional, tag="3")] - pub task_id: ::core::option::Option, - #[prost(uint32, tag="4")] - pub retry_attempt: u32, - /// Uploads, either from the UI or from the CLI, or FlyteRemote, will have this. - #[prost(string, tag="5")] - pub principal: ::prost::alloc::string::String, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ArtifactSpec { - #[prost(message, optional, tag="1")] - pub value: ::core::option::Option, - /// This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but - /// forgets to change the name, that is okay. And the reason why this is a separate field is because adding the - /// type to all Literals is a lot of work. - #[prost(message, optional, tag="2")] - pub r#type: ::core::option::Option, - #[prost(string, tag="3")] - pub short_description: ::prost::alloc::string::String, - /// Additional user metadata - #[prost(message, optional, tag="4")] - pub user_metadata: ::core::option::Option<::prost_types::Any>, - #[prost(string, tag="5")] - pub metadata_type: ::prost::alloc::string::String, - #[prost(message, optional, tag="6")] - pub created_at: ::core::option::Option<::prost_types::Timestamp>, - #[prost(string, tag="7")] - pub file_format: ::prost::alloc::string::String, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreateArtifactResponse { - #[prost(message, optional, tag="1")] - pub artifact: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct GetArtifactRequest { - #[prost(message, optional, tag="1")] - pub query: ::core::option::Option, - /// If false, then long_description is not returned. - #[prost(bool, tag="2")] - pub details: bool, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct GetArtifactResponse { - #[prost(message, optional, tag="1")] - pub artifact: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct SearchOptions { - /// If true, this means a strict partition search. meaning if you don't specify the partition - /// field, that will mean, non-partitioned, rather than any partition. - #[prost(bool, tag="1")] - pub strict_partitions: bool, - /// If true, only one artifact per key will be returned. It will be the latest one by creation time. - #[prost(bool, tag="2")] - pub latest_by_key: bool, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct SearchArtifactsRequest { - #[prost(message, optional, tag="1")] - pub artifact_key: ::core::option::Option, - #[prost(message, optional, tag="2")] - pub partitions: ::core::option::Option, - #[prost(message, optional, tag="3")] - pub time_partition_value: ::core::option::Option<::prost_types::Timestamp>, - #[prost(string, tag="4")] - pub principal: ::prost::alloc::string::String, - #[prost(string, tag="5")] - pub version: ::prost::alloc::string::String, - #[prost(message, optional, tag="6")] - pub options: ::core::option::Option, - #[prost(string, tag="7")] - pub token: ::prost::alloc::string::String, - #[prost(int32, tag="8")] - pub limit: i32, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct SearchArtifactsResponse { - /// If artifact specs are not requested, the resultant artifacts may be empty. - #[prost(message, repeated, tag="1")] - pub artifacts: ::prost::alloc::vec::Vec, - /// continuation token if relevant. - #[prost(string, tag="2")] - pub token: ::prost::alloc::string::String, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct FindByWorkflowExecRequest { - #[prost(message, optional, tag="1")] - pub exec_id: ::core::option::Option, - #[prost(enumeration="find_by_workflow_exec_request::Direction", tag="2")] - pub direction: i32, -} -/// Nested message and enum types in `FindByWorkflowExecRequest`. -pub mod find_by_workflow_exec_request { - #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] - #[repr(i32)] - pub enum Direction { - Inputs = 0, - Outputs = 1, - } - impl Direction { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - Direction::Inputs => "INPUTS", - Direction::Outputs => "OUTPUTS", - } - } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "INPUTS" => Some(Self::Inputs), - "OUTPUTS" => Some(Self::Outputs), - _ => None, - } - } - } -} -/// Aliases identify a particular version of an artifact. They are different than tags in that they -/// have to be unique for a given artifact project/domain/name. That is, for a given project/domain/name/kind, -/// at most one version can have any given value at any point. -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct AddTagRequest { - #[prost(message, optional, tag="1")] - pub artifact_id: ::core::option::Option, - #[prost(string, tag="2")] - pub value: ::prost::alloc::string::String, - /// If true, and another version already has the specified kind/value, set this version instead - #[prost(bool, tag="3")] - pub overwrite: bool, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct AddTagResponse { -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreateTriggerRequest { - #[prost(message, optional, tag="1")] - pub trigger_launch_plan: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreateTriggerResponse { -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct DeactivateTriggerRequest { - #[prost(message, optional, tag="1")] - pub trigger_id: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct DeactivateTriggerResponse { -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ArtifactProducer { - /// These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in - /// Admin's domain. - #[prost(message, optional, tag="1")] - pub entity_id: ::core::option::Option, - #[prost(message, optional, tag="2")] - pub outputs: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct RegisterProducerRequest { - #[prost(message, repeated, tag="1")] - pub producers: ::prost::alloc::vec::Vec, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ArtifactConsumer { - /// These should all be launch plan IDs - #[prost(message, optional, tag="1")] - pub entity_id: ::core::option::Option, - #[prost(message, optional, tag="2")] - pub inputs: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct RegisterConsumerRequest { - #[prost(message, repeated, tag="1")] - pub consumers: ::prost::alloc::vec::Vec, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct RegisterResponse { -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ExecutionInputsRequest { - #[prost(message, optional, tag="1")] - pub execution_id: ::core::option::Option, - /// can make this a map in the future, currently no need. - #[prost(message, repeated, tag="2")] - pub inputs: ::prost::alloc::vec::Vec, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ExecutionInputsResponse { -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListUsageRequest { - #[prost(message, optional, tag="1")] - pub artifact_id: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListUsageResponse { - #[prost(message, repeated, tag="1")] - pub executions: ::prost::alloc::vec::Vec, -} -// @@protoc_insertion_point(module) diff --git a/flyteidl/gen/pb_rust/flyteidl.core.rs b/flyteidl/gen/pb_rust/flyteidl.core.rs index fab797191d..64fb26d7d7 100644 --- a/flyteidl/gen/pb_rust/flyteidl.core.rs +++ b/flyteidl/gen/pb_rust/flyteidl.core.rs @@ -841,17 +841,6 @@ pub mod artifact_query { Binding(super::ArtifactBindingData), } } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct Trigger { - /// This will be set to a launch plan type, but note that this is different than the actual launch plan type. - #[prost(message, optional, tag="1")] - pub trigger_id: ::core::option::Option, - /// These are partial artifact IDs that will be triggered on - /// Consider making these ArtifactQuery instead. - #[prost(message, repeated, tag="2")] - pub triggers: ::prost::alloc::vec::Vec, -} /// Defines a strongly typed variable. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/flyteidl/generate_protos.sh b/flyteidl/generate_protos.sh index 7955536737..38d663e276 100755 --- a/flyteidl/generate_protos.sh +++ b/flyteidl/generate_protos.sh @@ -12,14 +12,13 @@ export LC_ALL=C.UTF-8 docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/service --with_gateway -l go --go_source_relative docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/admin --with_gateway -l go --go_source_relative -docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/artifact --with_gateway -l go --go_source_relative docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/core --with_gateway -l go --go_source_relative docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/event --with_gateway -l go --go_source_relative docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/plugins -l go --go_source_relative docker run --rm -u $(id -u):$(id -g) -v $DIR:/defs $LYFT_IMAGE -i ./protos -d protos/flyteidl/datacatalog -l go --go_source_relative languages=("cpp" "java") -idlfolders=("service" "admin" "core" "event" "plugins" "datacatalog" "artifact") +idlfolders=("service" "admin" "core" "event" "plugins" "datacatalog") for lang in "${languages[@]}" do diff --git a/flyteidl/protos/flyteidl/artifact/artifacts.proto b/flyteidl/protos/flyteidl/artifact/artifacts.proto deleted file mode 100644 index b67da65c86..0000000000 --- a/flyteidl/protos/flyteidl/artifact/artifacts.proto +++ /dev/null @@ -1,247 +0,0 @@ -syntax = "proto3"; -package flyteidl.artifact; - -option go_package = "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/artifact"; - -import "google/protobuf/any.proto"; -import "google/api/annotations.proto"; -import "google/protobuf/timestamp.proto"; - -import "flyteidl/admin/launch_plan.proto"; -import "flyteidl/core/literals.proto"; -import "flyteidl/core/types.proto"; -import "flyteidl/core/identifier.proto"; -import "flyteidl/core/artifact_id.proto"; -import "flyteidl/core/interface.proto"; -import "flyteidl/event/cloudevents.proto"; - -message Artifact { - core.ArtifactID artifact_id = 1; - - ArtifactSpec spec = 2; - - // references the tag field in ArtifactTag - repeated string tags = 3; - - ArtifactSource source = 4; -} - -message CreateArtifactRequest { - // Specify just project/domain on creation - core.ArtifactKey artifact_key = 1; - - string version = 3; - - ArtifactSpec spec = 2; - - map partitions = 4; - - google.protobuf.Timestamp time_partition_value = 5; - - ArtifactSource source = 6; -} - -message ArtifactSource { - core.WorkflowExecutionIdentifier workflow_execution = 1; - string node_id = 2; - core.Identifier task_id = 3; - uint32 retry_attempt = 4; - - // Uploads, either from the UI or from the CLI, or FlyteRemote, will have this. - string principal = 5; -} - -message ArtifactSpec { - core.Literal value = 1; - - // This type will not form part of the artifact key, so for user-named artifacts, if the user changes the type, but - // forgets to change the name, that is okay. And the reason why this is a separate field is because adding the - // type to all Literals is a lot of work. - core.LiteralType type = 2; - - string short_description = 3; - - // Additional user metadata - google.protobuf.Any user_metadata = 4; - - string metadata_type = 5; - - google.protobuf.Timestamp created_at = 6; - - string file_format = 7; -} - - -message CreateArtifactResponse { - Artifact artifact = 1; -} - -message GetArtifactRequest { - core.ArtifactQuery query = 1; - - // If false, then long_description is not returned. - bool details = 2; -} - -message GetArtifactResponse { - Artifact artifact = 1; -} - -message SearchOptions { - // If true, this means a strict partition search. meaning if you don't specify the partition - // field, that will mean, non-partitioned, rather than any partition. - bool strict_partitions = 1; - - // If true, only one artifact per key will be returned. It will be the latest one by creation time. - bool latest_by_key = 2; -} - -message SearchArtifactsRequest { - core.ArtifactKey artifact_key = 1; - - core.Partitions partitions = 2; - - google.protobuf.Timestamp time_partition_value = 3; - - string principal = 4; - string version = 5; - - SearchOptions options = 6; - - string token = 7; - int32 limit = 8; -} - -message SearchArtifactsResponse { - // If artifact specs are not requested, the resultant artifacts may be empty. - repeated Artifact artifacts = 1; - - // continuation token if relevant. - string token = 2; -} - -message FindByWorkflowExecRequest { - core.WorkflowExecutionIdentifier exec_id = 1; - - enum Direction { - INPUTS = 0; - OUTPUTS = 1; - } - - Direction direction = 2; -} - -// Aliases identify a particular version of an artifact. They are different than tags in that they -// have to be unique for a given artifact project/domain/name. That is, for a given project/domain/name/kind, -// at most one version can have any given value at any point. -message AddTagRequest { - core.ArtifactID artifact_id = 1; - - string value = 2; - - // If true, and another version already has the specified kind/value, set this version instead - bool overwrite = 3; -} - -message AddTagResponse {} - -message CreateTriggerRequest { - admin.LaunchPlan trigger_launch_plan = 1; -} - -message CreateTriggerResponse {} - -message DeactivateTriggerRequest { - core.Identifier trigger_id = 1; -} - -message DeactivateTriggerResponse {} - -message ArtifactProducer { - // These can be tasks, and workflows. Keeping track of the launch plans that a given workflow has is purely in - // Admin's domain. - core.Identifier entity_id = 1; - - core.VariableMap outputs = 2; -} - -message RegisterProducerRequest { - repeated ArtifactProducer producers = 1; -} - -message ArtifactConsumer { - // These should all be launch plan IDs - core.Identifier entity_id = 1; - - core.ParameterMap inputs = 2; -} - -message RegisterConsumerRequest { - repeated ArtifactConsumer consumers = 1; -} - -message RegisterResponse {} - -message ExecutionInputsRequest { - core.WorkflowExecutionIdentifier execution_id = 1; - - // can make this a map in the future, currently no need. - repeated core.ArtifactID inputs = 2; -} - -message ExecutionInputsResponse {} - -message ListUsageRequest { - core.ArtifactID artifact_id = 1; -} - -message ListUsageResponse { - repeated core.WorkflowExecutionIdentifier executions = 1; -} - -service ArtifactRegistry { - rpc CreateArtifact (CreateArtifactRequest) returns (CreateArtifactResponse) {} - - rpc GetArtifact (GetArtifactRequest) returns (GetArtifactResponse) { - option (google.api.http) = { - post: "/artifacts/api/v1/artifacts" - body: "*" - }; - } - - rpc SearchArtifacts (SearchArtifactsRequest) returns (SearchArtifactsResponse) { - option (google.api.http) = { - post: "/artifacts/api/v1/search" - body: "*" - }; - } - - rpc CreateTrigger (CreateTriggerRequest) returns (CreateTriggerResponse) {} - - rpc DeactivateTrigger (DeactivateTriggerRequest) returns (DeactivateTriggerResponse) { - option (google.api.http) = { - patch: "/artifacts/api/v1/trigger/deactivate" - body: "*" - }; - } - - rpc AddTag(AddTagRequest) returns (AddTagResponse) {} - - rpc RegisterProducer(RegisterProducerRequest) returns (RegisterResponse) {} - - rpc RegisterConsumer(RegisterConsumerRequest) returns (RegisterResponse) {} - - rpc SetExecutionInputs(ExecutionInputsRequest) returns (ExecutionInputsResponse) {} - - rpc FindByWorkflowExec (FindByWorkflowExecRequest) returns (SearchArtifactsResponse) { - option (google.api.http) = { - get: "/artifacts/api/v1/search/execution/{exec_id.project}/{exec_id.domain}/{exec_id.name}/{direction}" - }; - } - - rpc ListUsage (ListUsageRequest) returns (ListUsageResponse) { - option (google.api.http) = { - get: "/artifacts/api/v1/usage/{artifact_id.artifact_key.project}/{artifact_id.artifact_key.domain}/{artifact_id.artifact_key.name}/{artifact_id.version}" - }; - } -} diff --git a/flyteidl/protos/flyteidl/core/artifact_id.proto b/flyteidl/protos/flyteidl/core/artifact_id.proto index c00a176239..ab0fa86623 100644 --- a/flyteidl/protos/flyteidl/core/artifact_id.proto +++ b/flyteidl/protos/flyteidl/core/artifact_id.proto @@ -88,12 +88,3 @@ message ArtifactQuery { ArtifactBindingData binding = 4; } } - -message Trigger { - // This will be set to a launch plan type, but note that this is different than the actual launch plan type. - Identifier trigger_id = 1; - - // These are partial artifact IDs that will be triggered on - // Consider making these ArtifactQuery instead. - repeated core.ArtifactID triggers = 2; -}